diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 53431b94fb1..43a7280c302 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -52,7 +52,7 @@ jobs: stale-issue-label: stale stale-pr-label: stale exempt-issue-labels: 'Evergreen,Security,Bug,Proposal,Design Review,Improvement,Performance,Refactoring,Apache,Area - Automation/Static Analysis,Area - Batch Indexing,Area - Cache,Area - Deep Storage,Area - Dependencies,Area - Dependency Injection,Area - Dev,Area - Documentation,Area - Extension,Area - Kafka/Kinesis Indexing,Area - Lookups,Area - Metadata,Area - Metrics/Event Emitting,Area - Null Handling,Area - Operations,Area - Query UI,Area - Querying,Area - Router,Area - Segment Balancing/Coordination,Area - Segment Format and Ser/De,Area - SQL,Area - Testing,Area - Web Console,Area - Zookeeper/Curator,Compatibility,Contributions Welcome,Development Blocker,Ease of Use,Error handling,HTTP,Incompatible,Stable API' - exempt-pr-labels: 'Evergreen' + exempt-pr-labels: 'Evergreen,Area - Dependencies' exempt-milestones: true exempt-assignees: true ascending: true diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java new file mode 100644 index 00000000000..c3ebbe5122f --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java @@ -0,0 +1,193 @@ +/* + * 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.benchmark; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.jackson.AggregatorsModule; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.aggregation.SerializablePairLongString; +import org.apache.druid.query.aggregation.post.ConstantPostAggregator; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; +import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +public class GroupByDeserializationBenchmark +{ + + static { + NullHandling.initializeForTests(); + NestedDataModule.registerHandlersAndSerde(); + AggregatorsModule.registerComplexMetricsAndSerde(); + } + + @Param({"100", "1000"}) + private int numDimensions; + + @Param({"0", "0.25", "0.5", "0.75", "0.85", "0.95", "0.99", "1.0"}) + private double primitiveToComplexDimensionRatio; + + @Param({"json", "serializablePairLongString"}) + private String complexDimensionType; + + @Param({"true", "false"}) + private boolean backwardCompatibility; + + private GroupByQuery sqlQuery; + private String serializedRow; + private GroupByQueryQueryToolChest groupByQueryQueryToolChest; + private ObjectMapper decoratedMapper; + + @Setup(Level.Trial) + public void setup() throws JsonProcessingException + { + final ObjectMapper undecoratedMapper = TestHelper.makeJsonMapper(); + undecoratedMapper.registerModules(NestedDataModule.getJacksonModulesList()); + undecoratedMapper.registerModule(new AggregatorsModule()); + final Pair sqlQueryAndResultRow = sqlQueryAndResultRow( + numDimensions, + primitiveToComplexDimensionRatio, + complexDimensionType, + undecoratedMapper + ); + sqlQuery = sqlQueryAndResultRow.lhs; + serializedRow = sqlQueryAndResultRow.rhs; + + groupByQueryQueryToolChest = new GroupByQueryQueryToolChest( + null, + () -> new GroupByQueryConfig() + { + @Override + public boolean isIntermediateResultAsMapCompat() + { + return backwardCompatibility; + } + }, + null, + null + ); + + decoratedMapper = groupByQueryQueryToolChest.decorateObjectMapper(undecoratedMapper, sqlQuery); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void deserializeResultRows(Blackhole blackhole) throws JsonProcessingException + { + blackhole.consume(decoratedMapper.readValue(serializedRow, ResultRow.class)); + } + + private static Pair sqlQueryAndResultRow( + final int numDimensions, + final double primitiveToComplexDimensionRatio, + final String complexDimensionType, + final ObjectMapper mapper + ) throws JsonProcessingException + { + final int numPrimitiveDimensions = (int) Math.floor(primitiveToComplexDimensionRatio * numDimensions); + final int numComplexDimensions = numDimensions - numPrimitiveDimensions; + + final List dimensions = new ArrayList<>(); + final List rowList = new ArrayList<>(); + + // Add timestamp + rowList.add(DateTimes.of("2000").getMillis()); + + for (int i = 0; i < numPrimitiveDimensions; ++i) { + dimensions.add( + new DefaultDimensionSpec( + StringUtils.format("primitive%d", i), + StringUtils.format("primitive%d", i), + ColumnType.STRING + ) + ); + rowList.add("foo"); + } + + for (int i = 0; i < numComplexDimensions; ++i) { + dimensions.add( + new DefaultDimensionSpec( + StringUtils.format("complex%d", i), + StringUtils.format("complex%d", i), + ColumnType.ofComplex(complexDimensionType) + ) + ); + + // Serialized version of this object is a valid value for both json and long-string pair dimensions + rowList.add(new SerializablePairLongString(1L, "test")); + } + + // Add aggregator + rowList.add(100); + + // Add post aggregator + rowList.add(10.0); + + GroupByQuery query = GroupByQuery.builder() + .setDataSource("foo") + .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) + .setDimensions(dimensions) + .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) + .setPostAggregatorSpecs(Collections.singletonList(new ConstantPostAggregator( + "post", + 10 + ))) + .setContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true)) + .setGranularity(Granularities.DAY) + .build(); + + return Pair.of(query, mapper.writeValueAsString(rowList)); + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java index b7bfe56ff02..a33dbc0dcfc 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -262,12 +262,8 @@ public class ScanBenchmark rowsPerSegment ); - final ScanQueryConfig config = new ScanQueryConfig().setLegacy(false); factory = new ScanQueryRunnerFactory( - new ScanQueryQueryToolChest( - config, - DefaultGenericQueryMetricsFactory.instance() - ), + new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()), new ScanQueryEngine(), new ScanQueryConfig() ); diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index 98c27c4b2b8..87e92ab6fb1 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -92,21 +92,12 @@ public class NewestSegmentFirstPolicyBenchmark final String dataSource = DATA_SOURCE_PREFIX + i; compactionConfigs.put( dataSource, - new DataSourceCompactionConfig( - dataSource, - 0, - inputSegmentSizeBytes, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + DataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withTaskPriority(0) + .withInputSegmentSizeBytes(inputSegmentSizeBytes) + .build() ); } diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index 1b587cf4bac..275be9e63dc 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -474,7 +474,6 @@ The response shows an example report for a query. "agent_type", "timestamp" ], - "legacy": false, "context": { "finalize": false, "finalizeAggregations": false, diff --git a/docs/configuration/extensions.md b/docs/configuration/extensions.md index d396bc29000..bc7a05a4ae2 100644 --- a/docs/configuration/extensions.md +++ b/docs/configuration/extensions.md @@ -80,7 +80,7 @@ All of these community extensions can be downloaded using [pull-deps](../operati |aliyun-oss-extensions|Aliyun OSS deep storage |[link](../development/extensions-contrib/aliyun-oss-extensions.md)| |ambari-metrics-emitter|Ambari Metrics Emitter |[link](../development/extensions-contrib/ambari-metrics-emitter.md)| |druid-cassandra-storage|Apache Cassandra deep storage.|[link](../development/extensions-contrib/cassandra.md)| -|druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage and firehose.|[link](../development/extensions-contrib/cloudfiles.md)| +|druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage.|[link](../development/extensions-contrib/cloudfiles.md)| |druid-compressed-bigdecimal|Compressed Big Decimal Type | [link](../development/extensions-contrib/compressed-big-decimal.md)| |druid-ddsketch|Support for DDSketch approximate quantiles based on [DDSketch](https://github.com/datadog/sketches-java) | [link](../development/extensions-contrib/ddsketch-quantiles.md)| |druid-deltalake-extensions|Support for ingesting Delta Lake tables.|[link](../development/extensions-contrib/delta-lake.md)| diff --git a/docs/configuration/index.md b/docs/configuration/index.md index c627e9fd7f0..3b3c2711d3b 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -395,7 +395,6 @@ Metric monitoring is an essential part of Druid operations. The following monito |`org.apache.druid.java.util.metrics.CgroupCpuSetMonitor`|Reports CPU core/HT and memory node allocations as per the `cpuset` cgroup.| |`org.apache.druid.java.util.metrics.CgroupDiskMonitor`|Reports disk statistic as per the blkio cgroup.| |`org.apache.druid.java.util.metrics.CgroupMemoryMonitor`|Reports memory statistic as per the memory cgroup.| -|`org.apache.druid.server.metrics.EventReceiverFirehoseMonitor`|Reports how many events have been queued in the EventReceiverFirehose.| |`org.apache.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical services. Available only on Historical services.| |`org.apache.druid.server.metrics.SegmentStatsMonitor` | **EXPERIMENTAL** Reports statistics about segments on Historical services. Available only on Historical services. Not to be used when lazy loading is configured.| |`org.apache.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted.| @@ -607,7 +606,7 @@ the [HDFS input source](../ingestion/input-sources.md#hdfs-input-source). |Property|Possible values|Description|Default| |--------|---------------|-----------|-------| -|`druid.ingestion.hdfs.allowedProtocols`|List of protocols|Allowed protocols for the HDFS input source and HDFS firehose.|`["hdfs"]`| +|`druid.ingestion.hdfs.allowedProtocols`|List of protocols|Allowed protocols for the HDFS input source.|`["hdfs"]`| #### HTTP input source @@ -616,7 +615,7 @@ the [HTTP input source](../ingestion/input-sources.md#http-input-source). |Property|Possible values|Description|Default| |--------|---------------|-----------|-------| -|`druid.ingestion.http.allowedProtocols`|List of protocols|Allowed protocols for the HTTP input source and HTTP firehose.|`["http", "https"]`| +|`druid.ingestion.http.allowedProtocols`|List of protocols|Allowed protocols for the HTTP input source.|`["http", "https"]`| ### External data access security configuration @@ -1501,7 +1500,6 @@ Additional Peon configs include: |`druid.peon.mode`|One of `local` or `remote`. Setting this property to `local` means you intend to run the Peon as a standalone process which is not recommended.|`remote`| |`druid.indexer.task.baseDir`|Base temporary working directory.|`System.getProperty("java.io.tmpdir")`| |`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|`${druid.indexer.task.baseDir}/persistent/task`| -|`druid.indexer.task.batchProcessingMode`| Batch ingestion tasks have three operating modes to control construction and tracking for intermediary segments: `OPEN_SEGMENTS`, `CLOSED_SEGMENTS`, and `CLOSED_SEGMENT_SINKS`. `OPEN_SEGMENTS` uses the streaming ingestion code path and performs a `mmap` on intermediary segments to build a timeline to make these segments available to realtime queries. Batch ingestion doesn't require intermediary segments, so the default mode, `CLOSED_SEGMENTS`, eliminates `mmap` of intermediary segments. `CLOSED_SEGMENTS` mode still tracks the entire set of segments in heap. The `CLOSED_SEGMENTS_SINKS` mode is the most aggressive configuration and should have the smallest memory footprint. It eliminates in-memory tracking and `mmap` of intermediary segments produced during segment creation. `CLOSED_SEGMENTS_SINKS` mode isn't as well tested as other modes so is currently considered experimental. You can use `OPEN_SEGMENTS` mode if problems occur with the 2 newer modes. |`CLOSED_SEGMENTS`| |`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|`org.apache.hadoop:hadoop-client-api:3.3.6`, `org.apache.hadoop:hadoop-client-runtime:3.3.6`| |`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|75000| |`druid.indexer.task.directoryLockTimeout`|Wait this long for zombie Peons to exit before giving up on their replacements.|PT10M| diff --git a/docs/development/extensions-contrib/cloudfiles.md b/docs/development/extensions-contrib/cloudfiles.md index 83a1d0c7e10..d4e7592ee7f 100644 --- a/docs/development/extensions-contrib/cloudfiles.md +++ b/docs/development/extensions-contrib/cloudfiles.md @@ -40,59 +40,3 @@ To use this Apache Druid extension, [include](../../configuration/extensions.md# |`druid.cloudfiles.apiKey`||Rackspace Cloud API key.|Must be set.| |`druid.cloudfiles.provider`|rackspace-cloudfiles-us,rackspace-cloudfiles-uk|Name of the provider depending on the region.|Must be set.| |`druid.cloudfiles.useServiceNet`|true,false|Whether to use the internal service net.|true| - -## Firehose - - - -#### StaticCloudFilesFirehose - -This firehose ingests events, similar to the StaticAzureBlobStoreFirehose, but from Rackspace's Cloud Files. - -Data is newline delimited, with one JSON object per line and parsed as per the `InputRowParser` configuration. - -The storage account is shared with the one used for Rackspace's Cloud Files deep storage functionality, but blobs can be in a different region and container. - -As with the Azure blobstore, it is assumed to be gzipped if the extension ends in .gz - -This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native-batch.md). -Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object. - -Sample spec: - -```json -"firehose" : { - "type" : "static-cloudfiles", - "blobs": [ - { - "region": "DFW" - "container": "container", - "path": "/path/to/your/file.json" - }, - { - "region": "ORD" - "container": "anothercontainer", - "path": "/another/path.json" - } - ] -} -``` -This firehose provides caching and prefetching features. In IndexTask, a firehose can be read twice if intervals or -shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow. - -|property|description|default|required?| -|--------|-----------|-------|---------| -|type|This should be `static-cloudfiles`.|N/A|yes| -|blobs|JSON array of Cloud Files blobs.|N/A|yes| -|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache.|1073741824|no| -|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|no| -|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|no| -|fetchTimeout|Timeout for fetching a Cloud Files object.|60000|no| -|maxFetchRetry|Maximum retry for fetching a Cloud Files object.|3|no| - -Cloud Files Blobs: - -|property|description|default|required?| -|--------|-----------|-------|---------| -|container|Name of the Cloud Files container|N/A|yes| -|path|The path where data is located.|N/A|yes| diff --git a/docs/development/extensions-core/postgresql.md b/docs/development/extensions-core/postgresql.md index e0d9337b2cc..919bf372b84 100644 --- a/docs/development/extensions-core/postgresql.md +++ b/docs/development/extensions-core/postgresql.md @@ -87,7 +87,7 @@ In most cases, the configuration options map directly to the [postgres JDBC conn | `druid.metadata.postgres.ssl.sslPasswordCallback` | The classname of the SSL password provider. | none | no | | `druid.metadata.postgres.dbTableSchema` | druid meta table schema | `public` | no | -### PostgreSQL Firehose +### PostgreSQL InputSource The PostgreSQL extension provides an implementation of an [SQL input source](../../ingestion/input-sources.md) which can be used to ingest data into Druid from a PostgreSQL database. diff --git a/docs/development/overview.md b/docs/development/overview.md index 5ff77af07cf..11c67ddfa2c 100644 --- a/docs/development/overview.md +++ b/docs/development/overview.md @@ -53,8 +53,17 @@ Most of the coordination logic for (real-time) ingestion is in the Druid indexin ## Real-time Ingestion -Druid loads data through `FirehoseFactory.java` classes. Firehoses often wrap other firehoses, where, similar to the design of the -query runners, each firehose adds a layer of logic, and the persist and hand-off logic is in `RealtimePlumber.java`. +Druid streaming tasks are based on the 'seekable stream' classes such as `SeekableStreamSupervisor.java`, +`SeekableStreamIndexTask.java`, and `SeekableStreamIndexTaskRunner.java`. The data processing happens through +`StreamAppenderator.java`, and the persist and hand-off logic is in `StreamAppenderatorDriver.java`. + +## Native Batch Ingestion + +Druid native batch ingestion main task types are based on `AbstractBatchTask.java` and `AbstractBatchSubtask.java`. +Parallel processing uses `ParallelIndexSupervisorTask.java`, which spawns subtasks to perform various operations such +as data analysis and partitioning depending on the task specification. Segment generation happens in +`SinglePhaseSubTask.java`, `PartialHashSegmentGenerateTask.java`, or `PartialRangeSegmentGenerateTask.java` through +`BatchAppenderator`, and the persist and hand-off logic is in `BatchAppenderatorDriver.java`. ## Hadoop-based Batch Ingestion diff --git a/docs/ingestion/kafka-ingestion.md b/docs/ingestion/kafka-ingestion.md index 5d14b1ad6ba..5a3ac7f29c4 100644 --- a/docs/ingestion/kafka-ingestion.md +++ b/docs/ingestion/kafka-ingestion.md @@ -254,7 +254,6 @@ The Kinesis indexing service supports the following values for `inputFormat`: * `json` * `kafka` * `avro_stream` -* `avro_ocf` * `protobuf` You can use `parser` to read [`thrift`](../development/extensions-contrib/thrift.md) formats. diff --git a/docs/ingestion/kinesis-ingestion.md b/docs/ingestion/kinesis-ingestion.md index fb4bfde235a..3b4c5de8548 100644 --- a/docs/ingestion/kinesis-ingestion.md +++ b/docs/ingestion/kinesis-ingestion.md @@ -143,7 +143,6 @@ The Kinesis indexing service supports the following values for `inputFormat`: * `tvs` * `json` * `avro_stream` -* `avro_ocf` * `protobuf` You can use `parser` to read [`thrift`](../development/extensions-contrib/thrift.md) formats. diff --git a/docs/ingestion/native-batch-firehose.md b/docs/ingestion/native-batch-firehose.md index db2b3e8779e..16e9634ff29 100644 --- a/docs/ingestion/native-batch-firehose.md +++ b/docs/ingestion/native-batch-firehose.md @@ -24,319 +24,5 @@ sidebar_label: "Firehose (deprecated)" --> :::info - Firehose ingestion is deprecated. See [Migrate from firehose to input source ingestion](../operations/migrate-from-firehose-ingestion.md) for instructions on migrating from firehose ingestion to using native batch ingestion input sources. + Firehose ingestion has been removed in Druid 26.0. See [Migrate from firehose to input source ingestion](../operations/migrate-from-firehose-ingestion.md) for instructions on migrating from firehose ingestion to using native batch ingestion input sources. ::: - -There are several firehoses readily available in Druid, some are meant for examples, others can be used directly in a production environment. - -## StaticS3Firehose - -You need to include the [`druid-s3-extensions`](../development/extensions-core/s3.md) as an extension to use the StaticS3Firehose. - -This firehose ingests events from a predefined list of S3 objects. -This firehose is _splittable_ and can be used by the [Parallel task](./native-batch.md). -Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object. - -Sample spec: - -```json -"firehose" : { - "type" : "static-s3", - "uris": ["s3://foo/bar/file.gz", "s3://bar/foo/file2.gz"] -} -``` - -This firehose provides caching and prefetching features. In the Simple task, a firehose can be read twice if intervals or -shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow. -Note that prefetching or caching isn't that useful in the Parallel task. - -|property|description|default|required?| -|--------|-----------|-------|---------| -|type|This should be `static-s3`.|None|yes| -|uris|JSON array of URIs where s3 files to be ingested are located.|None|`uris` or `prefixes` must be set| -|prefixes|JSON array of URI prefixes for the locations of s3 files to be ingested.|None|`uris` or `prefixes` must be set| -|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|no| -|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|no| -|prefetchTriggerBytes|Threshold to trigger prefetching s3 objects.|maxFetchCapacityBytes / 2|no| -|fetchTimeout|Timeout for fetching an s3 object.|60000|no| -|maxFetchRetry|Maximum retry for fetching an s3 object.|3|no| - -## StaticGoogleBlobStoreFirehose - -You need to include the [`druid-google-extensions`](../development/extensions-core/google.md) as an extension to use the StaticGoogleBlobStoreFirehose. - -This firehose ingests events, similar to the StaticS3Firehose, but from an Google Cloud Store. - -As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz - -This firehose is _splittable_ and can be used by the [Parallel task](./native-batch.md). -Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object. - -Sample spec: - -```json -"firehose" : { - "type" : "static-google-blobstore", - "blobs": [ - { - "bucket": "foo", - "path": "/path/to/your/file.json" - }, - { - "bucket": "bar", - "path": "/another/path.json" - } - ] -} -``` - -This firehose provides caching and prefetching features. In the Simple task, a firehose can be read twice if intervals or -shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow. -Note that prefetching or caching isn't that useful in the Parallel task. - -|property|description|default|required?| -|--------|-----------|-------|---------| -|type|This should be `static-google-blobstore`.|None|yes| -|blobs|JSON array of Google Blobs.|None|yes| -|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|no| -|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|no| -|prefetchTriggerBytes|Threshold to trigger prefetching Google Blobs.|maxFetchCapacityBytes / 2|no| -|fetchTimeout|Timeout for fetching a Google Blob.|60000|no| -|maxFetchRetry|Maximum retry for fetching a Google Blob.|3|no| - -Google Blobs: - -|property|description|default|required?| -|--------|-----------|-------|---------| -|bucket|Name of the Google Cloud bucket|None|yes| -|path|The path where data is located.|None|yes| - -## HDFSFirehose - -You need to include the [`druid-hdfs-storage`](../development/extensions-core/hdfs.md) as an extension to use the HDFSFirehose. - -This firehose ingests events from a predefined list of files from the HDFS storage. -This firehose is _splittable_ and can be used by the [Parallel task](./native-batch.md). -Since each split represents an HDFS file, each worker task of `index_parallel` will read files. - -Sample spec: - -```json -"firehose" : { - "type" : "hdfs", - "paths": "/foo/bar,/foo/baz" -} -``` - -This firehose provides caching and prefetching features. During native batch indexing, a firehose can be read twice if -`intervals` are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scanning -of files is slow. -Note that prefetching or caching isn't that useful in the Parallel task. - -|Property|Description|Default| -|--------|-----------|-------| -|type|This should be `hdfs`.|none (required)| -|paths|HDFS paths. Can be either a JSON array or comma-separated string of paths. Wildcards like `*` are supported in these paths.|none (required)| -|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824| -|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824| -|prefetchTriggerBytes|Threshold to trigger prefetching files.|maxFetchCapacityBytes / 2| -|fetchTimeout|Timeout for fetching each file.|60000| -|maxFetchRetry|Maximum number of retries for fetching each file.|3| - -You can also ingest from other storage using the HDFS firehose if the HDFS client supports that storage. -However, if you want to ingest from cloud storage, consider using the service-specific input source for your data storage. -If you want to use a non-hdfs protocol with the HDFS firehose, you need to include the protocol you want -in `druid.ingestion.hdfs.allowedProtocols`. See [HDFS firehose security configuration](../configuration/index.md#hdfs-input-source) for more details. - -## LocalFirehose - -This Firehose can be used to read the data from files on local disk, and is mainly intended for proof-of-concept testing, and works with `string` typed parsers. -This Firehose is _splittable_ and can be used by [native parallel index tasks](native-batch.md). -Since each split represents a file in this Firehose, each worker task of `index_parallel` will read a file. -A sample local Firehose spec is shown below: - -```json -{ - "type": "local", - "filter" : "*.csv", - "baseDir": "/data/directory" -} -``` - -|property|description|required?| -|--------|-----------|---------| -|type|This should be "local".|yes| -|filter|A wildcard filter for files. See [here](http://commons.apache.org/proper/commons-io/apidocs/org/apache/commons/io/filefilter/WildcardFileFilter) for more information.|yes| -|baseDir|directory to search recursively for files to be ingested. |yes| - - - -## HttpFirehose - -This Firehose can be used to read the data from remote sites via HTTP, and works with `string` typed parsers. -This Firehose is _splittable_ and can be used by [native parallel index tasks](native-batch.md). -Since each split represents a file in this Firehose, each worker task of `index_parallel` will read a file. -A sample HTTP Firehose spec is shown below: - -```json -{ - "type": "http", - "uris": ["http://example.com/uri1", "http://example2.com/uri2"] -} -``` - -You can only use protocols listed in the `druid.ingestion.http.allowedProtocols` property as HTTP firehose input sources. -The `http` and `https` protocols are allowed by default. See [HTTP firehose security configuration](../configuration/index.md#http-input-source) for more details. - -The below configurations can be optionally used if the URIs specified in the spec require a Basic Authentication Header. -Omitting these fields from your spec will result in HTTP requests with no Basic Authentication Header. - -|property|description|default| -|--------|-----------|-------| -|httpAuthenticationUsername|Username to use for authentication with specified URIs|None| -|httpAuthenticationPassword|PasswordProvider to use with specified URIs|None| - -Example with authentication fields using the DefaultPassword provider (this requires the password to be in the ingestion spec): - -```json -{ - "type": "http", - "uris": ["http://example.com/uri1", "http://example2.com/uri2"], - "httpAuthenticationUsername": "username", - "httpAuthenticationPassword": "password123" -} -``` - -You can also use the other existing Druid PasswordProviders. Here is an example using the EnvironmentVariablePasswordProvider: - -```json -{ - "type": "http", - "uris": ["http://example.com/uri1", "http://example2.com/uri2"], - "httpAuthenticationUsername": "username", - "httpAuthenticationPassword": { - "type": "environment", - "variable": "HTTP_FIREHOSE_PW" - } -} -``` - -The below configurations can optionally be used for tuning the Firehose performance. -Note that prefetching or caching isn't that useful in the Parallel task. - -|property|description|default| -|--------|-----------|-------| -|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824| -|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824| -|prefetchTriggerBytes|Threshold to trigger prefetching HTTP objects.|maxFetchCapacityBytes / 2| -|fetchTimeout|Timeout for fetching an HTTP object.|60000| -|maxFetchRetry|Maximum retries for fetching an HTTP object.|3| - - - -## IngestSegmentFirehose - -This Firehose can be used to read the data from existing druid segments, potentially using a new schema and changing the name, dimensions, metrics, rollup, etc. of the segment. -This Firehose is _splittable_ and can be used by [native parallel index tasks](native-batch.md). -This firehose will accept any type of parser, but will only utilize the list of dimensions and the timestamp specification. - A sample ingest Firehose spec is shown below: - -```json -{ - "type": "ingestSegment", - "dataSource": "wikipedia", - "interval": "2013-01-01/2013-01-02" -} -``` - -|property|description|required?| -|--------|-----------|---------| -|type|This should be "ingestSegment".|yes| -|dataSource|A String defining the data source to fetch rows from, very similar to a table in a relational database|yes| -|interval|A String representing the ISO-8601 interval. This defines the time range to fetch the data over.|yes| -|dimensions|The list of dimensions to select. If left empty, no dimensions are returned. If left null or not defined, all dimensions are returned. |no| -|metrics|The list of metrics to select. If left empty, no metrics are returned. If left null or not defined, all metrics are selected.|no| -|filter| See [Filters](../querying/filters.md)|no| -|maxInputSegmentBytesPerTask|Deprecated. Use [Segments Split Hint Spec](./native-batch.md#segments-split-hint-spec) instead. When used with the native parallel index task, the maximum number of bytes of input segments to process in a single task. If a single segment is larger than this number, it will be processed by itself in a single task (input segments are never split across tasks). Defaults to 150MB.|no| - - - -## SqlFirehose - -This Firehose can be used to ingest events residing in an RDBMS. The database connection information is provided as part of the ingestion spec. -For each query, the results are fetched locally and indexed. -If there are multiple queries from which data needs to be indexed, queries are prefetched in the background, up to `maxFetchCapacityBytes` bytes. -This Firehose is _splittable_ and can be used by [native parallel index tasks](native-batch.md). -This firehose will accept any type of parser, but will only utilize the list of dimensions and the timestamp specification. See the extension documentation for more detailed ingestion examples. - -Requires one of the following extensions: - * [MySQL Metadata Store](../development/extensions-core/mysql.md). - * [PostgreSQL Metadata Store](../development/extensions-core/postgresql.md). - - -```json -{ - "type": "sql", - "database": { - "type": "mysql", - "connectorConfig": { - "connectURI": "jdbc:mysql://host:port/schema", - "user": "user", - "password": "password" - } - }, - "sqls": ["SELECT * FROM table1", "SELECT * FROM table2"] -} -``` - -|property|description|default|required?| -|--------|-----------|-------|---------| -|type|This should be "sql".||Yes| -|database|Specifies the database connection details. The database type corresponds to the extension that supplies the `connectorConfig` support. The specified extension must be loaded into Druid:

  • [mysql-metadata-storage](../development/extensions-core/mysql.md) for `mysql`
  • [postgresql-metadata-storage](../development/extensions-core/postgresql.md) extension for `postgresql`.


You can selectively allow JDBC properties in `connectURI`. See [JDBC connections security config](../configuration/index.md#jdbc-connections-to-external-databases) for more details.||Yes| -|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|No| -|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|No| -|prefetchTriggerBytes|Threshold to trigger prefetching SQL result objects.|maxFetchCapacityBytes / 2|No| -|fetchTimeout|Timeout for fetching the result set.|60000|No| -|foldCase|Toggle case folding of database column names. This may be enabled in cases where the database returns case insensitive column names in query results.|false|No| -|sqls|List of SQL queries where each SQL query would retrieve the data to be indexed.||Yes| - -### Database - -|property|description|default|required?| -|--------|-----------|-------|---------| -|type|The type of database to query. Valid values are `mysql` and `postgresql`_||Yes| -|connectorConfig|Specify the database connection properties via `connectURI`, `user` and `password`||Yes| - -## InlineFirehose - -This Firehose can be used to read the data inlined in its own spec. -It can be used for demos or for quickly testing out parsing and schema, and works with `string` typed parsers. -A sample inline Firehose spec is shown below: - -```json -{ - "type": "inline", - "data": "0,values,formatted\n1,as,CSV" -} -``` - -|property|description|required?| -|--------|-----------|---------| -|type|This should be "inline".|yes| -|data|Inlined data to ingest.|yes| - -## CombiningFirehose - -This Firehose can be used to combine and merge data from a list of different Firehoses. - -```json -{ - "type": "combining", - "delegates": [ { firehose1 }, { firehose2 }, ... ] -} -``` - -|property|description|required?| -|--------|-----------|---------| -|type|This should be "combining"|yes| -|delegates|List of Firehoses to combine data from|yes| \ No newline at end of file diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index 1d37169684e..b5be94ba4ba 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -428,15 +428,6 @@ These metrics are available only when `druid.zk.service.enabled = true`. |`zk/connected`|Indicator of connection status. `1` for connected, `0` for disconnected. Emitted once per monitor period.|None|1| |`zk/reconnect/time`|Amount of time, in milliseconds, that a server was disconnected from ZooKeeper before reconnecting. Emitted on reconnection. Not emitted if connection to ZooKeeper is permanently lost, because in this case, there is no reconnection.|None|Not present| -### EventReceiverFirehose - -The following metric is only available if the `EventReceiverFirehoseMonitor` module is included. - -|Metric|Description|Dimensions|Normal value| -|------|-----------|----------|------------| -|`ingest/events/buffered`|Number of events queued in the `EventReceiverFirehose` buffer.|`serviceName`, `dataSource`, `taskId`, `taskType`, `bufferCapacity`|Equal to the current number of events in the buffer queue.| -|`ingest/bytes/received`|Number of bytes received by the `EventReceiverFirehose`.|`serviceName`, `dataSource`, `taskId`, `taskType`|Varies| - ## Sys [Deprecated] > SysMonitor is now deprecated and will be removed in future releases. diff --git a/docs/operations/migrate-from-firehose-ingestion.md b/docs/operations/migrate-from-firehose-ingestion.md index f470324b7f4..540685a717f 100644 --- a/docs/operations/migrate-from-firehose-ingestion.md +++ b/docs/operations/migrate-from-firehose-ingestion.md @@ -23,9 +23,7 @@ sidebar_label: "Migrate from firehose" ~ under the License. --> -Apache deprecated support for Druid firehoses in version 0.17. Support for firehose ingestion will be removed in version 26.0. - -If you're using a firehose for batch ingestion, we strongly recommend that you follow the instructions on this page to transition to using native batch ingestion input sources as soon as possible. +Apache deprecated support for Druid firehoses in version 0.17. Support for firehose ingestion was removed in version 26.0. Firehose ingestion doesn't work with newer Druid versions, so you must be using an ingestion spec with a defined input source before you upgrade. diff --git a/docs/operations/request-logging.md b/docs/operations/request-logging.md index 3bccfd3ef9a..6ce65c0421e 100644 --- a/docs/operations/request-logging.md +++ b/docs/operations/request-logging.md @@ -217,7 +217,6 @@ The following shows an example log emitter output: "user", "v0" ], - "legacy": false, "context": { "populateCache": false, diff --git a/docs/operations/security-overview.md b/docs/operations/security-overview.md index 279a1327b97..96389c8a72c 100644 --- a/docs/operations/security-overview.md +++ b/docs/operations/security-overview.md @@ -52,7 +52,7 @@ The following recommendations apply to the network where Druid runs: The following recommendation applies to Druid's authorization and authentication model: * Only grant `WRITE` permissions to any `DATASOURCE` to trusted users. Druid's trust model assumes those users have the same privileges as the operating system user that runs the web console process. Additionally, users with `WRITE` permissions can make changes to datasources and they have access to both task and supervisor update (POST) APIs which may affect ingestion. * Only grant `STATE READ`, `STATE WRITE`, `CONFIG WRITE`, and `DATASOURCE WRITE` permissions to highly-trusted users. These permissions allow users to access resources on behalf of the Druid server process regardless of the datasource. -* If your Druid client application allows less-trusted users to control the input source or firehose of an ingestion task, validate the URLs from the users. It is possible to point unchecked URLs to other locations and resources within your network or local file system. +* If your Druid client application allows less-trusted users to control the input source of an ingestion task, validate the URLs from the users. It is possible to point unchecked URLs to other locations and resources within your network or local file system. ## Enable TLS diff --git a/docs/querying/arrays.md b/docs/querying/arrays.md index a7eebaa32af..d9a9f391188 100644 --- a/docs/querying/arrays.md +++ b/docs/querying/arrays.md @@ -71,46 +71,10 @@ The following shows an example `dimensionsSpec` for native ingestion of the data ### SQL-based ingestion -#### `arrayIngestMode` - -Arrays can be inserted with [SQL-based ingestion](../multi-stage-query/index.md) when you include the query context -parameter `arrayIngestMode: array`. - -When `arrayIngestMode` is `array`, SQL ARRAY types are stored using Druid array columns. This is recommended for new -tables. - -When `arrayIngestMode` is `mvd`, SQL `VARCHAR ARRAY` are implicitly wrapped in [`ARRAY_TO_MV`](sql-functions.md#array_to_mv). -This causes them to be stored as [multi-value strings](multi-value-dimensions.md), using the same `STRING` column type -as regular scalar strings. SQL `BIGINT ARRAY` and `DOUBLE ARRAY` cannot be loaded under `arrayIngestMode: mvd`. This -is the default behavior when `arrayIngestMode` is not provided in your query context, although the default behavior -may change to `array` in a future release. - -When `arrayIngestMode` is `none`, Druid throws an exception when trying to store any type of arrays. This mode is most -useful when set in the system default query context with `druid.query.default.context.arrayIngestMode = none`, in cases -where the cluster administrator wants SQL query authors to explicitly provide one or the other in their query context. - -The following table summarizes the differences in SQL ARRAY handling between `arrayIngestMode: array` and -`arrayIngestMode: mvd`. - -| SQL type | Stored type when `arrayIngestMode: array` | Stored type when `arrayIngestMode: mvd` (default) | -|---|---|---| -|`VARCHAR ARRAY`|`ARRAY`|[multi-value `STRING`](multi-value-dimensions.md)| -|`BIGINT ARRAY`|`ARRAY`|not possible (validation error)| -|`DOUBLE ARRAY`|`ARRAY`|not possible (validation error)| - -In either mode, you can explicitly wrap string arrays in `ARRAY_TO_MV` to cause them to be stored as -[multi-value strings](multi-value-dimensions.md). - -When validating a SQL INSERT or REPLACE statement that contains arrays, Druid checks whether the statement would lead -to mixing string arrays and multi-value strings in the same column. If this condition is detected, the statement fails -validation unless the column is named under the `skipTypeVerification` context parameter. This parameter can be either -a comma-separated list of column names, or a JSON array in string form. This validation is done to prevent accidentally -mixing arrays and multi-value strings in the same column. +Arrays can be inserted with [SQL-based ingestion](../multi-stage-query/index.md). #### Examples -Set [`arrayIngestMode: array`](#arrayingestmode) in your query context to run the following examples. - ```sql REPLACE INTO "array_example" OVERWRITE ALL WITH "ext" AS ( @@ -169,6 +133,35 @@ GROUP BY 1,2,3,4,5 PARTITIONED BY DAY ``` +#### `arrayIngestMode` + +For seamless backwards compatible behavior with Druid versions older than 31, there is an `arrayIngestMode` query context flag. + +When `arrayIngestMode` is `array`, SQL ARRAY types are stored using Druid array columns. This is recommended for new +tables and the default configuration for Druid 31 and newer. + +When `arrayIngestMode` is `mvd` (legacy), SQL `VARCHAR ARRAY` are implicitly wrapped in [`ARRAY_TO_MV`](sql-functions.md#array_to_mv). +This causes them to be stored as [multi-value strings](multi-value-dimensions.md), using the same `STRING` column type +as regular scalar strings. SQL `BIGINT ARRAY` and `DOUBLE ARRAY` cannot be loaded under `arrayIngestMode: mvd`. This +mode is not recommended and will be removed in a future release, but provided for backwards compatibility. + +The following table summarizes the differences in SQL ARRAY handling between `arrayIngestMode: array` and +`arrayIngestMode: mvd`. + +| SQL type | Stored type when `arrayIngestMode: array` (default) | Stored type when `arrayIngestMode: mvd` | +|---|---|---| +|`VARCHAR ARRAY`|`ARRAY`|[multi-value `STRING`](multi-value-dimensions.md)| +|`BIGINT ARRAY`|`ARRAY`|not possible (validation error)| +|`DOUBLE ARRAY`|`ARRAY`|not possible (validation error)| + +In either mode, you can explicitly wrap string arrays in `ARRAY_TO_MV` to cause them to be stored as +[multi-value strings](multi-value-dimensions.md). + +When validating a SQL INSERT or REPLACE statement that contains arrays, Druid checks whether the statement would lead +to mixing string arrays and multi-value strings in the same column. If this condition is detected, the statement fails +validation unless the column is named under the `skipTypeVerification` context parameter. This parameter can be either +a comma-separated list of column names, or a JSON array in string form. This validation is done to prevent accidentally +mixing arrays and multi-value strings in the same column. ## Querying arrays @@ -284,9 +277,9 @@ Avoid confusing string arrays with [multi-value dimensions](multi-value-dimensio Use care during ingestion to ensure you get the type you want. -To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays and set the context parameter `"arrayIngestMode": "array"`. Arrays may contain strings or numbers. +To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays. Arrays may contain strings or numbers. -To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion), which ensures you get multi-value dimensions in any `arrayIngestMode`. Multi-value dimensions can only contain strings. +To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion), which ensures you get multi-value dimensions. Multi-value dimensions can only contain strings. You can tell which type you have by checking the `INFORMATION_SCHEMA.COLUMNS` table, using a query like: diff --git a/docs/querying/lookups.md b/docs/querying/lookups.md index a22fbf03928..05176f229b1 100644 --- a/docs/querying/lookups.md +++ b/docs/querying/lookups.md @@ -50,6 +50,10 @@ Other lookup types are available as extensions, including: - Globally cached lookups from local files, remote URIs, or JDBC through [lookups-cached-global](./lookups-cached-global.md). - Globally cached lookups from a Kafka topic through [kafka-extraction-namespace](./kafka-extraction-namespace.md). +:::info +[Multi-value dimensions](multi-value-dimensions.md) (MVDs) are not supported as keys in lookups. For example, to map the MVD `["A", "B", "C"]` to the value `x` in your lookup, flatten the MVD and map each element of the MVD to the value. Your lookup will have separate key-value pairs for each element of the MVD: `"A": "x"`, `"B": "x"`, and `"C": "x"`. +::: + Query Syntax ------------ diff --git a/docs/querying/multi-value-dimensions.md b/docs/querying/multi-value-dimensions.md index 1ce3a618dac..e41e8630ebb 100644 --- a/docs/querying/multi-value-dimensions.md +++ b/docs/querying/multi-value-dimensions.md @@ -507,9 +507,9 @@ Avoid confusing string arrays with [multi-value dimensions](multi-value-dimensio Use care during ingestion to ensure you get the type you want. -To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays and set the context parameter [`"arrayIngestMode": "array"`](arrays.md#arrayingestmode). Arrays may contain strings or numbers. +To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays. Arrays may contain strings or numbers. -To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion), which ensures you get multi-value dimensions in any [`arrayIngestMode`](arrays.md#arrayingestmode). Multi-value dimensions can only contain strings. +To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion). Multi-value dimensions can only contain strings. You can tell which type you have by checking the `INFORMATION_SCHEMA.COLUMNS` table, using a query like: diff --git a/docs/querying/scan-query.md b/docs/querying/scan-query.md index d758450715e..503664633ba 100644 --- a/docs/querying/scan-query.md +++ b/docs/querying/scan-query.md @@ -66,7 +66,6 @@ The following are the main parameters for Scan queries: |limit|How many rows to return. If not specified, all rows will be returned.|no| |offset|Skip this many rows when returning results. Skipped rows will still need to be generated internally and then discarded, meaning that raising offsets to high values can cause queries to use additional resources.

Together, "limit" and "offset" can be used to implement pagination. However, note that if the underlying datasource is modified in between page fetches in ways that affect overall query results, then the different pages will not necessarily align with each other.|no| |order|The ordering of returned rows based on timestamp. "ascending", "descending", and "none" (default) are supported. Currently, "ascending" and "descending" are only supported for queries where the `__time` column is included in the `columns` field and the requirements outlined in the [time ordering](#time-ordering) section are met.|none| -|legacy|Return results consistent with the legacy "scan-query" contrib extension. Defaults to the value set by `druid.query.scan.legacy`, which in turn defaults to false. See [Legacy mode](#legacy-mode) for details.|no| |context|An additional JSON Object which can be used to specify certain flags (see the `query context properties` section below).|no| ## Example results @@ -159,14 +158,14 @@ The format of the result when resultFormat equals `compactedList`: ## Time ordering -The Scan query currently supports ordering based on timestamp for non-legacy queries. Note that using time ordering -will yield results that do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore, -time ordering is only supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForOrdering` -rows **or** all segments scanned have fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` partitions. Also, -time ordering is not supported for queries issued directly to historicals unless a list of segments is specified. The -reasoning behind these limitations is that the implementation of time ordering uses two strategies that can consume too -much heap memory if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on -query result set limit and the number of segments being scanned. +The Scan query currently supports ordering based on timestamp. Note that using time ordering will yield results that +do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore, time ordering is only +supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForOrdering` rows **or** all segments +scanned have fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` partitions. Also, time ordering is not +supported for queries issued directly to historicals unless a list of segments is specified. The reasoning behind +these limitations is that the implementation of time ordering uses two strategies that can consume too much heap memory +if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on query result set +limit and the number of segments being scanned. 1. Priority Queue: Each segment on a Historical is opened sequentially. Every row is added to a bounded priority queue which is ordered by timestamp. For every row above the result set limit, the row with the earliest (if descending) @@ -187,21 +186,6 @@ configurable and can be tuned based on hardware specs and number of dimensions b can also be overridden using the `maxRowsQueuedForOrdering` and `maxSegmentPartitionsOrderedInMemory` properties in the query context (see the Query Context Properties section). -## Legacy mode - -The Scan query supports a legacy mode designed for protocol compatibility with the former scan-query contrib extension. -In legacy mode you can expect the following behavior changes: - -- The `__time` column is returned as `"timestamp"` rather than `"__time"`. This will take precedence over any other column -you may have that is named `"timestamp"`. -- The `__time` column is included in the list of columns even if you do not specifically ask for it. -- Timestamps are returned as ISO8601 time strings rather than integers (milliseconds since 1970-01-01 00:00:00 UTC). - -Legacy mode can be triggered either by passing `"legacy" : true` in your query JSON, or by setting -`druid.query.scan.legacy = true` on your Druid processes. If you were previously using the scan-query contrib extension, -the best way to migrate is to activate legacy mode during a rolling upgrade, then switch it off after the upgrade -is complete. - ## Configuration Properties Configuration properties: @@ -210,7 +194,6 @@ Configuration properties: |--------|-----------|------|-------| |druid.query.scan.maxRowsQueuedForOrdering|The maximum number of rows returned when time ordering is used|An integer in [1, 2147483647]|100000| |druid.query.scan.maxSegmentPartitionsOrderedInMemory|The maximum number of segments scanned per historical when time ordering is used|An integer in [1, 2147483647]|50| -|druid.query.scan.legacy|Whether legacy mode should be turned on for Scan queries|true or false|false| ## Query context properties @@ -228,3 +211,7 @@ Sample query context JSON object: "maxSegmentPartitionsOrderedInMemory": 100 } ``` + +## Legacy mode + +In older versions of Druid, the scan query supported a legacy mode designed for protocol compatibility with the former scan-query contrib extension from versions of Druid older than 0.11. This mode has been removed. diff --git a/docs/querying/sql-functions.md b/docs/querying/sql-functions.md index 7151c23b918..1b2f68c7a89 100644 --- a/docs/querying/sql-functions.md +++ b/docs/querying/sql-functions.md @@ -27,26 +27,69 @@ sidebar_label: "All functions" Apache Druid supports two query languages: Druid SQL and [native queries](querying.md). This document describes the SQL language. ::: + +This page provides a reference of Apache Druid® SQL functions in alphabetical order. For more details on a function, refer to the following: +* [Aggregation functions](sql-aggregations.md) +* [Array functions](sql-array-functions.md) +* [JSON functions](sql-json-functions.md) +* [Multi-value string functions](sql-multivalue-string-functions.md) +* [Scalar functions](sql-scalar.md) +* [Window functions](sql-window-functions.md) - -This page provides a reference of all Druid SQL functions in alphabetical order. -Click the linked function type for documentation on a particular function. +The examples on this page use the following example datasources: +* `flight-carriers` using `FlightCarrierOnTime (1 month)` +* `taxi-trips` using `NYC Taxi cabs (3 files)` ## ABS -`ABS()` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Calculates the absolute value of a numeric expression. +* **Syntax:** `ABS()` +* **Function type:** Scalar, numeric + +
Example + +The following example applies the ABS function to the `ArrDelay` column from the `flight-carriers` datasource. + +```sql +SELECT + "ArrDelay" AS "arrival_delay", + ABS("ArrDelay") AS "absolute_arrival_delay" +FROM "flight-carriers" +WHERE "ArrDelay" < 0 +LIMIT 1 +``` +Returns the following: + +| `arrival_delay` | `absolute_arrival_delay` | +| -- | -- | +| `-27` | `27` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## ACOS -`ACOS()` +Calculates the arc cosine (arccosine) of a numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `ACOS(expr)` +* **Function type:** Scalar, numeric -Calculates the arc cosine of a numeric expression. +
Example + +The following example calculates the arc cosine of `0`. + +```sql +SELECT ACOS(0) AS "arc_cosine" +``` +Returns the following: + +| `arc_cosine` | +| -- | +| `1.5707963267948966` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## ANY_VALUE @@ -64,6 +107,7 @@ Returns any value of the specified expression. Counts distinct values of a regular column or a prebuilt sketch column. +## APPROX_COUNT_DISTINCT_BUILTIN `APPROX_COUNT_DISTINCT_BUILTIN(expr)` **Function type:** [Aggregation](sql-aggregations.md) @@ -251,27 +295,72 @@ Joins all elements of `arr` by the delimiter specified by `str`. ## ASIN -`ASIN()` +Calculates the arc sine (arcsine) of a numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `ASIN(expr)` +* **Function type:** Scalar, numeric -Calculates the arc sine of a numeric expression. +
Example + +The following example calculates the arc sine of `1`. + +```sql +SELECT ASIN(1) AS "arc_sine" +``` +Returns the following: + +| `arc_sine` | +| -- | +| `1.5707963267948966` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## ATAN -`ATAN()` +Calculates the arc tangent (arctangent) of a numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `ATAN(expr)` +* **Function type:** Scalar, numeric -Calculates the arc tangent of a numeric expression. +
Example + +The following example calculates the arc tangent of `1`. + +```sql +SELECT ATAN(1) AS "arc_tangent" +``` +Returns the following: + +| `arc_tangent` | +| -- | +| `0.7853981633974483` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## ATAN2 -`ATAN2(, )` +Calculates the arc tangent (arctangent) of a specified x and y coordinate. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `ATAN2(x, y)` +* **Function type:** Scalar, numeric -Calculates the arc tangent of the two arguments. +
Example + +The following example calculates the arc tangent of the coordinate `(1, -1)` + +```sql +SELECT ATAN2(1,-1) AS "arc_tangent_2" +``` +Returns the following: + +| `arc_tangent_2` | +| -- | +| `2.356194490192345` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## AVG @@ -419,11 +508,29 @@ Rounds up a timestamp by a given time unit. ## CEIL (numeric) -`CEIL()` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Calculates the smallest integer value greater than or equal to the numeric expression. +* **Syntax:** `CEIL()` +* **Function type:** Scalar, numeric + +
Example + +The following example applies the CEIL function to the `fare_amount` column from the `taxi-trips` datasource. + +```sql +SELECT + "fare_amount" AS "fare_amount", + CEIL("fare_amount") AS "ceiling_fare_amount" +FROM "taxi-trips" +LIMIT 1 +``` +Returns the following: + +| `fare_amount` | `ceiling_fare_amount` | +| -- | -- | +| `21.25` | `22` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## CHAR_LENGTH @@ -467,20 +574,50 @@ Finds whether a string is in a given expression, case-sensitive. ## COS -`COS()` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Calculates the trigonometric cosine of an angle expressed in radians. +* **Syntax:** `COS(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example calculates the cosine of angle `PI/3` radians. + +```sql +SELECT COS(PI / 3) AS "cosine" +``` +Returns the following: + +| `cosine` | +| -- | +| `0.5000000000000001` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## COT -`COT()` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Calculates the trigonometric cotangent of an angle expressed in radians. +* **Syntax:** `COT(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example calculates the cotangent of angle `PI/3` radians. + +```sql +SELECT COT(PI / 3) AS "cotangent" +``` +Returns the following: + +| `cotangent` | +| -- | +| `0.577350269189626` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## COUNT `COUNT([DISTINCT] expr)` @@ -542,12 +679,27 @@ Decodes a Base64-encoded string into a UTF-8 encoded string. ## DEGREES -`DEGREES()` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Converts an angle from radians to degrees. +* **Syntax:** `DEGREES(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example converts an angle of `PI` radians to degrees + +```sql +SELECT DEGREES(PI) AS "degrees" +``` +Returns the following: + +| `degrees` | +| -- | +| `180` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## DENSE_RANK `DENSE_RANK()` @@ -697,12 +849,27 @@ Returns the value of a numeric or string expression corresponding to the earlies ## EXP -`EXP()` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Calculates _e_ raised to the power of the numeric expression. +* **Syntax:** `EXP()` +* **Function type:** Scalar, numeric + +
Example + +The following example calculates _e_ to the power of 1. + +```sql +SELECT EXP(1) AS "exponential" +``` +Returns the following: + +| `exponential` | +| -- | +| `2.7182818284590455` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## EXTRACT `EXTRACT( FROM )` @@ -729,11 +896,30 @@ Rounds down a timestamp by a given time unit. ## FLOOR (numeric) -`FLOOR()` +Calculates the largest integer less than or equal to the numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `FLOOR()` +* **Function type:** Scalar, numeric -Calculates the largest integer value less than or equal to the numeric expression. +
Example + +The following example applies the FLOOR function to the `fare_amount` column from the `taxi-trips` datasource. + +```sql +SELECT + "fare_amount" AS "fare_amount", + FLOOR("fare_amount") AS "floor_fare_amount" +FROM "taxi-trips" +LIMIT 1 +``` +Returns the following: + +| `fare_amount` | `floor_fare_amount` | +| -- | -- | +| `21.25` | `21` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## GREATEST @@ -961,19 +1147,57 @@ Returns the length of the expression in UTF-16 encoding. ## LN -`LN(expr)` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Calculates the natural logarithm of the numeric expression. +* **Syntax:** `LN()` +* **Function type:** Scalar, numeric + +
Example + +The following example applies the LN function to the `max_temperature` column from the `taxi-trips` datasource. + +```sql +SELECT + "max_temperature" AS "max_temperature", + LN("max_temperature") AS "natural_log_max_temp" +FROM "taxi-trips" +LIMIT 1 +``` +Returns the following: + +| `max_temperature` | `natural_log_max_temp` | +| -- | -- | +| `76` | `4.330733340286331` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## LOG10 -`LOG10(expr)` +Calculates the base-10 logarithm of the numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `LOG10()` +* **Function type:** Scalar, numeric -Calculates the base-10 of the numeric expression. +
Example + +The following example applies the LOG10 function to the `max_temperature` column from the `taxi-trips` datasource. + +```sql +SELECT + "max_temperature" AS "max_temperature", + LOG10("max_temperature") AS "log10_max_temp" +FROM "taxi-trips" +LIMIT 1 +``` +Returns the following: + +| `max_temperature` | `log10_max_temp` | +| -- | -- | +| `76` | `1.8808135922807914` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## LOOKUP @@ -1033,11 +1257,26 @@ Returns the minimum value of a set of values. ## MOD -`MOD(x, y)` +Calculates x modulo y, or the remainder of x divided by y. Where x and y are numeric expressions. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `MOD(x, y)` +* **Function type:** Scalar, numeric -Calculates x modulo y, or the remainder of x divided by y. +
Example + +The following calculates 78 MOD 10. + +```sql +SELECT MOD(78, 10) as "modulo" +``` +Returns the following: + +| `modulo` | +| -- | +| `8` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## MV_APPEND @@ -1217,20 +1456,50 @@ Returns the one-based index position of a substring within an expression, option ## POWER -`POWER(expr, power)` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Calculates a numerical expression raised to the specified power. +* **Syntax:** `POWER(base, exponent)` +* **Function type:** Scalar, numeric + +
Example + +The following example raises 5 to the power of 2. + +```sql +SELECT POWER(5, 2) AS "power" +``` +Returns the following: + +| `power` | +| -- | +| `25` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## RADIANS -`RADIANS(expr)` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Converts an angle from degrees to radians. +* **Syntax:** `RADIANS(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example converts an angle of `180` degrees to radians + +```sql +SELECT RADIANS(180) AS "radians" +``` +Returns the following: + +| `radians` | +| -- | +| `3.141592653589793` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## RANK `RANK()` @@ -1298,12 +1567,32 @@ Returns the rightmost number of characters from an expression. ## ROUND -`ROUND(expr[, digits])` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Calculates the rounded value for a numerical expression. +* **Syntax:** `ROUND(expr[, digits])` +* **Function type:** Scalar, numeric + +
Example + +The following applies the ROUND function to 0 decimal points on the `pickup_longitude` column from the `taxi-trips` datasource. + +```sql +SELECT + "pickup_longitude" AS "pickup_longitude", + ROUND("pickup_longitude", 0) as "rounded_pickup_longitude" +FROM "taxi-trips" +WHERE "pickup_longitude" IS NOT NULL +LIMIT 1 +``` +Returns the following: + +| `pickup_longitude` | `rounded_pickup_longitude` | +| -- | -- | +| `-73.9377670288086` | `-74` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## ROW_NUMBER `ROW_NUMBER()` @@ -1338,20 +1627,50 @@ Returns `x` divided by `y`, guarded on division by 0. ## SIN -`SIN(expr)` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Calculates the trigonometric sine of an angle expressed in radians. +* **Syntax:** `SIN(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example calculates the sine of angle `PI/3` radians. + +```sql +SELECT SIN(PI / 3) AS "sine" +``` +Returns the following: + +| `sine` | +| -- | +| `0.8660254037844386` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## SQRT -`SQRT(expr)` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Calculates the square root of a numeric expression. +* **Syntax:** `SQRT()` +* **Function type:** Scalar, numeric + +
Example + +The following example calculates the square root of 25. + +```sql +SELECT SQRT(25) AS "square_root" +``` +Returns the following: + +| `square_root` | +| -- | +| `5` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## STDDEV `STDDEV(expr)` @@ -1451,12 +1770,27 @@ Calculates the sum of a set of values. ## TAN -`TAN(expr)` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Calculates the trigonometric tangent of an angle expressed in radians. +* **Syntax:** `TAN(expr)` +* **Function type:** Scalar, numeric + +
Example + +The following example calculates the tangent of angle `PI/3` radians. + +```sql +SELECT TAN(PI / 3) AS "tangent" +``` +Returns the following: + +| `tangent` | +| -- | +| `1.7320508075688767` | +
+ +[Learn more](sql-scalar.md#numeric-functions) + ## TDIGEST_GENERATE_SKETCH `TDIGEST_GENERATE_SKETCH(expr, [compression])` @@ -1620,20 +1954,41 @@ Trims the leading or trailing characters of an expression. ## TRUNC -`TRUNC(expr[, digits])` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Alias for [`TRUNCATE`](#truncate). +* **Syntax:** `TRUNC(expr[, digits])` +* **Function type:** Scalar, numeric + +[Learn more](sql-scalar.md#numeric-functions) + ## TRUNCATE -`TRUNCATE(expr[, digits])` - -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) - Truncates a numerical expression to a specific number of decimal digits. +* **Syntax:** `TRUNCATE(expr[, digits])` +* **Function type:** Scalar, numeric + +
Example + +The following applies the TRUNCATE function to 1 decimal place on the `pickup_longitude` column from the `taxi-trips` datasource. + +```sql +SELECT + "pickup_longitude" as "pickup_longitude", + TRUNCATE("pickup_longitude", 1) as "truncate_pickup_longitude" +FROM "taxi-trips" +WHERE "pickup_longitude" IS NOT NULL +LIMIT 1 +``` +Returns the following: + +| `pickup_longitude` | `truncate_pickup_longitude` | +| -- | -- | +| `-73.9377670288086` | `-73.9` | +
+ + +[Learn more](sql-scalar.md#numeric-functions) ## TRY_PARSE_JSON @@ -1683,3 +2038,4 @@ Calculates the sample variance of a set of values. Alias for [`VAR_SAMP`](#var_samp). + diff --git a/docs/querying/sql-translation.md b/docs/querying/sql-translation.md index 056d63ece57..e430caa8bf0 100644 --- a/docs/querying/sql-translation.md +++ b/docs/querying/sql-translation.md @@ -342,7 +342,6 @@ The above EXPLAIN PLAN returns the following result: "regionName", "v0" ], - "legacy": false, "context": { "finalizeAggregations": false, "forceExpressionVirtualColumns": true, @@ -562,7 +561,6 @@ The above EXPLAIN PLAN query returns the following result: "regionName", "v0" ], - "legacy": false, "context": { "finalizeAggregations": false, "groupByEnableMultiValueUnnesting": false, diff --git a/docs/querying/sql-window-functions.md b/docs/querying/sql-window-functions.md index d64538779f0..7c2c3aef53e 100644 --- a/docs/querying/sql-window-functions.md +++ b/docs/querying/sql-window-functions.md @@ -246,11 +246,8 @@ Druid has guardrail logic to prevent you from executing window function queries For example: - You cannot set expressions as bounds for window frames. -- You cannot use two FOLLOWING expressions in the window frame. For example: `ROWS BETWEEN 2 ROWS FOLLOWING and 3 ROWS FOLLOWING`. - You can only use a RANGE frames when both endpoints are unbounded or current row. -If you write a query that violates one of these conditions, Druid throws an error: "The query contains a window frame which may return incorrect results. To disregard this warning, set `windowingStrictValidation` to false in the query context." - ## Window function reference |Function|Notes| diff --git a/docs/release-info/migr-ansi-sql-null.md b/docs/release-info/migr-ansi-sql-null.md new file mode 100644 index 00000000000..71c7f30ffb0 --- /dev/null +++ b/docs/release-info/migr-ansi-sql-null.md @@ -0,0 +1,386 @@ +--- +id: migr-ansi-sql-null +title: "Migration guide: SQL compliant mode" +sidebar_label: SQL compliant mode +--- + + +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + +In Apache Druid 28.0.0, the default [null handling](../querying/sql-data-types.md#null-values) mode changed to be compliant with the ANSI SQL standard. +This guide provides strategies for Druid operators who rely on legacy Druid null handling behavior in their applications to transition to SQL compliant mode. +Legacy mode is planned for removal from Druid. + +## SQL compliant null handling + +As of Druid 28.0.0, Druid writes segments in an ANSI SQL compatible null handling mode by default. +This means that Druid stores null values distinctly from empty strings for string dimensions and distinctly from 0 for numeric dimensions. + +This can impact your application behavior because the ANSI SQL standard defines any comparison to null to be unknown. +According to this three-valued logic, `x <> 'some value'` only returns non-null values. + +The default Druid configurations for 28.0.0 and later that enable ANSI SQL compatible null handling mode are the following: + +* `druid.generic.useDefaultValueForNull=false` +* `druid.expressions.useStrictBooleans=true` +* `druid.generic.useThreeValueLogicForNativeFilters=true`  + +Follow the [Null handling tutorial](../tutorials/tutorial-sql-null.md) to learn how the default null handling works in Druid. + +## Legacy null handling and two-valued logic + +Prior to Druid 28.0.0, Druid defaulted to a legacy mode which stored default values instead of nulls. +In legacy mode, Druid created segments with the following characteristics at ingestion time: + +- String columns couldn't distinguish an empty string, `''`, from null. + Therefore, Druid treated them both as interchangeable values. +- Numeric columns couldn't represent null valued rows. + Therefore, Druid stored `0` instead of `null`. + +The Druid configurations for the deprecated legacy mode were the following: + +* `druid.generic.useDefaultValueForNull=true` +* `druid.expressions.useStrictBooleans=false` +* `druid.generic.useThreeValueLogicForNativeFilters=true` + +These configurations are deprecated and scheduled for removal. +After the configurations are removed, Druid will ignore them if they exist in your configuration files and use the default SQL compliant mode. + +## Migrate to SQL compliant mode + +If your business logic relies on the behavior of legacy mode, you have the following options to operate Druid in an ANSI SQL compatible null handling mode: + +- Modify incoming data to either [avoid nulls](#replace-null-values-at-ingestion-time) or [avoid empty strings](#coerce-empty-strings-to-null-at-ingestion-time) to achieve the same query behavior as legacy mode. This means modifying your ingestion SQL queries and ingestion specs to handle nulls or empty strings. + For example, replacing a null for a string column with an empty string or a 0 for a numeric column. + However, it means that your existing queries should operate as if Druid were in legacy mode. + If you do not care about preserving null values, this is a good option for you. + +- Preserve null values and [update all of your SQL queries to be ANSI SQL compliant](#rewrite-your-queries-to-be-sql-compliant). + This means you can preserve the incoming data with nulls intact. + However, you must rewrite any affected client-side queries to be ANSI SQL compliant. + If you have a requirement to preserve null values, choose this option. + +### Replace null values at ingestion time + +If you don't need to preserve null values within Druid, you can use a transform at ingestion time to replace nulls with other values. + +Consider the following input data: + +```json +{"time":"2024-01-01T00:00:00.000Z","string_example":"my_string","number_example":99} +{"time":"2024-01-02T00:00:00.000Z","string_example":"","number_example":0} +{"time":"2024-01-03T00:00:00.000Z","string_example":null,"number_example":null} +``` + +The following example illustrates how to use COALESCE and NVL at ingestion time to avoid null values in Druid: + + + + + +```sql +REPLACE INTO "no_nulls_example" OVERWRITE ALL +WITH "ext" AS ( + SELECT * + FROM TABLE( + EXTERN( + '{"type":"inline","data":"{\"time\":\"2024-01-01T00:00:00.000Z\",\"string_example\":\"my_string\",\"number_example\":99}\n{\"time\":\"2024-01-02T00:00:00.000Z\",\"string_example\":\"\",\"number_example\":0}\n{\"time\":\"2024-01-03T00:00:00.000Z\",\"string_example\":null,\"number_example\":null}"}', + '{"type":"json"}' + ) + ) EXTEND ("time" VARCHAR, "string_example" VARCHAR, "number_example" BIGINT) +) +SELECT + TIME_PARSE("time") AS "__time", + -- Replace any null string values with an empty string + COALESCE("string_example",'') AS string_example, + -- Replace any null numeric values with 0 + NVL("number_example",0) AS number_example +FROM "ext" +PARTITIONED BY MONTH +``` + + + + +```json +{ + "type": "index_parallel", + "spec": { + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "inline", + "data": "{\"time\":\"2024-01-01T00:00:00.000Z\",\"string_example\":\"my_string\",\"number_example\":99}\n{\"time\":\"2024-01-02T00:00:00.000Z\",\"string_example\":\"\",\"number_example\":0}\n{\"time\":\"2024-01-03T00:00:00.000Z\",\"string_example\":null,\"number_example\":null}" + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "inline_data_native", + "timestampSpec": { + "column": "time", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "string_example", + { + "type": "long", + "name": "number_example" + } + ] + }, + "granularitySpec": { + "queryGranularity": "none", + "rollup": false, + "segmentGranularity": "MONTH" + }, + "transformSpec": { + "transforms": [ + { + "type": "expression", + "name": "string_example", + "expression": "COALESCE(\"string_example\",'')" + }, + { + "type": "expression", + "name": "number_example", + "expression": "NVL(\"number_example\",0)" + } + ] + } + } + } +} +``` + + + + +Druid ingests the data with no null values as follows: + +| `__time` | `string_examle` | `number_example`| +| -- | -- | -- | +| `2024-01-01T00:00:00.000Z`| `my_string`| 99 | +| `2024-01-02T00:00:00.000Z`| `empty`| 0 | +| `2024-01-03T00:00:00.000Z`| `empty`| 0 | + +### Coerce empty strings to null at ingestion time + +In legacy mode, Druid recognized empty strings as nulls for equality comparison. +If your queries rely on empty strings to represent nulls, you can coerce empty strings to null at ingestion time using NULLIF. + +For example, consider the following sample input data: + +```json +{"time":"2024-01-01T00:00:00.000Z","string_example":"my_string"} +{"time":"2024-01-02T00:00:00.000Z","string_example":""} +{"time":"2024-01-03T00:00:00.000Z","string_example":null} +``` + +In legacy mode, Druid wrote an empty string for the third record. +Therefore the following query returned 2: + +```sql +SELECT count(*) +FROM "null_string" +WHERE "string_example" IS NULL +``` + +In SQL compliant mode, Druid differentiates between empty strings and nulls, so the same query would return 1. +The following example shows how to coerce empty strings into null to accommodate IS NULL comparisons: + + + + + +```sql +REPLACE INTO "null_string" OVERWRITE ALL +WITH "ext" AS ( + SELECT * + FROM TABLE( + EXTERN( + '{"type":"inline","data":"{\"time\":\"2024-01-01T00:00:00.000Z\",\"string_example\":\"my_string\"}\n{\"time\":\"2024-01-02T00:00:00.000Z\",\"string_example\":\"\"}\n{\"time\":\"2024-01-03T00:00:00.000Z\",\"string_example\":null}"}', + '{"type":"json"}' + ) + ) EXTEND ("time" VARCHAR, "string_example" VARCHAR) +) +SELECT + TIME_PARSE("time") AS "__time", + NULLIF("string_example",'') AS "string_example" +FROM "ext" +PARTITIONED BY MONTH +``` + + + + + +```json +{ + "type": "index_parallel", + "spec": { + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "inline", + "data": "{\"time\":\"2024-01-01T00:00:00.000Z\",\"string_example\":\"my_string\"}\n{\"time\":\"2024-01-02T00:00:00.000Z\",\"string_example\":\"\"}\n{\"time\":\"2024-01-03T00:00:00.000Z\",\"string_example\":null}" + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "null_string", + "timestampSpec": { + "column": "time", + "format": "iso" + }, + "transformSpec": { + "transforms": [ + { + "type": "expression", + "expression": "case_searched((\"string_example\" == ''),null,\"string_example\")", + "name": "string_example" + } + ] + }, + "dimensionsSpec": { + "dimensions": [ + "string_example" + ] + }, + "granularitySpec": { + "queryGranularity": "none", + "rollup": false, + "segmentGranularity": "month" + } + } + } +} +``` + + + + +Druid ingests the data with no empty strings as follows: + +| `__time` | `string_examle` | +| -- | -- | -- | +| `2024-01-01T00:00:00.000Z`| `my_string`| +| `2024-01-02T00:00:00.000Z`| `null`| +| `2024-01-03T00:00:00.000Z`| `null`| + +Therefore `SELECT count(*) FROM "null_string" WHERE "string_example" IS NULL` returns 2. + +### Rewrite your queries to be SQL compliant + +If you want to maintain null values in your data within Druid, you can use the following ANSI SQL compliant querying strategies to achieve the same results as legacy null handling: + +- Modify inequality queries to include null values. + For example, `x <> 'some value'` becomes `(x <> 'some value' OR x IS NULL)`. +- Use COALESCE or NVL to replace nulls with a value. + For example, `x + 1` becomes `NVL(numeric_value, 0)+1` + +Consider the following Druid datasource `null_example`: + +| `__time` | `string_examle` | `number_example`| +| -- | -- | -- | +| `2024-01-01T00:00:00.000Z`| `my_string`| 99 | +| `2024-01-02T00:00:00.000Z`| `empty`| 0 | +| `2024-01-03T00:00:00.000Z`| `null`| null | + +Druid excludes null strings from equality comparisons. For example: + +```sql +SELECT COUNT(*) AS count_example +FROM "null_example" +WHERE "string_example"<> 'my_string' +``` + +Druid returns 1 because null is considered unknown: neither equal nor unequal to the value. + +To count null values in the result, you can use an OR operator: + +```sql +SELECT COUNT(*) AS count_example +FROM "null_example" +WHERE ("string_example"<> 'my_string') OR "string_example" IS NULL +``` + +Druid returns 2. +To achieve the same result, you can use IS DISTINCT FROM for null-safe comparison: + +```sql +SELECT COUNT(*) as count_example +FROM "null_example" +WHERE "string_example" IS DISTINCT FROM 'my_string' +``` + +Similarly, arithmetic operators on null return null. For example: + +```sql +SELECT "number_example" + 1 AS additon_example +FROM "null_example" +``` + +Druid returns the following because null + any value is null for the ANSI SQL standard: + +| `addition_example`| +| -- | +| 100 | +| 1 | +| null | + +Use NVL to avoid nulls with arithmetic. For example: + +```sql +SELECT NVL("number_example",0) + 1 AS additon_example +FROM "null_example" +``` + +Druid returns the following: + +| `addition_example` | +| -- | +| 100 | +| 1 | +| null | + +## Learn more + +See the following topics for more information: + - [Null handling tutorial](../tutorials/tutorial-sql-null.md) to learn how the default null handling works in Druid. + - [Null values](../querying/sql-data-types.md#null-values) for a description of Druid's behavior with null values. + - [Handling null values](../design/segments.md#handling-null-values) for details about how Druid stores null values. \ No newline at end of file diff --git a/docs/release-info/migration-guide.md b/docs/release-info/migration-guide.md index 92053b83a11..760b691d143 100644 --- a/docs/release-info/migration-guide.md +++ b/docs/release-info/migration-guide.md @@ -30,12 +30,18 @@ The guides in this section outline breaking changes introduced in Druid 25.0.0 a ## Migrate from multi-value dimensions to arrays -Druid now supports SQL-compliant array types. Whenever possible, you should use the array type over multi-value dimensions. See [Migration guide: MVDs to arrays](migr-mvd-array.md). +Druid now supports SQL-compliant array types. Whenever possible, you should use the array type over multi-value dimensions. See [Migration guide: MVDs to arrays](./migr-mvd-array.md). ## Migrate to front-coded dictionary encoding -Druid encodes string columns into dictionaries for better compression. Front-coded dictionary encoding reduces storage and improves performance by optimizing for strings that share similar beginning substrings. See [Migration guide: front-coded dictionaries](migr-front-coded-dict.md) for more information. +Druid encodes string columns into dictionaries for better compression. Front-coded dictionary encoding reduces storage and improves performance by optimizing for strings that share similar beginning substrings. See [Migration guide: front-coded dictionaries](./migr-front-coded-dict.md) for more information. ## Migrate from `maxSubqueryRows` to `maxSubqueryBytes` -Druid allows you to set a byte-based limit on subquery size to prevent Brokers from running out of memory when handling large subqueries. The byte-based subquery limit overrides Druid's row-based subquery limit. We recommend that you move towards using byte-based limits starting in Druid 30.0.0. See [Migration guide: subquery limit](migr-subquery-limit.md) for more information. +Druid allows you to set a byte-based limit on subquery size to prevent Brokers from running out of memory when handling large subqueries. The byte-based subquery limit overrides Druid's row-based subquery limit. We recommend that you move towards using byte-based limits starting in Druid 30.0.0. See [Migration guide: subquery limit](./migr-subquery-limit.md) for more information. + +## Migrate to SQL compliant null handling mode + +By default, the Druid [null handling](../querying/sql-data-types.md#null-values) mode is now compliant with ANSI SQL. +This guide provides strategies for Druid operators and users who rely on the legacy Druid null handling behavior in their applications to transition to ANSI SQL compliant mode. See [Migration guide: SQL compliant mode](./migr-ansi-sql-null.md +) for more information. \ No newline at end of file diff --git a/docs/tutorials/tutorial-rollup.md b/docs/tutorials/tutorial-rollup.md index b2c74d7e5b3..464197d551c 100644 --- a/docs/tutorials/tutorial-rollup.md +++ b/docs/tutorials/tutorial-rollup.md @@ -24,18 +24,21 @@ sidebar_label: Aggregate data with rollup --> -Apache Druid can summarize raw data at ingestion time using a process we refer to as "rollup". Rollup is a first-level aggregation operation over a selected set of columns that reduces the size of stored data. +Apache Druid® can summarize raw data at ingestion time using a process known as "rollup." [Rollup](../multi-stage-query/concepts.md#rollup) is a first-level aggregation operation over a selected set of columns that reduces the size of stored data. -This tutorial will demonstrate the effects of rollup on an example dataset. +This tutorial demonstrates how to apply rollup during ingestion and highlights its effects during query execution. The examples in the tutorial use the [multi-stage query (MSQ)](../multi-stage-query/index.md) task engine to execute SQL statements. -For this tutorial, we'll assume you've already downloaded Druid as described in -the [single-machine quickstart](index.md) and have it running on your local machine. +## Prerequisites -It will also be helpful to have finished [Load a file](../tutorials/tutorial-batch.md) and [Query data](../tutorials/tutorial-query.md) tutorials. +Before proceeding, download Druid as described in [Quickstart (local)](index.md) and have it running on your local machine. You don't need to load any data into the Druid cluster. -## Example data +You should be familiar with data querying in Druid. If you haven't already, go through the [Query data](../tutorials/tutorial-query.md) tutorial first. -For this tutorial, we'll use a small sample of network flow event data, representing packet and byte counts for traffic from a source to a destination IP address that occurred within a particular second. + +## Load the example data + +For this tutorial, you use a small sample of network flow event data representing IP traffic. +The data contains packet and byte counts from a source IP address to a destination IP address. ```json {"timestamp":"2018-01-01T01:01:35Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":20,"bytes":9024} @@ -49,102 +52,63 @@ For this tutorial, we'll use a small sample of network flow event data, represen {"timestamp":"2018-01-02T21:35:45Z","srcIP":"7.7.7.7", "dstIP":"8.8.8.8","packets":12,"bytes":2818} ``` -A file containing this sample input data is located at `quickstart/tutorial/rollup-data.json`. +Load the sample dataset using the [`INSERT INTO`](../multi-stage-query/reference.md/#insert) statement and the [`EXTERN`](../multi-stage-query/reference.md/#extern-function) function to ingest the data inline. In the [Druid web console](../operations/web-console.md), go to the **Query** view and run the following query: -We'll ingest this data using the following ingestion task spec, located at `quickstart/tutorial/rollup-index.json`. - -```json -{ - "type" : "index_parallel", - "spec" : { - "dataSchema" : { - "dataSource" : "rollup-tutorial", - "dimensionsSpec" : { - "dimensions" : [ - "srcIP", - "dstIP" - ] - }, - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "metricsSpec" : [ - { "type" : "count", "name" : "count" }, - { "type" : "longSum", "name" : "packets", "fieldName" : "packets" }, - { "type" : "longSum", "name" : "bytes", "fieldName" : "bytes" } - ], - "granularitySpec" : { - "type" : "uniform", - "segmentGranularity" : "week", - "queryGranularity" : "minute", - "intervals" : ["2018-01-01/2018-01-03"], - "rollup" : true - } - }, - "ioConfig" : { - "type" : "index_parallel", - "inputSource" : { - "type" : "local", - "baseDir" : "quickstart/tutorial", - "filter" : "rollup-data.json" - }, - "inputFormat" : { - "type" : "json" - }, - "appendToExisting" : false - }, - "tuningConfig" : { - "type" : "index_parallel", - "partitionsSpec": { - "type": "dynamic" - }, - "maxRowsInMemory" : 25000 - } - } -} +```sql +INSERT INTO "rollup_tutorial" +WITH "inline_data" AS ( + SELECT * + FROM TABLE(EXTERN('{ + "type":"inline", + "data":"{\"timestamp\":\"2018-01-01T01:01:35Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":20,\"bytes\":9024}\n{\"timestamp\":\"2018-01-01T01:02:14Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":38,\"bytes\":6289}\n{\"timestamp\":\"2018-01-01T01:01:59Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":11,\"bytes\":5780}\n{\"timestamp\":\"2018-01-01T01:01:51Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":255,\"bytes\":21133}\n{\"timestamp\":\"2018-01-01T01:02:29Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":377,\"bytes\":359971}\n{\"timestamp\":\"2018-01-01T01:03:29Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":49,\"bytes\":10204}\n{\"timestamp\":\"2018-01-02T21:33:14Z\",\"srcIP\":\"7.7.7.7\",\"dstIP\":\"8.8.8.8\",\"packets\":38,\"bytes\":6289}\n{\"timestamp\":\"2018-01-02T21:33:45Z\",\"srcIP\":\"7.7.7.7\",\"dstIP\":\"8.8.8.8\",\"packets\":123,\"bytes\":93999}\n{\"timestamp\":\"2018-01-02T21:35:45Z\",\"srcIP\":\"7.7.7.7\",\"dstIP\":\"8.8.8.8\",\"packets\":12,\"bytes\":2818}"}', + '{"type":"json"}')) + EXTEND ("timestamp" VARCHAR, "srcIP" VARCHAR, "dstIP" VARCHAR, "packets" BIGINT, "bytes" BIGINT) +) +SELECT + FLOOR(TIME_PARSE("timestamp") TO MINUTE) AS __time, + "srcIP", + "dstIP", + SUM("bytes") AS "bytes", + SUM("packets") AS "packets", + COUNT(*) AS "count" +FROM "inline_data" +GROUP BY 1, 2, 3 +PARTITIONED BY DAY ``` -Rollup has been enabled by setting `"rollup" : true` in the `granularitySpec`. +Note the following aspects of the ingestion statement: +* You transform the timestamp field using the `FLOOR` function to round timestamps down to the minute. +* You group by the dimensions `timestamp`, `srcIP`, and `dstIP`. +* You create the `bytes` and `packets` metrics, which are summed from their respective input fields. +* You also create the `count` metric that records the number of rows that get rolled-up per each row in the datasource. -Note that we have `srcIP` and `dstIP` defined as dimensions, a longSum metric is defined for the `packets` and `bytes` columns, and the `queryGranularity` has been defined as `minute`. +With rollup, Druid combines rows with identical timestamp and dimension values after the timestamp truncation. Druid computes and stores the metric values using the specified aggregation function over each set of rolled-up rows. -We will see how these definitions are used after we load this data. - -## Load the example data - -From the apache-druid-{{DRUIDVERSION}} package root, run the following command: - -```bash -bin/post-index-task --file quickstart/tutorial/rollup-index.json --url http://localhost:8081 -``` - -After the script completes, we will query the data. +After the ingestion completes, you can query the data. ## Query the example data -Let's run `bin/dsql` and issue a `select * from "rollup-tutorial";` query to see what data was ingested. +In the web console, open a new tab in the **Query** view. Run the following query to view the ingested data: -```bash -$ bin/dsql -Welcome to dsql, the command-line client for Druid SQL. -Type "\h" for help. -dsql> select * from "rollup-tutorial"; -┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐ -│ __time │ bytes │ count │ dstIP │ packets │ srcIP │ -├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤ -│ 2018-01-01T01:01:00.000Z │ 35937 │ 3 │ 2.2.2.2 │ 286 │ 1.1.1.1 │ -│ 2018-01-01T01:02:00.000Z │ 366260 │ 2 │ 2.2.2.2 │ 415 │ 1.1.1.1 │ -│ 2018-01-01T01:03:00.000Z │ 10204 │ 1 │ 2.2.2.2 │ 49 │ 1.1.1.1 │ -│ 2018-01-02T21:33:00.000Z │ 100288 │ 2 │ 8.8.8.8 │ 161 │ 7.7.7.7 │ -│ 2018-01-02T21:35:00.000Z │ 2818 │ 1 │ 8.8.8.8 │ 12 │ 7.7.7.7 │ -└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘ -Retrieved 5 rows in 1.18s. - -dsql> +```sql +SELECT * FROM "rollup_tutorial" ``` -Let's look at the three events in the original input data that occurred during `2018-01-01T01:01`: +Returns the following: + +| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` | +| -- | -- | -- | -- | -- | -- | +| `2018-01-01T01:01:00.000Z` | `1.1.1.1` | `2.2.2.2` | `35,937` | `3` | `286` | +| `2018-01-01T01:02:00.000Z` | `1.1.1.1` | `2.2.2.2` | `366,260` | `2` | `415` | +| `2018-01-01T01:03:00.000Z` | `1.1.1.1` | `2.2.2.2` | `10,204` | `1` | `49` | +| `2018-01-02T21:33:00.000Z` | `7.7.7.7` | `8.8.8.8` | `100,288` | `2` | `161` | +| `2018-01-02T21:35:00.000Z` | `7.7.7.7` | `8.8.8.8` | `2,818` | `1` | `12` | + +Notice there are only five rows as opposed to the nine rows in the example data. In the next section, you explore the components of the rolled-up rows. + +## View rollup in action + +Consider the three events in the original input data that occur over the course of minute `2018-01-01T01:01`: ```json {"timestamp":"2018-01-01T01:01:35Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":20,"bytes":9024} @@ -152,47 +116,47 @@ Let's look at the three events in the original input data that occurred during ` {"timestamp":"2018-01-01T01:01:59Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":11,"bytes":5780} ``` -These three rows have been "rolled up" into the following row: +Druid combines the three rows into one during rollup: -```bash -┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐ -│ __time │ bytes │ count │ dstIP │ packets │ srcIP │ -├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤ -│ 2018-01-01T01:01:00.000Z │ 35937 │ 3 │ 2.2.2.2 │ 286 │ 1.1.1.1 │ -└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘ -``` +| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` | +| -- | -- | -- | -- | -- | -- | +| `2018-01-01T01:01:00.000Z` | `1.1.1.1` | `2.2.2.2` | `35,937` | `3` | `286` | -The input rows have been grouped by the timestamp and dimension columns `{timestamp, srcIP, dstIP}` with sum aggregations on the metric columns `packets` and `bytes`. +Before the grouping occurs, the `FLOOR(TIME_PARSE("timestamp") TO MINUTE)` expression buckets (floors) the timestamp column of the original input by minute. -Before the grouping occurs, the timestamps of the original input data are bucketed/floored by minute, due to the `"queryGranularity":"minute"` setting in the ingestion spec. +The input rows are grouped because they have the same values for their dimension columns `{timestamp, srcIP, dstIP}`. The metric columns calculate the sum aggregation of the grouped rows for `packets` and `bytes`. The `count` metric shows how many rows from the original input data contributed to the final rolled-up row. -Likewise, these two events that occurred during `2018-01-01T01:02` have been rolled up: +Now, consider the two events in the original input data that occur over the course of minute `2018-01-01T01:02`: ```json {"timestamp":"2018-01-01T01:02:14Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":38,"bytes":6289} {"timestamp":"2018-01-01T01:02:29Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":377,"bytes":359971} ``` -```bash -┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐ -│ __time │ bytes │ count │ dstIP │ packets │ srcIP │ -├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤ -│ 2018-01-01T01:02:00.000Z │ 366260 │ 2 │ 2.2.2.2 │ 415 │ 1.1.1.1 │ -└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘ -``` +The rows are grouped into the following during rollup: -For the last event recording traffic between 1.1.1.1 and 2.2.2.2, no rollup took place, because this was the only event that occurred during `2018-01-01T01:03`: +| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` | +| -- | -- | -- | -- | -- | -- | +| `2018-01-01T01:02:00.000Z` | `1.1.1.1` | `2.2.2.2` | `366,260` | `2` | `415` | + +In the original input data, only one event occurs over the course of minute `2018-01-01T01:03`: ```json {"timestamp":"2018-01-01T01:03:29Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":49,"bytes":10204} ``` -```bash -┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐ -│ __time │ bytes │ count │ dstIP │ packets │ srcIP │ -├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤ -│ 2018-01-01T01:03:00.000Z │ 10204 │ 1 │ 2.2.2.2 │ 49 │ 1.1.1.1 │ -└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘ -``` +Therefore, no rollup takes place: -Note that the `count` metric shows how many rows in the original input data contributed to the final "rolled up" row. +| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` | +| -- | -- | -- | -- | -- | -- | +| `2018-01-01T01:03:00.000Z` | `1.1.1.1` | `2.2.2.2` | `10,204` | `1` | `49` | + + +## Learn more + +See the following topics for more information: + +* [Data rollup](../ingestion/rollup.md) for suggestions and best practices when performing rollup. +* [SQL-based ingestion concepts](../multi-stage-query/concepts.md#rollup) for information on rollup using SQL-based ingestion. +* [SQL-based ingestion query examples](../multi-stage-query/examples.md#insert-with-rollup) for another example of data rollup. +* [Druid schema model](../ingestion/schema-model.md) to learn about the primary timestamp, dimensions, and metrics. diff --git a/docs/tutorials/tutorial-sql-null.md b/docs/tutorials/tutorial-sql-null.md index 37cf23a7152..b91e8019bc2 100644 --- a/docs/tutorials/tutorial-sql-null.md +++ b/docs/tutorials/tutorial-sql-null.md @@ -118,7 +118,7 @@ Druid returns the following: |`another_value`|1|1| |`some_value`|1|1| -Also note that GROUP BY expressions yields distinct entries for `null` and the empty string. +Also note that GROUP BY expressions yield distinct entries for `null` and the empty string. ### Filter for empty strings in addition to null diff --git a/docs/tutorials/tutorial-unnest-arrays.md b/docs/tutorials/tutorial-unnest-arrays.md index 86b5407e0e7..e9fdeb158c1 100644 --- a/docs/tutorials/tutorial-unnest-arrays.md +++ b/docs/tutorials/tutorial-unnest-arrays.md @@ -304,7 +304,6 @@ The following native Scan query returns the rows of the datasource and unnests t "m2", "unnest-dim3" ], - "legacy": false, "granularity": { "type": "all" }, @@ -480,7 +479,6 @@ This query joins the `nested_data` table with itself and outputs the unnested da "m2", "v0" ], - "legacy": false, "context": { "sqlOuterLimit": 1001, "useNativeQueryExplain": true @@ -523,7 +521,6 @@ This query joins the `nested_data` table with itself and outputs the unnested da "m2", "unnest-dim3" ], - "legacy": false, "context": { "sqlOuterLimit": 1001, "useNativeQueryExplain": true @@ -572,7 +569,6 @@ The following query returns the columns `dim45` and `m1`. The `dim45` column is "dim45", "m1" ], - "legacy": false, "granularity": { "type": "all" }, @@ -632,7 +628,6 @@ The following Scan query unnests the column `dim3` into `d3` and a virtual colum "dim5", "d45" ], - "legacy": false, "context": { "queryId": "2618b9ce-6c0d-414e-b88d-16fb59b9c481", "sqlOuterLimit": 1001, diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml index e2df5b99531..2fca74ef55b 100644 --- a/extensions-contrib/cloudfiles-extensions/pom.xml +++ b/extensions-contrib/cloudfiles-extensions/pom.xml @@ -35,7 +35,7 @@ UTF-8 - 2.5.0 + 2.6.0 diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java index 6c195ed1515..568f8ed5a11 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java @@ -28,6 +28,7 @@ public class DruidK8sConstants public static final String TASK_TYPE = "task.type"; public static final String TASK_GROUP_ID = "task.group.id"; public static final String TASK_DATASOURCE = "task.datasource"; + public static final String TASK_JOB_TEMPLATE = "task.jobTemplate"; public static final int PORT = 8100; public static final int TLS_PORT = 8091; public static final int DEFAULT_CPU_MILLICORES = 1000; @@ -42,6 +43,7 @@ public class DruidK8sConstants public static final String DRUID_HOSTNAME_ENV = "HOSTNAME"; public static final String LABEL_KEY = "druid.k8s.peons"; public static final String DRUID_LABEL_PREFIX = "druid."; + public static final String BASE_TEMPLATE_NAME = "base"; public static final long MAX_ENV_VARIABLE_KBS = 130048; // 127 KB static final Predicate IS_TRANSIENT = e -> e instanceof KubernetesResourceNotFoundException; } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java index ae7869707fc..885a4e3e59a 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.fabric8.kubernetes.api.model.PodTemplate; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.k8s.overlord.taskadapter.PodTemplateWithName; import javax.validation.constraints.NotNull; import java.util.Map; @@ -42,7 +43,7 @@ public interface PodTemplateSelectStrategy * allows for customized resource allocation and management tailored to the task's specific requirements. * * @param task The task for which the Pod template is determined. - * @return The pod template that should be used to run the task. + * @return The PodTemplateWithName POJO that contains the name of the template selected and the template itself. */ - @NotNull PodTemplate getPodTemplateForTask(Task task, Map templates); + @NotNull PodTemplateWithName getPodTemplateForTask(Task task, Map templates); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategy.java index 4c2d01b5218..bf3082a79b1 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategy.java @@ -24,6 +24,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import io.fabric8.kubernetes.api.model.PodTemplate; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.k8s.overlord.common.DruidK8sConstants; +import org.apache.druid.k8s.overlord.taskadapter.PodTemplateWithName; import java.util.List; import java.util.Map; @@ -53,15 +55,18 @@ public class SelectorBasedPodTemplateSelectStrategy implements PodTemplateSelect * @return the template if a selector matches, otherwise fallback to base template */ @Override - public PodTemplate getPodTemplateForTask(Task task, Map templates) + public PodTemplateWithName getPodTemplateForTask(Task task, Map templates) { String templateKey = selectors.stream() .filter(selector -> selector.evaluate(task)) .findFirst() .map(Selector::getSelectionKey) - .orElse("base"); + .orElse(DruidK8sConstants.BASE_TEMPLATE_NAME); - return templates.getOrDefault(templateKey, templates.get("base")); + if (!templates.containsKey(templateKey)) { + templateKey = DruidK8sConstants.BASE_TEMPLATE_NAME; + } + return new PodTemplateWithName(templateKey, templates.get(templateKey)); } @JsonProperty diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskTypePodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskTypePodTemplateSelectStrategy.java index b374e0b6ff4..bda7788e3c4 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskTypePodTemplateSelectStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskTypePodTemplateSelectStrategy.java @@ -22,6 +22,8 @@ package org.apache.druid.k8s.overlord.execution; import com.fasterxml.jackson.annotation.JsonCreator; import io.fabric8.kubernetes.api.model.PodTemplate; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.k8s.overlord.common.DruidK8sConstants; +import org.apache.druid.k8s.overlord.taskadapter.PodTemplateWithName; import java.util.Map; @@ -40,9 +42,10 @@ public class TaskTypePodTemplateSelectStrategy implements PodTemplateSelectStrat } @Override - public PodTemplate getPodTemplateForTask(Task task, Map templates) + public PodTemplateWithName getPodTemplateForTask(Task task, Map templates) { - return templates.getOrDefault(task.getType(), templates.get("base")); + String templateKey = templates.containsKey(task.getType()) ? task.getType() : DruidK8sConstants.BASE_TEMPLATE_NAME; + return new PodTemplateWithName(templateKey, templates.get(templateKey)); } @Override diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index 19cc788b3ee..e8aaf1bbab3 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -138,19 +138,21 @@ public class PodTemplateTaskAdapter implements TaskAdapter podTemplateSelectStrategy = dynamicConfig.getPodTemplateSelectStrategy(); } - PodTemplate podTemplate = podTemplateSelectStrategy.getPodTemplateForTask(task, templates); + PodTemplateWithName podTemplateWithName = podTemplateSelectStrategy.getPodTemplateForTask(task, templates); return new JobBuilder() .withNewMetadata() .withName(new K8sTaskId(task).getK8sJobName()) .addToLabels(getJobLabels(taskRunnerConfig, task)) .addToAnnotations(getJobAnnotations(taskRunnerConfig, task)) + .addToAnnotations(DruidK8sConstants.TASK_JOB_TEMPLATE, podTemplateWithName.getName()) .endMetadata() .withNewSpec() - .withTemplate(podTemplate.getTemplate()) + .withTemplate(podTemplateWithName.getPodTemplate().getTemplate()) .editTemplate() .editOrNewMetadata() .addToAnnotations(getPodTemplateAnnotations(task)) + .addToAnnotations(DruidK8sConstants.TASK_JOB_TEMPLATE, podTemplateWithName.getName()) .addToLabels(getPodLabels(taskRunnerConfig, task)) .endMetadata() .editSpec() diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateWithName.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateWithName.java new file mode 100644 index 00000000000..eeebd9fdb1f --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateWithName.java @@ -0,0 +1,78 @@ +/* + * 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.k8s.overlord.taskadapter; + +import io.fabric8.kubernetes.api.model.PodTemplate; + +import javax.annotation.Nonnull; +import java.util.Objects; + +public class PodTemplateWithName +{ + private final String name; + private final PodTemplate podTemplate; + + public PodTemplateWithName(String name, PodTemplate podTemplate) + { + this.name = name; + this.podTemplate = podTemplate; + } + + @Nonnull + public String getName() + { + return name; + } + + @Nonnull + public PodTemplate getPodTemplate() + { + return podTemplate; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PodTemplateWithName that = (PodTemplateWithName) o; + return Objects.equals(name, that.name) && + Objects.equals(podTemplate, that.podTemplate); + } + + @Override + public int hashCode() + { + return Objects.hash(name, podTemplate); + } + + @Override + public String toString() + { + return "PodTemplateWithName{" + + "name='" + name + '\'' + + ", podTemplate=" + podTemplate + + '}'; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java index 04b332aac85..b3fda99b222 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java @@ -79,7 +79,6 @@ public class K8sTestUtils null ), new IndexTask.IndexIOConfig( - null, new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/PodTemplateWithNameTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/PodTemplateWithNameTest.java new file mode 100644 index 00000000000..58259606c86 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/PodTemplateWithNameTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord.common; + +import io.fabric8.kubernetes.api.model.PodTemplate; +import io.fabric8.kubernetes.api.model.PodTemplateBuilder; +import org.apache.druid.k8s.overlord.taskadapter.PodTemplateWithName; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; + +public class PodTemplateWithNameTest +{ + @Test + void testEqualityToMakeCoverageHappy() + { + PodTemplateWithName podTemplateWithName = new PodTemplateWithName( + "name", + new PodTemplateBuilder().build() + ); + PodTemplateWithName podTemplateWithName2 = podTemplateWithName; + + assertEquals(podTemplateWithName, podTemplateWithName2); + assertNotEquals(podTemplateWithName, null); + assertNotEquals(podTemplateWithName, "string"); + assertEquals(podTemplateWithName.hashCode(), podTemplateWithName2.hashCode()); + } + + @Test + void testGettersToMakeCoverageHappy() + { + String name = "name"; + PodTemplate podTemplate = new PodTemplateBuilder().build(); + PodTemplateWithName podTemplateWithName = new PodTemplateWithName( + name, + podTemplate + ); + + assertEquals(name, podTemplateWithName.getName()); + assertEquals(podTemplate, podTemplateWithName.getPodTemplate()); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategyTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategyTest.java index a82bb076855..81589c75128 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategyTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategyTest.java @@ -27,6 +27,7 @@ import io.fabric8.kubernetes.api.model.ObjectMeta; import io.fabric8.kubernetes.api.model.PodTemplate; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.k8s.overlord.taskadapter.PodTemplateWithName; import org.apache.druid.segment.TestHelper; import org.junit.Assert; import org.junit.Before; @@ -97,7 +98,10 @@ public class SelectorBasedPodTemplateSelectStrategyTest List emptySelectors = Collections.emptyList(); SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(emptySelectors); Task task = NoopTask.create(); - Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).getMetadata().getName()); + PodTemplateWithName podTemplateWithName = strategy.getPodTemplateForTask(task, templates); + Assert.assertEquals("base", podTemplateWithName.getName()); + Assert.assertEquals("base", podTemplateWithName.getPodTemplate().getMetadata().getName()); + } @Test @@ -107,7 +111,9 @@ public class SelectorBasedPodTemplateSelectStrategyTest List selectors = Collections.singletonList(noMatchSelector); SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors); Task task = NoopTask.create(); - Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).getMetadata().getName()); + PodTemplateWithName podTemplateWithName = strategy.getPodTemplateForTask(task, templates); + Assert.assertEquals("base", podTemplateWithName.getName()); + Assert.assertEquals("base", podTemplateWithName.getPodTemplate().getMetadata().getName()); } @Test @@ -124,7 +130,9 @@ public class SelectorBasedPodTemplateSelectStrategyTest ); SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors); Task task = NoopTask.create(); - Assert.assertEquals("match", strategy.getPodTemplateForTask(task, templates).getMetadata().getName()); + PodTemplateWithName podTemplateWithName = strategy.getPodTemplateForTask(task, templates); + Assert.assertEquals("match", podTemplateWithName.getName()); + Assert.assertEquals("match", podTemplateWithName.getPodTemplate().getMetadata().getName()); } @Test diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 241b4d9fc68..e2d97accc97 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -19,7 +19,6 @@ package org.apache.druid.k8s.overlord.taskadapter; -import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import io.fabric8.kubernetes.api.model.Pod; @@ -27,7 +26,6 @@ import io.fabric8.kubernetes.api.model.PodSpec; import io.fabric8.kubernetes.api.model.batch.v1.Job; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; @@ -83,9 +81,6 @@ public class DruidPeonClientIntegrationTest { TestUtils utils = new TestUtils(); jsonMapper = utils.getTestObjectMapper(); - for (Module jacksonModule : new FirehoseModule().getJacksonModules()) { - jsonMapper.registerModule(jacksonModule); - } jsonMapper.registerSubtypes( new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), new NamedType(IndexTask.IndexTuningConfig.class, "index") diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index 102565efc35..f8e7186a026 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -19,7 +19,6 @@ package org.apache.druid.k8s.overlord.taskadapter; -import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Joiner; @@ -44,7 +43,6 @@ import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.StringUtils; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; @@ -101,9 +99,6 @@ class K8sTaskAdapterTest { TestUtils utils = new TestUtils(); jsonMapper = utils.getTestObjectMapper(); - for (Module jacksonModule : new FirehoseModule().getJacksonModules()) { - jsonMapper.registerModule(jacksonModule); - } jsonMapper.registerSubtypes( new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), new NamedType(IndexTask.IndexTuningConfig.class, "index") diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java index 58993b9a6a0..9308835d967 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java @@ -19,7 +19,6 @@ package org.apache.druid.k8s.overlord.taskadapter; -import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; @@ -28,7 +27,6 @@ import io.fabric8.kubernetes.api.model.PodSpec; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; @@ -67,9 +65,6 @@ class MultiContainerTaskAdapterTest { TestUtils utils = new TestUtils(); jsonMapper = utils.getTestObjectMapper(); - for (Module jacksonModule : new FirehoseModule().getJacksonModules()) { - jsonMapper.registerModule(jacksonModule); - } jsonMapper.registerSubtypes( new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), new NamedType(IndexTask.IndexTuningConfig.class, "index") diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 4aad419007e..ac2aaa70558 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -163,7 +163,7 @@ public class PodTemplateTaskAdapterTest Task task = new NoopTask("id", "id", "datasource", 0, 0, null); Job actual = adapter.fromTask(task); - Job expected = K8sTestUtils.fileToResource("expectedNoopJob.yaml", Job.class); + Job expected = K8sTestUtils.fileToResource("expectedNoopJobBase.yaml", Job.class); assertJobSpecsEqual(actual, expected); } @@ -197,7 +197,7 @@ public class PodTemplateTaskAdapterTest Task task = new NoopTask("id", "id", "datasource", 0, 0, null); Job actual = adapter.fromTask(task); - Job expected = K8sTestUtils.fileToResource("expectedNoopJobTlsEnabled.yaml", Job.class); + Job expected = K8sTestUtils.fileToResource("expectedNoopJobTlsEnabledBase.yaml", Job.class); assertJobSpecsEqual(actual, expected); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java index afc5299927f..4cf9c3e4cbb 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java @@ -19,14 +19,12 @@ package org.apache.druid.k8s.overlord.taskadapter; -import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; @@ -66,9 +64,6 @@ class SingleContainerTaskAdapterTest { TestUtils utils = new TestUtils(); jsonMapper = utils.getTestObjectMapper(); - for (Module jacksonModule : new FirehoseModule().getJacksonModules()) { - jsonMapper.registerModule(jacksonModule); - } jsonMapper.registerSubtypes( new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), new NamedType(IndexTask.IndexTuningConfig.class, "index") diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml index 2cef837f397..ddae7c0567f 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml @@ -8,11 +8,13 @@ metadata: druid.task.type: "noop" druid.task.group.id: "id" druid.task.datasource: "datasource" + annotations: task.id: "id" task.type: "noop" task.group.id: "id" task.datasource: "datasource" + task.jobTemplate: noop spec: activeDeadlineSeconds: 14400 backoffLimit: 0 @@ -32,6 +34,7 @@ spec: task.type: "noop" task.group.id: "id" task.datasource: "datasource" + task.jobTemplate: noop spec: containers: - command: diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBase.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBase.yaml new file mode 100644 index 00000000000..532c3dd53e8 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBase.yaml @@ -0,0 +1,55 @@ +apiVersion: batch/v1 +kind: Job +metadata: + name: "id-3e70afe5cd823dfc7dd308eea616426b" + labels: + druid.k8s.peons: "true" + druid.task.id: "id" + druid.task.type: "noop" + druid.task.group.id: "id" + druid.task.datasource: "datasource" + + annotations: + task.id: "id" + task.type: "noop" + task.group.id: "id" + task.datasource: "datasource" + task.jobTemplate: base +spec: + activeDeadlineSeconds: 14400 + backoffLimit: 0 + ttlSecondsAfterFinished: 172800 + template: + metadata: + labels: + druid.k8s.peons: "true" + druid.task.id: "id" + druid.task.type: "noop" + druid.task.group.id: "id" + druid.task.datasource: "datasource" + annotations: + task: "H4sIAAAAAAAAAD2MvQ4CIRCE32VqijsTG1qLi7W+wArEbHICrmC8EN7dJf40k/lmJtNQthxgEVPKMGCvXsXgKqnm4x89FTqlKm6MBzw+YCA1nvmm8W4/TQYuxRJeBbZ17cJ3ZhvoSbzShVcu2zLOf9cS7pUl+ANlclrCzr2/AQUK0FqZAAAA" + tls.enabled: "false" + task.id: "id" + task.type: "noop" + task.group.id: "id" + task.datasource: "datasource" + task.jobTemplate: base + spec: + containers: + - command: + - sleep + - "3600" + env: + - name: "TASK_DIR" + value: "/tmp" + - name: "TASK_ID" + value: "id" + - name: "LOAD_BROADCAST_SEGMENTS" + value: "false" + - name: "TASK_JSON" + valueFrom: + fieldRef: + fieldPath: "metadata.annotations['task']" + image: one + name: primary diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml index cf16c49c5db..d6c316dcdde 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml @@ -13,6 +13,7 @@ metadata: task.type: "noop" task.group.id: "api-issued_kill_wikipedia3_omjobnbc_1000-01-01T00:00:00.000Z_2023-05-14T00:00:00.000Z_2023-05-15T17:03:01.220Z" task.datasource: "data_source" + task.jobTemplate: noop spec: activeDeadlineSeconds: 14400 backoffLimit: 0 @@ -32,6 +33,7 @@ spec: task.type: "noop" task.group.id: "api-issued_kill_wikipedia3_omjobnbc_1000-01-01T00:00:00.000Z_2023-05-14T00:00:00.000Z_2023-05-15T17:03:01.220Z" task.datasource: "data_source" + task.jobTemplate: noop spec: containers: - command: diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml index d72d0ef37b0..90ae9970959 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml @@ -13,6 +13,7 @@ metadata: task.type: "noop" task.group.id: "id" task.datasource: "datasource" + task.jobTemplate: noop spec: activeDeadlineSeconds: 14400 backoffLimit: 0 @@ -31,6 +32,7 @@ spec: task.type: "noop" task.group.id: "id" task.datasource: "datasource" + task.jobTemplate: noop spec: containers: - command: diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml index a230ac913a6..72405445414 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml @@ -13,6 +13,7 @@ metadata: task.type: "noop" task.group.id: "id" task.datasource: "datasource" + task.jobTemplate: noop spec: activeDeadlineSeconds: 14400 backoffLimit: 0 @@ -32,6 +33,7 @@ spec: task.type: "noop" task.group.id: "id" task.datasource: "datasource" + task.jobTemplate: noop spec: containers: - command: diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabledBase.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabledBase.yaml new file mode 100644 index 00000000000..0e52beac9e3 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabledBase.yaml @@ -0,0 +1,54 @@ +apiVersion: batch/v1 +kind: Job +metadata: + name: "id-3e70afe5cd823dfc7dd308eea616426b" + labels: + druid.k8s.peons: "true" + druid.task.id: "id" + druid.task.type: "noop" + druid.task.group.id: "id" + druid.task.datasource: "datasource" + annotations: + task.id: "id" + task.type: "noop" + task.group.id: "id" + task.datasource: "datasource" + task.jobTemplate: base +spec: + activeDeadlineSeconds: 14400 + backoffLimit: 0 + ttlSecondsAfterFinished: 172800 + template: + metadata: + labels: + druid.k8s.peons: "true" + druid.task.id: "id" + druid.task.type: "noop" + druid.task.group.id: "id" + druid.task.datasource: "datasource" + annotations: + task: "H4sIAAAAAAAAAD2MvQ4CIRCE32VqijsTG1qLi7W+wArEbHICrmC8EN7dJf40k/lmJtNQthxgEVPKMGCvXsXgKqnm4x89FTqlKm6MBzw+YCA1nvmm8W4/TQYuxRJeBbZ17cJ3ZhvoSbzShVcu2zLOf9cS7pUl+ANlclrCzr2/AQUK0FqZAAAA" + tls.enabled: "true" + task.id: "id" + task.type: "noop" + task.group.id: "id" + task.datasource: "datasource" + task.jobTemplate: base + spec: + containers: + - command: + - sleep + - "3600" + env: + - name: "TASK_DIR" + value: "/tmp" + - name: "TASK_ID" + value: "id" + - name: "LOAD_BROADCAST_SEGMENTS" + value: "false" + - name: "TASK_JSON" + valueFrom: + fieldRef: + fieldPath: "metadata.annotations['task']" + image: one + name: primary diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java index 7d59cb63ea5..01039375259 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java @@ -47,7 +47,7 @@ import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; +import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java index 14bd5987125..65e71f626e8 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java @@ -40,8 +40,8 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; import org.easymock.EasyMock; import org.hamcrest.CoreMatchers; diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index 38f15a840dc..ff175919586 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -52,7 +52,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; +import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; diff --git a/extensions-contrib/opentelemetry-emitter/pom.xml b/extensions-contrib/opentelemetry-emitter/pom.xml index b48e8837eff..71a67270966 100644 --- a/extensions-contrib/opentelemetry-emitter/pom.xml +++ b/extensions-contrib/opentelemetry-emitter/pom.xml @@ -39,7 +39,7 @@ 32.0.1-jre - 1.57.2 + 1.65.1 diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json index 7e625c9edb3..9cb01646f20 100644 --- a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json +++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json @@ -201,9 +201,6 @@ "zk/connected" : { "dimensions" : [], "type" : "gauge", "help": "Indicator of connection status to zookeeper."}, "zk/reconnect/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Amount of time, in seconds, that a server was disconnected from ZooKeeper before reconnecting." }, - "ingest/events/buffered" : { "dimensions" : ["serviceName", "dataSource", "bufferCapacity"], "type" : "gauge", "help": "Number of events queued in the EventReceiverFirehose's buffer"}, - "ingest/bytes/received" : { "dimensions" : ["serviceName", "dataSource"], "type" : "gauge", "help": "Number of bytes received by the EventReceiverFirehose."}, - "sys/swap/free" : { "dimensions" : [], "type" : "gauge", "help": "Free swap"}, "sys/swap/max" : { "dimensions" : [], "type" : "gauge", "help": "Max swap"}, "sys/swap/pageIn" : { "dimensions" : [], "type" : "gauge", "help": "Paged in swap"}, diff --git a/extensions-contrib/spectator-histogram/pom.xml b/extensions-contrib/spectator-histogram/pom.xml index 476e562bf4f..fd6fff411aa 100644 --- a/extensions-contrib/spectator-histogram/pom.xml +++ b/extensions-contrib/spectator-histogram/pom.xml @@ -92,6 +92,11 @@ error_prone_annotations provided + + joda-time + joda-time + provided + org.apache.druid druid-sql @@ -137,5 +142,10 @@ test-jar test + + org.easymock + easymock + test + diff --git a/extensions-contrib/spectator-histogram/src/main/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumn.java b/extensions-contrib/spectator-histogram/src/main/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumn.java index 2e54fcf0d45..8965b595e59 100644 --- a/extensions-contrib/spectator-histogram/src/main/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumn.java +++ b/extensions-contrib/spectator-histogram/src/main/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumn.java @@ -19,18 +19,12 @@ package org.apache.druid.spectator.histogram; -import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ComplexColumn; -import org.apache.druid.segment.data.ReadableOffset; - -import javax.annotation.Nullable; public class SpectatorHistogramIndexBasedComplexColumn implements ComplexColumn { private final SpectatorHistogramIndexed index; private final String typeName; - private static final Number ZERO = 0; public SpectatorHistogramIndexBasedComplexColumn(String typeName, SpectatorHistogramIndexed index) { @@ -59,72 +53,11 @@ public class SpectatorHistogramIndexBasedComplexColumn implements ComplexColumn @Override public int getLength() { - return index.size(); + return -1; } @Override public void close() { } - - @Override - public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) - { - // Use ColumnValueSelector directly so that we support being queried as a Number using - // longSum or doubleSum aggregators, the NullableNumericBufferAggregator will call isNull. - // This allows us to behave as a Number or SpectatorHistogram object. - // When queried as a Number, we're returning the count of entries in the histogram. - // As such, we can safely return 0 where the histogram is null. - return new ColumnValueSelector() - { - @Override - public boolean isNull() - { - return getObject() == null; - } - - private Number getOrZero() - { - SpectatorHistogram histogram = getObject(); - return histogram != null ? histogram : ZERO; - } - - @Override - public long getLong() - { - return getOrZero().longValue(); - } - - @Override - public float getFloat() - { - return getOrZero().floatValue(); - } - - @Override - public double getDouble() - { - return getOrZero().doubleValue(); - } - - @Nullable - @Override - public SpectatorHistogram getObject() - { - return (SpectatorHistogram) getRowValue(offset.getOffset()); - } - - @Override - public Class classOfObject() - { - return getClazz(); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("column", SpectatorHistogramIndexBasedComplexColumn.this); - } - }; - } } diff --git a/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramAggregatorTest.java b/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramAggregatorTest.java index 1c30cfc05c3..88e710645d1 100644 --- a/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramAggregatorTest.java +++ b/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramAggregatorTest.java @@ -20,8 +20,14 @@ package org.apache.druid.spectator.histogram; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.netflix.spectator.api.histogram.PercentileBuckets; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.impl.NoopInputRowParser; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.Druids; @@ -32,6 +38,9 @@ import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregationTestHelper; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorUtil; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.ResultRow; @@ -42,13 +51,17 @@ 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.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; +import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -59,6 +72,7 @@ import org.junit.runners.Parameterized; import java.io.File; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -716,6 +730,59 @@ public class SpectatorHistogramAggregatorTest extends InitializedNullHandlingTes } } + @Test + public void testBuildingAndCountingHistogramsIncrementalIndex() throws Exception + { + List dimensions = Collections.singletonList("d"); + int n = 10; + DateTime startOfDay = DateTimes.of("2000-01-01"); + List inputRows = new ArrayList<>(n); + for (int i = 1; i <= n; i++) { + String val = String.valueOf(i * 1.0d); + + inputRows.add(new MapBasedInputRow( + startOfDay.plusMinutes(i), + dimensions, + ImmutableMap.of("x", i, "d", val) + )); + } + + IncrementalIndex index = AggregationTestHelper.createIncrementalIndex( + inputRows.iterator(), + new NoopInputRowParser(null), + new AggregatorFactory[]{ + new CountAggregatorFactory("count"), + new SpectatorHistogramAggregatorFactory("histogram", "x") + }, + 0, + Granularities.NONE, + 100, + false + ); + + ImmutableList segments = ImmutableList.of( + new IncrementalIndexSegment(index, SegmentId.dummy("test")), + helper.persistIncrementalIndex(index, null) + ); + + GroupByQuery query = new GroupByQuery.Builder() + .setDataSource("test") + .setGranularity(Granularities.HOUR) + .setInterval("1970/2050") + .setAggregatorSpecs( + new DoubleSumAggregatorFactory("doubleSum", "histogram") + ).build(); + + Sequence seq = helper.runQueryOnSegmentsObjs(segments, query); + + List results = seq.toList(); + Assert.assertEquals(1, results.size()); + // Check timestamp + Assert.assertEquals(startOfDay.getMillis(), results.get(0).get(0)); + // Check doubleSum + Assert.assertEquals(n * segments.size(), (Double) results.get(0).get(1), 0.001); + } + private static void assertResultsMatch(List results, int rowNum, String expectedProduct) { ResultRow row = results.get(rowNum); diff --git a/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumnTest.java b/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumnTest.java new file mode 100644 index 00000000000..643e2e57cc5 --- /dev/null +++ b/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumnTest.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.spectator.histogram; + +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +public class SpectatorHistogramIndexBasedComplexColumnTest +{ + @Test + public void testComplexColumn() + { + final SpectatorHistogramIndexed mockIndexed = EasyMock.createMock(SpectatorHistogramIndexed.class); + EasyMock.replay(mockIndexed); + + final String typeName = "type"; + final SpectatorHistogramIndexBasedComplexColumn column = + new SpectatorHistogramIndexBasedComplexColumn("type", mockIndexed); + Assert.assertEquals(typeName, column.getTypeName()); + Assert.assertEquals(-1, column.getLength()); + + EasyMock.verify(mockIndexed); + } +} diff --git a/extensions-core/azure-extensions/pom.xml b/extensions-core/azure-extensions/pom.xml index 88a1198b61e..7f0f767af85 100644 --- a/extensions-core/azure-extensions/pom.xml +++ b/extensions-core/azure-extensions/pom.xml @@ -204,10 +204,6 @@ org.jacoco jacoco-maven-plugin - - - org/apache/druid/firehose/azure/**/* - BUNDLE diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java index 833df8ab1a5..7d0bb79c71e 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.datasketches.hll.HllSketch; import org.apache.datasketches.hll.TgtHllType; import org.apache.datasketches.hll.Union; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -34,7 +35,9 @@ import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -107,6 +110,8 @@ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory @Override public Aggregator factorize(final ColumnSelectorFactory columnSelectorFactory) { + validateInputs(columnSelectorFactory.getColumnCapabilities(getFieldName())); + final ColumnValueSelector selector = columnSelectorFactory.makeColumnValueSelector(getFieldName()); return new HllSketchMergeAggregator(selector, getLgK(), TgtHllType.valueOf(getTgtHllType())); } @@ -115,6 +120,8 @@ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory @Override public BufferAggregator factorizeBuffered(final ColumnSelectorFactory columnSelectorFactory) { + validateInputs(columnSelectorFactory.getColumnCapabilities(getFieldName())); + final ColumnValueSelector selector = columnSelectorFactory.makeColumnValueSelector(getFieldName()); return new HllSketchMergeBufferAggregator( selector, @@ -133,6 +140,7 @@ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory @Override public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory) { + validateInputs(selectorFactory.getColumnCapabilities(getFieldName())); return new HllSketchMergeVectorAggregator( selectorFactory, getFieldName(), @@ -142,6 +150,34 @@ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory ); } + /** + * Validates whether the aggregator supports the input column type. + * Supported column types are complex types of HLLSketch, HLLSketchBuild, HLLSketchMerge, as well as UNKNOWN_COMPLEX. + * @param capabilities + */ + private void validateInputs(@Nullable ColumnCapabilities capabilities) + { + if (capabilities != null) { + final ColumnType type = capabilities.toColumnType(); + boolean isSupportedComplexType = ValueType.COMPLEX.equals(type.getType()) && + ( + HllSketchModule.TYPE_NAME.equals(type.getComplexTypeName()) || + HllSketchModule.BUILD_TYPE_NAME.equals(type.getComplexTypeName()) || + HllSketchModule.MERGE_TYPE_NAME.equals(type.getComplexTypeName()) || + type.getComplexTypeName() == null + ); + if (!isSupportedComplexType) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build( + "Using aggregator [%s] is not supported for complex columns with type [%s].", + getIntermediateType().getComplexTypeName(), + type + ); + } + } + } + @Override public int getMaxIntermediateSize() { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java index 757674d6aa6..7d303d27274 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java @@ -21,28 +21,68 @@ package org.apache.druid.query.aggregation.datasketches.hll.sql; import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.type.CastedLiteralOperandTypeCheckers; import org.apache.calcite.sql.type.InferTypes; +import org.apache.calcite.sql.type.OperandTypes; +import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchMergeAggregatorFactory; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.OperatorConversions; +import org.apache.druid.sql.calcite.table.RowSignatures; import java.util.Collections; +/** + * Approximate count distinct aggregator using HLL sketches. + * Supported column types: String, Numeric, HLLSketchMerge, HLLSketchBuild. + */ public class HllSketchApproxCountDistinctSqlAggregator extends HllSketchBaseSqlAggregator implements SqlAggregator { public static final String NAME = "APPROX_COUNT_DISTINCT_DS_HLL"; + + private static final SqlSingleOperandTypeChecker AGGREGATED_COLUMN_TYPE_CHECKER = OperandTypes.or( + OperandTypes.STRING, + OperandTypes.NUMERIC, + RowSignatures.complexTypeChecker(HllSketchMergeAggregatorFactory.TYPE), + RowSignatures.complexTypeChecker(HllSketchBuildAggregatorFactory.TYPE) + ); + private static final SqlAggFunction FUNCTION_INSTANCE = OperatorConversions.aggregatorBuilder(NAME) - .operandNames("column", "lgK", "tgtHllType") - .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING) .operandTypeInference(InferTypes.VARCHAR_1024) - .requiredOperandCount(1) - .literalOperands(1, 2) + .operandTypeChecker( + OperandTypes.or( + // APPROX_COUNT_DISTINCT_DS_HLL(column) + AGGREGATED_COLUMN_TYPE_CHECKER, + // APPROX_COUNT_DISTINCT_DS_HLL(column, lgk) + OperandTypes.and( + OperandTypes.sequence( + StringUtils.format("'%s(column, lgk)'", NAME), + AGGREGATED_COLUMN_TYPE_CHECKER, + CastedLiteralOperandTypeCheckers.POSITIVE_INTEGER_LITERAL + ), + OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.EXACT_NUMERIC) + ), + // APPROX_COUNT_DISTINCT_DS_HLL(column, lgk, tgtHllType) + OperandTypes.and( + OperandTypes.sequence( + StringUtils.format("'%s(column, lgk, tgtHllType)'", NAME), + AGGREGATED_COLUMN_TYPE_CHECKER, + CastedLiteralOperandTypeCheckers.POSITIVE_INTEGER_LITERAL, + OperandTypes.STRING + ), + OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.EXACT_NUMERIC, SqlTypeFamily.STRING) + ) + ) + ) .returnTypeNonNull(SqlTypeName.BIGINT) .functionCategory(SqlFunctionCategory.NUMERIC) .build(); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java index d221b72ac1c..15221c0f6f8 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java @@ -31,6 +31,7 @@ import org.apache.druid.query.aggregation.datasketches.SketchQueryContext; import org.apache.druid.query.aggregation.datasketches.hll.HllSketchAggregatorFactory; import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory; import org.apache.druid.query.aggregation.datasketches.hll.HllSketchMergeAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchModule; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.column.ColumnType; @@ -40,6 +41,7 @@ import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; import org.apache.druid.sql.calcite.planner.Calcites; +import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.InputAccessor; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; @@ -115,7 +117,7 @@ public abstract class HllSketchBaseSqlAggregator implements SqlAggregator if (columnArg.isDirectColumnAccess() && inputAccessor.getInputRowSignature() .getColumnType(columnArg.getDirectColumn()) - .map(type -> type.is(ValueType.COMPLEX)) + .map(this::isValidComplexInputType) .orElse(false)) { aggregatorFactory = new HllSketchMergeAggregatorFactory( aggregatorName, @@ -154,6 +156,15 @@ public abstract class HllSketchBaseSqlAggregator implements SqlAggregator } if (inputType.is(ValueType.COMPLEX)) { + if (!isValidComplexInputType(inputType)) { + plannerContext.setPlanningError( + "Using APPROX_COUNT_DISTINCT() or enabling approximation with COUNT(DISTINCT) is not supported for" + + " column type [%s]. You can disable approximation by setting [%s: false] in the query context.", + columnArg.getDruidType(), + PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT + ); + return null; + } aggregatorFactory = new HllSketchMergeAggregatorFactory( aggregatorName, dimensionSpec.getOutputName(), @@ -192,4 +203,11 @@ public abstract class HllSketchBaseSqlAggregator implements SqlAggregator boolean finalizeAggregations, AggregatorFactory aggregatorFactory ); + + private boolean isValidComplexInputType(ColumnType columnType) + { + return HllSketchMergeAggregatorFactory.TYPE.equals(columnType) || + HllSketchModule.TYPE_NAME.equals(columnType.getComplexTypeName()) || + HllSketchModule.BUILD_TYPE_NAME.equals(columnType.getComplexTypeName()); + } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java index 211373e873b..b24e382ec0a 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java @@ -27,7 +27,7 @@ import org.apache.datasketches.common.Util; import org.apache.datasketches.theta.SetOperation; import org.apache.datasketches.theta.Union; import org.apache.datasketches.thetacommon.ThetaUtil; -import org.apache.druid.error.InvalidInput; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.aggregation.AggregateCombiner; import org.apache.druid.query.aggregation.Aggregator; @@ -41,6 +41,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -80,10 +81,7 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { - ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName); - if (capabilities != null && capabilities.isArray()) { - throw InvalidInput.exception("ARRAY types are not supported for theta sketch"); - } + validateInputs(metricFactory.getColumnCapabilities(fieldName)); BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName); return new SketchAggregator(selector, size); } @@ -91,10 +89,7 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory @Override public AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory) { - ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName); - if (capabilities != null && capabilities.isArray()) { - throw InvalidInput.exception("ARRAY types are not supported for theta sketch"); - } + validateInputs(metricFactory.getColumnCapabilities(fieldName)); BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName); final SketchAggregator aggregator = new SketchAggregator(selector, size); return new AggregatorAndSize(aggregator, aggregator.getInitialSizeBytes()); @@ -104,10 +99,7 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName); - if (capabilities != null && capabilities.isArray()) { - throw InvalidInput.exception("ARRAY types are not supported for theta sketch"); - } + validateInputs(metricFactory.getColumnCapabilities(fieldName)); BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName); return new SketchBufferAggregator(selector, size, getMaxIntermediateSizeWithNulls()); } @@ -115,9 +107,41 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory @Override public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory) { + validateInputs(selectorFactory.getColumnCapabilities(fieldName)); return new SketchVectorAggregator(selectorFactory, fieldName, size, getMaxIntermediateSizeWithNulls()); } + /** + * Validates whether the aggregator supports the input column type. + * Unsupported column types are: + *
    + *
  • Arrays
  • + *
  • Complex types of thetaSketch, thetaSketchMerge, thetaSketchBuild.
  • + *
+ * @param capabilities + */ + private void validateInputs(@Nullable ColumnCapabilities capabilities) + { + if (capabilities != null) { + boolean isUnsupportedComplexType = capabilities.is(ValueType.COMPLEX) && !( + SketchModule.THETA_SKETCH_TYPE.equals(capabilities.toColumnType()) || + SketchModule.MERGE_TYPE.equals(capabilities.toColumnType()) || + SketchModule.BUILD_TYPE.equals(capabilities.toColumnType()) + ); + + if (capabilities.isArray() || isUnsupportedComplexType) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build( + "Unsupported input [%s] of type [%s] for aggregator [%s].", + getFieldName(), + capabilities.asTypeString(), + getIntermediateType() + ); + } + } + } + @Override public boolean canVectorize(ColumnInspector columnInspector) { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchApproxCountDistinctSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchApproxCountDistinctSqlAggregator.java index eac77901f1d..5ecd289c728 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchApproxCountDistinctSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchApproxCountDistinctSqlAggregator.java @@ -21,27 +21,55 @@ package org.apache.druid.query.aggregation.datasketches.theta.sql; import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.type.CastedLiteralOperandTypeCheckers; import org.apache.calcite.sql.type.InferTypes; +import org.apache.calcite.sql.type.OperandTypes; +import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.OperatorConversions; +import org.apache.druid.sql.calcite.table.RowSignatures; import java.util.Collections; +/** + * Approximate count distinct aggregator using theta sketches. + * Supported column types: String, Numeric, Theta Sketch. + */ public class ThetaSketchApproxCountDistinctSqlAggregator extends ThetaSketchBaseSqlAggregator implements SqlAggregator { public static final String NAME = "APPROX_COUNT_DISTINCT_DS_THETA"; + + private static final SqlSingleOperandTypeChecker AGGREGATED_COLUMN_TYPE_CHECKER = OperandTypes.or( + OperandTypes.STRING, + OperandTypes.NUMERIC, + RowSignatures.complexTypeChecker(SketchModule.THETA_SKETCH_TYPE) + ); + private static final SqlAggFunction FUNCTION_INSTANCE = OperatorConversions.aggregatorBuilder(NAME) - .operandNames("column", "size") - .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC) .operandTypeInference(InferTypes.VARCHAR_1024) - .requiredOperandCount(1) - .literalOperands(1) + .operandTypeChecker( + OperandTypes.or( + // APPROX_COUNT_DISTINCT_DS_THETA(expr) + AGGREGATED_COLUMN_TYPE_CHECKER, + // APPROX_COUNT_DISTINCT_DS_THETA(expr, size) + OperandTypes.and( + OperandTypes.sequence( + StringUtils.format("'%s(expr, size)'", NAME), + AGGREGATED_COLUMN_TYPE_CHECKER, + CastedLiteralOperandTypeCheckers.POSITIVE_INTEGER_LITERAL + ), + OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.EXACT_NUMERIC) + ) + ) + ) .returnTypeNonNull(SqlTypeName.BIGINT) .functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION) .build(); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java index bf35cd665ae..1f45f31496a 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java @@ -29,6 +29,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.datasketches.SketchQueryContext; import org.apache.druid.query.aggregation.datasketches.theta.SketchAggregatorFactory; import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.column.ColumnType; @@ -38,6 +39,7 @@ import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; import org.apache.druid.sql.calcite.planner.Calcites; +import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.InputAccessor; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; @@ -95,7 +97,11 @@ public abstract class ThetaSketchBaseSqlAggregator implements SqlAggregator if (columnArg.isDirectColumnAccess() && inputAccessor.getInputRowSignature() .getColumnType(columnArg.getDirectColumn()) - .map(type -> type.is(ValueType.COMPLEX)) + .map(type -> ( + SketchModule.THETA_SKETCH_TYPE.equals(type) || + SketchModule.MERGE_TYPE.equals(type) || + SketchModule.BUILD_TYPE.equals(type) + )) .orElse(false)) { aggregatorFactory = new SketchMergeAggregatorFactory( aggregatorName, @@ -116,6 +122,16 @@ public abstract class ThetaSketchBaseSqlAggregator implements SqlAggregator ); } + if (inputType.is(ValueType.COMPLEX)) { + plannerContext.setPlanningError( + "Using APPROX_COUNT_DISTINCT() or enabling approximation with COUNT(DISTINCT) is not supported for" + + " column type [%s]. You can disable approximation by setting [%s: false] in the query context.", + columnArg.getDruidType(), + PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT + ); + return null; + } + final DimensionSpec dimensionSpec; if (columnArg.isDirectColumnAccess()) { diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactoryTest.java index 101b25b99be..fcecef62d4a 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactoryTest.java @@ -22,10 +22,17 @@ package org.apache.druid.query.aggregation.datasketches.hll; import com.fasterxml.jackson.databind.ObjectMapper; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.datasketches.hll.TgtHllType; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.TestColumnSelectorFactory; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.vector.TestVectorColumnSelectorFactory; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -45,6 +52,9 @@ public class HllSketchMergeAggregatorFactoryTest private HllSketchMergeAggregatorFactory targetRound; private HllSketchMergeAggregatorFactory targetNoRound; + private ColumnSelectorFactory metricFactory; + private VectorColumnSelectorFactory vectorFactory; + @Before public void setUp() { @@ -66,6 +76,10 @@ public class HllSketchMergeAggregatorFactoryTest SHOULD_FINALIZE, !ROUND ); + + final ColumnCapabilitiesImpl columnCapabilities = ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA); + metricFactory = new TestColumnSelectorFactory().addCapabilities(FIELD_NAME, columnCapabilities); + vectorFactory = new TestVectorColumnSelectorFactory().addCapabilities(FIELD_NAME, columnCapabilities); } @Test(expected = AggregatorFactoryNotMergeableException.class) @@ -291,4 +305,39 @@ public class HllSketchMergeAggregatorFactoryTest Assert.assertEquals(factory, factory.withName(targetRound.getName())); Assert.assertEquals("newTest", factory.withName("newTest").getName()); } + + @Test + public void testFactorizeOnUnsupportedComplexColumn() + { + final ColumnSelectorFactory metricFactory = new TestColumnSelectorFactory() + .addCapabilities( + FIELD_NAME, + ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA) + ); + Throwable exception = Assert.assertThrows(DruidException.class, () -> targetRound.factorize(metricFactory)); + Assert.assertEquals( + "Using aggregator [HLLSketchMerge] is not supported for complex columns with type [COMPLEX].", + exception.getMessage() + ); + } + + @Test + public void testFactorizeBufferedOnUnsupportedComplexColumn() + { + Throwable exception = Assert.assertThrows(DruidException.class, () -> targetRound.factorizeBuffered(metricFactory)); + Assert.assertEquals( + "Using aggregator [HLLSketchMerge] is not supported for complex columns with type [COMPLEX].", + exception.getMessage() + ); + } + + @Test + public void testFactorizeVectorOnUnsupportedComplexColumn() + { + Throwable exception = Assert.assertThrows(DruidException.class, () -> targetRound.factorizeVector(vectorFactory)); + Assert.assertEquals( + "Using aggregator [HLLSketchMerge] is not supported for complex columns with type [COMPLEX].", + exception.getMessage() + ); + } } 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 2907d6f8bb8..edb7dc5a11f 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 @@ -24,7 +24,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.inject.Injector; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.DruidInjectorBuilder; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -86,6 +88,7 @@ import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.sql.guice.SqlModule; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.DateTimeZone; import org.joda.time.Period; import org.junit.Assert; @@ -100,6 +103,10 @@ import java.util.stream.Collectors; @SqlTestFrameworkConfig.ComponentSupplier(HllSketchComponentSupplier.class) public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest { + static { + NullHandling.initializeForTests(); + } + private static final boolean ROUND = true; // For testHllSketchPostAggsGroupBy, testHllSketchPostAggsTimeseries @@ -300,6 +307,15 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest .size(0) .build(), index + ).add( + DataSegment.builder() + .dataSource(CalciteTests.WIKIPEDIA_FIRST_LAST) + .interval(Intervals.of("2015-09-12/2015-09-13")) + .version("1") + .shardSpec(new NumberedShardSpec(0, 0)) + .size(0) + .build(), + TestDataBuilder.makeWikipediaIndexWithAggregation(tempDirProducer.newTempFolder()) ); } } @@ -508,6 +524,33 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest ); } + @Test + public void testApproxCountDistinctOnUnsupportedComplexColumn() + { + assertQueryIsUnplannable( + "SELECT COUNT(distinct double_first_added) FROM druid.wikipedia_first_last", + "Query could not be planned. A possible reason is [Using APPROX_COUNT_DISTINCT() or enabling " + + "approximation with COUNT(DISTINCT) is not supported for column type [COMPLEX]." + + " You can disable approximation by setting [useApproximateCountDistinct: false] in the query context." + ); + } + + @Test + public void testApproxCountDistinctFunctionOnUnsupportedComplexColumn() + { + DruidException druidException = Assert.assertThrows( + DruidException.class, + () -> testQuery( + "SELECT APPROX_COUNT_DISTINCT_DS_HLL(double_first_added) FROM druid.wikipedia_first_last", + ImmutableList.of(), + ImmutableList.of() + ) + ); + Assert.assertTrue(druidException.getMessage().contains( + "Cannot apply 'APPROX_COUNT_DISTINCT_DS_HLL' to arguments of type 'APPROX_COUNT_DISTINCT_DS_HLL(>)'" + )); + } + @Test public void testHllSketchFilteredAggregatorsGroupBy() { diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java index 1d70ff30f25..23887652a73 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java @@ -20,6 +20,7 @@ package org.apache.druid.query.aggregation.datasketches.theta; import com.google.common.collect.ImmutableList; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.Druids; import org.apache.druid.query.aggregation.AggregatorAndSize; @@ -32,10 +33,15 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.TestColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.vector.TestVectorColumnSelectorFactory; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.easymock.EasyMock; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; public class SketchAggregatorFactoryTest @@ -46,6 +52,17 @@ public class SketchAggregatorFactoryTest private static final SketchMergeAggregatorFactory AGGREGATOR_32768 = new SketchMergeAggregatorFactory("x", "x", 32768, null, false, null); + private ColumnSelectorFactory metricFactory; + private VectorColumnSelectorFactory vectorFactory; + + @Before + public void setup() + { + final ColumnCapabilitiesImpl columnCapabilities = ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA); + metricFactory = new TestColumnSelectorFactory().addCapabilities("x", columnCapabilities); + vectorFactory = new TestVectorColumnSelectorFactory().addCapabilities("x", columnCapabilities); + } + @Test public void testGuessAggregatorHeapFootprint() { @@ -168,4 +185,32 @@ public class SketchAggregatorFactoryTest Assert.assertEquals(AGGREGATOR_16384, AGGREGATOR_16384.withName("x")); Assert.assertEquals("newTest", AGGREGATOR_16384.withName("newTest").getName()); } + + @Test + public void testFactorizeOnUnsupportedComplexColumn() + { + Throwable exception = Assert.assertThrows(DruidException.class, () -> AGGREGATOR_16384.factorize(metricFactory)); + Assert.assertEquals("Unsupported input [x] of type [COMPLEX] for aggregator [COMPLEX].", exception.getMessage()); + } + + @Test + public void testFactorizeWithSizeOnUnsupportedComplexColumn() + { + Throwable exception = Assert.assertThrows(DruidException.class, () -> AGGREGATOR_16384.factorizeWithSize(metricFactory)); + Assert.assertEquals("Unsupported input [x] of type [COMPLEX] for aggregator [COMPLEX].", exception.getMessage()); + } + + @Test + public void testFactorizeBufferedOnUnsupportedComplexColumn() + { + Throwable exception = Assert.assertThrows(DruidException.class, () -> AGGREGATOR_16384.factorizeBuffered(metricFactory)); + Assert.assertEquals("Unsupported input [x] of type [COMPLEX] for aggregator [COMPLEX].", exception.getMessage()); + } + + @Test + public void testFactorizeVectorOnUnsupportedComplexColumn() + { + Throwable exception = Assert.assertThrows(DruidException.class, () -> AGGREGATOR_16384.factorizeVector(vectorFactory)); + Assert.assertEquals("Unsupported input [x] of type [COMPLEX] for aggregator [COMPLEX].", exception.getMessage()); + } } 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 247f924357a..7afd2710ccd 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 @@ -23,7 +23,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.inject.Injector; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.DruidInjectorBuilder; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.PeriodGranularity; @@ -71,6 +73,7 @@ import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.sql.guice.SqlModule; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.DateTimeZone; import org.joda.time.Period; import org.junit.Assert; @@ -158,6 +161,15 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest .size(0) .build(), index + ).add( + DataSegment.builder() + .dataSource(CalciteTests.WIKIPEDIA_FIRST_LAST) + .interval(Intervals.of("2015-09-12/2015-09-13")) + .version("1") + .shardSpec(new NumberedShardSpec(0, 0)) + .size(0) + .build(), + TestDataBuilder.makeWikipediaIndexWithAggregation(tempDirProducer.newTempFolder()) ); } } @@ -373,6 +385,33 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest ); } + @Test + public void testApproxCountDistinctOnUnsupportedComplexColumn() + { + assertQueryIsUnplannable( + "SELECT COUNT(distinct double_first_added) FROM druid.wikipedia_first_last", + "Query could not be planned. A possible reason is [Using APPROX_COUNT_DISTINCT() or enabling " + + "approximation with COUNT(DISTINCT) is not supported for column type [COMPLEX]." + + " You can disable approximation by setting [useApproximateCountDistinct: false] in the query context." + ); + } + + @Test + public void testApproxCountDistinctFunctionOnUnsupportedComplexColumn() + { + DruidException druidException = Assert.assertThrows( + DruidException.class, + () -> testQuery( + "SELECT APPROX_COUNT_DISTINCT_DS_THETA(double_first_added) FROM druid.wikipedia_first_last", + ImmutableList.of(), + ImmutableList.of() + ) + ); + Assert.assertTrue(druidException.getMessage().contains( + "Cannot apply 'APPROX_COUNT_DISTINCT_DS_THETA' to arguments of type 'APPROX_COUNT_DISTINCT_DS_THETA(>)'" + )); + } + @Test public void testThetaSketchPostAggs() { diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java index 0c7dff247c6..33450a3efcc 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java @@ -267,7 +267,7 @@ public class CoordinatorPollingBasicAuthenticatorCacheManager implements BasicAu writeUserMapToDisk(prefix, userMapBytes); } } else { - LOG.info("Empty cached serialized user map retrieved, authenticator - %s", prefix); + LOG.debug("Empty cached serialized user map retrieved, authenticator - %s", prefix); } return userMap; } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 58f982e71b4..15b77be307d 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2908,10 +2908,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase .put( ScanQuery.class, new ScanQueryRunnerFactory( - new ScanQueryQueryToolChest( - new ScanQueryConfig(), - new DefaultGenericQueryMetricsFactory() - ), + new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()), new ScanQueryEngine(), new ScanQueryConfig() ) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java index da04e3ab0a6..60d8f686a28 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java @@ -33,9 +33,9 @@ import org.apache.druid.initialization.DruidModule; import org.apache.druid.initialization.Initialization; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index a30e9686087..d2d5cc657e6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -20,7 +20,6 @@ package org.apache.druid.msq.exec; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -1357,7 +1356,10 @@ public class ControllerImpl implements Controller * Publish the list of segments. Additionally, if {@link DataSourceMSQDestination#isReplaceTimeChunks()}, * also drop all other segments within the replacement intervals. */ - private void publishAllSegments(final Set segments) throws IOException + private void publishAllSegments( + final Set segments, + Function, Set> compactionStateAnnotateFunction + ) throws IOException { final DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); @@ -1413,7 +1415,7 @@ public class ControllerImpl implements Controller } performSegmentPublish( context.taskActionClient(), - createOverwriteAction(taskLockType, segmentsWithTombstones) + createOverwriteAction(taskLockType, compactionStateAnnotateFunction.apply(segmentsWithTombstones)) ); } } else if (!segments.isEmpty()) { @@ -1543,6 +1545,7 @@ public class ControllerImpl implements Controller if (MSQControllerTask.isIngestion(querySpec)) { // Publish segments if needed. final StageId finalStageId = queryKernel.getStageId(queryDef.getFinalStageDefinition().getStageNumber()); + Function, Set> compactionStateAnnotateFunction = Function.identity(); @SuppressWarnings("unchecked") Set segments = (Set) queryKernel.getResultObjectForStage(finalStageId); @@ -1553,7 +1556,7 @@ public class ControllerImpl implements Controller Tasks.DEFAULT_STORE_COMPACTION_STATE ); - if (!segments.isEmpty() && storeCompactionState) { + if (storeCompactionState) { DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); if (!destination.isReplaceTimeChunks()) { // Store compaction state only for replace queries. @@ -1565,20 +1568,21 @@ public class ControllerImpl implements Controller DataSchema dataSchema = ((SegmentGeneratorFrameProcessorFactory) queryKernel .getStageDefinition(finalStageId).getProcessorFactory()).getDataSchema(); - ShardSpec shardSpec = segments.stream().findFirst().get().getShardSpec(); + ShardSpec shardSpec = segments.isEmpty() ? null : segments.stream().findFirst().get().getShardSpec(); + ClusterBy clusterBy = queryKernel.getStageDefinition(finalStageId).getClusterBy(); - Function, Set> compactionStateAnnotateFunction = addCompactionStateToSegments( + compactionStateAnnotateFunction = addCompactionStateToSegments( querySpec, context.jsonMapper(), dataSchema, shardSpec, + clusterBy, queryDef.getQueryId() ); - segments = compactionStateAnnotateFunction.apply(segments); } } log.info("Query [%s] publishing %d segments.", queryDef.getQueryId(), segments.size()); - publishAllSegments(segments); + publishAllSegments(segments, compactionStateAnnotateFunction); } else if (MSQControllerTask.isExport(querySpec)) { // Write manifest file. ExportMSQDestination destination = (ExportMSQDestination) querySpec.getDestination(); @@ -1624,33 +1628,49 @@ public class ControllerImpl implements Controller MSQSpec querySpec, ObjectMapper jsonMapper, DataSchema dataSchema, - ShardSpec shardSpec, + @Nullable ShardSpec shardSpec, + @Nullable ClusterBy clusterBy, String queryId ) { final MSQTuningConfig tuningConfig = querySpec.getTuningConfig(); PartitionsSpec partitionSpec; - if (Objects.equals(shardSpec.getType(), ShardSpec.Type.RANGE)) { - List partitionDimensions = ((DimensionRangeShardSpec) shardSpec).getDimensions(); + // shardSpec is absent in the absence of segments, which happens when only tombstones are generated by an + // MSQControllerTask. + if (shardSpec != null) { + if (Objects.equals(shardSpec.getType(), ShardSpec.Type.RANGE)) { + List partitionDimensions = ((DimensionRangeShardSpec) shardSpec).getDimensions(); + partitionSpec = new DimensionRangePartitionsSpec( + tuningConfig.getRowsPerSegment(), + null, + partitionDimensions, + false + ); + } else if (Objects.equals(shardSpec.getType(), ShardSpec.Type.NUMBERED)) { + // MSQ tasks don't use maxTotalRows. Hence using LONG.MAX_VALUE. + partitionSpec = new DynamicPartitionsSpec(tuningConfig.getRowsPerSegment(), Long.MAX_VALUE); + } else { + // SingleDimenionShardSpec and other shard specs are never created in MSQ. + throw new MSQException( + UnknownFault.forMessage( + StringUtils.format( + "Query[%s] cannot store compaction state in segments as shard spec of unsupported type[%s].", + queryId, + shardSpec.getType() + ))); + } + } else if (clusterBy != null && !clusterBy.getColumns().isEmpty()) { partitionSpec = new DimensionRangePartitionsSpec( tuningConfig.getRowsPerSegment(), null, - partitionDimensions, + clusterBy.getColumns() + .stream() + .map(KeyColumn::columnName).collect(Collectors.toList()), false ); - } else if (Objects.equals(shardSpec.getType(), ShardSpec.Type.NUMBERED)) { - // MSQ tasks don't use maxTotalRows. Hence using LONG.MAX_VALUE. - partitionSpec = new DynamicPartitionsSpec(tuningConfig.getRowsPerSegment(), Long.MAX_VALUE); } else { - // SingleDimenionShardSpec and other shard specs are never created in MSQ. - throw new MSQException( - UnknownFault.forMessage( - StringUtils.format( - "Query[%s] cannot store compaction state in segments as shard spec of unsupported type[%s].", - queryId, - shardSpec.getType() - ))); + partitionSpec = new DynamicPartitionsSpec(tuningConfig.getRowsPerSegment(), Long.MAX_VALUE); } Granularity segmentGranularity = ((DataSourceMSQDestination) querySpec.getDestination()) @@ -1671,13 +1691,26 @@ public class ControllerImpl implements Controller : new ClientCompactionTaskTransformSpec( dataSchema.getTransformSpec().getFilter() ).asMap(jsonMapper); - List metricsSpec = dataSchema.getAggregators() == null - ? null - : jsonMapper.convertValue( - dataSchema.getAggregators(), - new TypeReference>() {} - ); + List metricsSpec = Collections.emptyList(); + if (querySpec.getQuery() instanceof GroupByQuery) { + // For group-by queries, the aggregators are transformed to their combining factories in the dataschema, resulting + // in a mismatch between schema in compaction spec and the one in compaction state. Sourcing the original + // AggregatorFactory definition for aggregators in the dataSchema, therefore, directly from the querySpec. + GroupByQuery groupByQuery = (GroupByQuery) querySpec.getQuery(); + // Collect all aggregators that are part of the current dataSchema, since a non-rollup query (isRollup() is false) + // moves metrics columns to dimensions in the final schema. + Set aggregatorsInDataSchema = Arrays.stream(dataSchema.getAggregators()) + .map(AggregatorFactory::getName) + .collect( + Collectors.toSet()); + metricsSpec = new ArrayList<>( + groupByQuery.getAggregatorSpecs() + .stream() + .filter(aggregatorFactory -> aggregatorsInDataSchema.contains(aggregatorFactory.getName())) + .collect(Collectors.toList()) + ); + } IndexSpec indexSpec = tuningConfig.getIndexSpec(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java index e960a8aea29..0fef9d32e6d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java @@ -53,7 +53,7 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.QueryContext; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.indexing.OverlordClient; -import org.apache.druid.segment.realtime.firehose.ChatHandler; +import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.server.DruidNode; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index ac43e7c864b..7b4d3235dc0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -49,7 +49,9 @@ import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.expression.TimestampFloorExprMacro; @@ -58,11 +60,13 @@ import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; @@ -123,7 +127,8 @@ public class MSQCompactionRunner implements CompactionRunner */ @Override public CompactionConfigValidationResult validateCompactionTask( - CompactionTask compactionTask + CompactionTask compactionTask, + Map intervalToDataSchemaMap ) { List validationResults = new ArrayList<>(); @@ -139,11 +144,55 @@ public class MSQCompactionRunner implements CompactionRunner )); } validationResults.add(ClientCompactionRunnerInfo.validateMaxNumTasksForMSQ(compactionTask.getContext())); - validationResults.add(ClientCompactionRunnerInfo.validateMetricsSpecForMSQ(compactionTask.getMetricsSpec())); + validationResults.add(validateRolledUpSegments(intervalToDataSchemaMap)); return validationResults.stream() .filter(result -> !result.isValid()) .findFirst() - .orElse(new CompactionConfigValidationResult(true, null)); + .orElse(CompactionConfigValidationResult.success()); + } + + /** + * Valides that there are no rolled-up segments where either: + *
    + *
  • aggregator factory differs from its combining factory
  • + *
  • input col name is different from the output name (non-idempotent)
  • + *
+ */ + private CompactionConfigValidationResult validateRolledUpSegments(Map intervalToDataSchemaMap) + { + for (Map.Entry intervalDataSchema : intervalToDataSchemaMap.entrySet()) { + if (intervalDataSchema.getValue() instanceof CombinedDataSchema) { + CombinedDataSchema combinedDataSchema = (CombinedDataSchema) intervalDataSchema.getValue(); + if (combinedDataSchema.hasRolledUpSegments()) { + for (AggregatorFactory aggregatorFactory : combinedDataSchema.getAggregators()) { + // This is a conservative check as existing rollup may have been idempotent but the aggregator provided in + // compaction spec isn't. This would get properly compacted yet fails in the below pre-check. + if ( + !( + aggregatorFactory.getClass().equals(aggregatorFactory.getCombiningFactory().getClass()) && + ( + aggregatorFactory.requiredFields().isEmpty() || + (aggregatorFactory.requiredFields().size() == 1 && + aggregatorFactory.requiredFields() + .get(0) + .equals(aggregatorFactory.getName())) + ) + ) + ) { + // MSQ doesn't support rolling up already rolled-up segments when aggregate column name is different from + // the aggregated column name. This is because the aggregated values would then get overwritten by new + // values and the existing values would be lost. Note that if no rollup is specified in an index spec, + // the default value is true. + return CompactionConfigValidationResult.failure( + "MSQ: Rolled-up segments in compaction interval[%s].", + intervalDataSchema.getKey() + ); + } + } + } + } + } + return CompactionConfigValidationResult.success(); } @Override @@ -291,6 +340,10 @@ public class MSQCompactionRunner implements CompactionRunner for (DimensionSchema dimensionSchema : dataSchema.getDimensionsSpec().getDimensions()) { rowSignatureBuilder.add(dimensionSchema.getName(), ColumnType.fromString(dimensionSchema.getTypeName())); } + // There can be columns that are part of metricsSpec for a datasource. + for (AggregatorFactory aggregatorFactory : dataSchema.getAggregators()) { + rowSignatureBuilder.add(aggregatorFactory.getName(), aggregatorFactory.getIntermediateType()); + } return rowSignatureBuilder.build(); } @@ -354,15 +407,30 @@ public class MSQCompactionRunner implements CompactionRunner private static Query buildScanQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema) { RowSignature rowSignature = getRowSignature(dataSchema); - return new Druids.ScanQueryBuilder().dataSource(dataSchema.getDataSource()) - .columns(rowSignature.getColumnNames()) - .virtualColumns(getVirtualColumns(dataSchema, interval)) - .columnTypes(rowSignature.getColumnTypes()) - .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval))) - .legacy(false) - .filters(dataSchema.getTransformSpec().getFilter()) - .context(compactionTask.getContext()) - .build(); + Druids.ScanQueryBuilder scanQueryBuilder = new Druids.ScanQueryBuilder() + .dataSource(dataSchema.getDataSource()) + .columns(rowSignature.getColumnNames()) + .virtualColumns(getVirtualColumns(dataSchema, interval)) + .columnTypes(rowSignature.getColumnTypes()) + .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval))) + .filters(dataSchema.getTransformSpec().getFilter()) + .context(compactionTask.getContext()); + + if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { + List orderByColumnSpecs = getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec()); + + scanQueryBuilder.orderBy( + orderByColumnSpecs + .stream() + .map(orderByColumnSpec -> + new ScanQuery.OrderBy( + orderByColumnSpec.getDimension(), + ScanQuery.Order.fromString(orderByColumnSpec.getDirection().toString()) + )) + .collect(Collectors.toList()) + ); + } + return scanQueryBuilder.build(); } private static boolean isGroupBy(DataSchema dataSchema) @@ -469,7 +537,10 @@ public class MSQCompactionRunner implements CompactionRunner ); } // Similar to compaction using the native engine, don't finalize aggregations. + // Used for writing the data schema during segment generation phase. context.putIfAbsent(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false); + // Add appropriate finalization to native query context i.e. for the GroupBy query + context.put(QueryContexts.FINALIZE_KEY, false); // Only scalar or array-type dimensions are allowed as grouping keys. context.putIfAbsent(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false); return context; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/ControllerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/ControllerChatHandler.java index bf3dd4a6bf1..d161b01bd0b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/ControllerChatHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/ControllerChatHandler.java @@ -22,7 +22,7 @@ package org.apache.druid.msq.indexing.client; import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.indexing.IndexerResourcePermissionMapper; import org.apache.druid.msq.rpc.ControllerResource; -import org.apache.druid.segment.realtime.firehose.ChatHandler; +import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.server.security.AuthorizerMapper; public class ControllerChatHandler extends ControllerResource implements ChatHandler diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java index 4ef6ab077ca..70d1ab11d38 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java @@ -33,8 +33,8 @@ import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.msq.statistics.serde.ClusterByStatisticsSnapshotSerde; -import org.apache.druid.segment.realtime.firehose.ChatHandler; -import org.apache.druid.segment.realtime.firehose.ChatHandlers; +import org.apache.druid.segment.realtime.ChatHandler; +import org.apache.druid.segment.realtime.ChatHandlers; import org.apache.druid.server.security.Action; import org.apache.druid.utils.CloseableUtils; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java index 3817e63ca49..16f9deff63d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java @@ -177,9 +177,8 @@ public class SegmentGeneratorFrameProcessorFactory // Create directly, without using AppenderatorsManager, because we need different memory overrides due to // using one Appenderator per processing thread instead of per task. - // Note: "createOffline" ignores the batchProcessingMode and always acts like CLOSED_SEGMENTS_SINKS. final Appenderator appenderator = - Appenderators.createOffline( + Appenderators.createBatch( idString, dataSchema, makeAppenderatorConfig( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index 2bf21397ffb..5fbfd3119d0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -51,6 +51,7 @@ import org.apache.druid.query.rowsandcols.semantic.ColumnSelectorFactoryMaker; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.column.NullableTypeStrategy; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; @@ -59,7 +60,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -84,7 +84,10 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor private Supplier rowSupplierFromFrameCursor; private ResultRow outputRow = null; private FrameWriter frameWriter = null; - private final boolean isOverEmpty; + + // List of type strategies to compare the partition columns across rows. + // Type strategies are pushed in the same order as column types in frameReader.signature() + private final NullableTypeStrategy[] typeStrategies; public WindowOperatorQueryFrameProcessor( WindowOperatorQuery query, @@ -95,7 +98,6 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor ObjectMapper jsonMapper, final List operatorFactoryList, final RowSignature rowSignature, - final boolean isOverEmpty, final int maxRowsMaterializedInWindow, final List partitionColumnNames ) @@ -105,14 +107,18 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor this.frameWriterFactory = frameWriterFactory; this.operatorFactoryList = operatorFactoryList; this.jsonMapper = jsonMapper; - this.frameReader = frameReader; this.query = query; this.frameRowsAndCols = new ArrayList<>(); this.resultRowAndCols = new ArrayList<>(); this.objectsOfASingleRac = new ArrayList<>(); - this.isOverEmpty = isOverEmpty; this.maxRowsMaterialized = maxRowsMaterializedInWindow; this.partitionColumnNames = partitionColumnNames; + + this.frameReader = frameReader; + this.typeStrategies = new NullableTypeStrategy[frameReader.signature().size()]; + for (int i = 0; i < frameReader.signature().size(); i++) { + typeStrategies[i] = frameReader.signature().getColumnType(i).get().getNullableStrategy(); + } } @Override @@ -162,7 +168,7 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor * * * The flow would look like: - * 1. Validate if the operator has an empty OVER clause + * 1. Validate if the operator doesn't have any OVER() clause with PARTITION BY for this stage. * 2. If 1 is true make a giant rows and columns (R&C) using concat as shown above * Let all operators run amok on that R&C * 3. If 1 is false @@ -187,14 +193,12 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor * We might think to reimplement them in the MSQ way so that we do not have to materialize so much data */ - // Phase 1 of the execution - // eagerly validate presence of empty OVER() clause - if (isOverEmpty) { - // if OVER() found - // have to bring all data to a single executor for processing - // convert each frame to rac - // concat all the racs to make a giant rac - // let all operators run on the giant rac when channel is finished + if (partitionColumnNames.isEmpty()) { + // If we do not have any OVER() clause with PARTITION BY for this stage. + // Bring all data to a single executor for processing. + // Convert each frame to RAC. + // Concatenate all the racs to make a giant RAC. + // Let all operators run on the giant RAC until channel is finished. if (inputChannel.canRead()) { final Frame frame = inputChannel.read(); convertRowFrameToRowsAndColumns(frame); @@ -484,7 +488,7 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor /** * Compare two rows based on the columns in partitionColumnNames. - * If the partitionColumnNames is empty or null, compare entire row. + * If the partitionColumnNames is empty, the method will end up returning true. *

* For example, say: *

    @@ -501,17 +505,13 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor */ private boolean comparePartitionKeys(ResultRow row1, ResultRow row2, List partitionColumnNames) { - if (partitionColumnNames == null || partitionColumnNames.isEmpty()) { - return row1.equals(row2); - } else { - int match = 0; - for (String columnName : partitionColumnNames) { - int i = frameReader.signature().indexOf(columnName); - if (Objects.equals(row1.get(i), row2.get(i))) { - match++; - } + int match = 0; + for (String columnName : partitionColumnNames) { + int i = frameReader.signature().indexOf(columnName); + if (typeStrategies[i].compare(row1.get(i), row2.get(i)) == 0) { + match++; } - return match == partitionColumnNames.size(); } + return match == partitionColumnNames.size(); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java index d9c14390736..9852f4f4098 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java @@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2ObjectSortedMap; +import org.apache.druid.error.DruidException; import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.OutputChannel; import org.apache.druid.frame.processor.OutputChannelFactory; @@ -59,7 +60,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor private final WindowOperatorQuery query; private final List operatorList; private final RowSignature stageRowSignature; - private final boolean isEmptyOver; private final int maxRowsMaterializedInWindow; private final List partitionColumnNames; @@ -68,7 +68,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor @JsonProperty("query") WindowOperatorQuery query, @JsonProperty("operatorList") List operatorFactoryList, @JsonProperty("stageRowSignature") RowSignature stageRowSignature, - @JsonProperty("emptyOver") boolean emptyOver, @JsonProperty("maxRowsMaterializedInWindow") int maxRowsMaterializedInWindow, @JsonProperty("partitionColumnNames") List partitionColumnNames ) @@ -76,8 +75,11 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor this.query = Preconditions.checkNotNull(query, "query"); this.operatorList = Preconditions.checkNotNull(operatorFactoryList, "bad operator"); this.stageRowSignature = Preconditions.checkNotNull(stageRowSignature, "stageSignature"); - this.isEmptyOver = emptyOver; this.maxRowsMaterializedInWindow = maxRowsMaterializedInWindow; + + if (partitionColumnNames == null) { + throw DruidException.defensive("List of partition column names encountered as null."); + } this.partitionColumnNames = partitionColumnNames; } @@ -105,12 +107,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor return stageRowSignature; } - @JsonProperty("emptyOver") - public boolean isEmptyOverFound() - { - return isEmptyOver; - } - @JsonProperty("maxRowsMaterializedInWindow") public int getMaxRowsMaterializedInWindow() { @@ -166,7 +162,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor frameContext.jsonMapper(), operatorList, stageRowSignature, - isEmptyOver, maxRowsMaterializedInWindow, partitionColumnNames ); @@ -190,8 +185,7 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor return false; } WindowOperatorQueryFrameProcessorFactory that = (WindowOperatorQueryFrameProcessorFactory) o; - return isEmptyOver == that.isEmptyOver - && maxRowsMaterializedInWindow == that.maxRowsMaterializedInWindow + return maxRowsMaterializedInWindow == that.maxRowsMaterializedInWindow && Objects.equals(query, that.query) && Objects.equals(operatorList, that.operatorList) && Objects.equals(partitionColumnNames, that.partitionColumnNames) @@ -201,6 +195,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor @Override public int hashCode() { - return Objects.hash(query, operatorList, partitionColumnNames, stageRowSignature, isEmptyOver, maxRowsMaterializedInWindow); + return Objects.hash(query, operatorList, partitionColumnNames, stageRowSignature, maxRowsMaterializedInWindow); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java index 3754f081a27..23e13f176d7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java @@ -46,6 +46,7 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -124,8 +125,11 @@ public class WindowOperatorQueryKit implements QueryKit } if (isEmptyOverPresent) { - // empty over clause found - // moving everything to a single partition + // Move everything to a single partition since we have to load all the data on a single worker anyway to compute empty over() clause. + log.info( + "Empty over clause is present in the query. Creating a single stage with all operator factories [%s].", + queryToRun.getOperators() + ); queryDefBuilder.add( StageDefinition.builder(firstStageNumber) .inputs(new StageInputSpec(firstStageNumber - 1)) @@ -136,9 +140,8 @@ public class WindowOperatorQueryKit implements QueryKit queryToRun, queryToRun.getOperators(), rowSignature, - true, maxRowsMaterialized, - new ArrayList<>() + Collections.emptyList() )) ); } else { @@ -237,7 +240,6 @@ public class WindowOperatorQueryKit implements QueryKit queryToRun, operatorList.get(i), stageRowSignature, - false, maxRowsMaterialized, partitionColumnNames )) @@ -257,20 +259,34 @@ public class WindowOperatorQueryKit implements QueryKit { List> operatorList = new ArrayList<>(); final List operators = originalQuery.getOperators(); - List operatorFactoryList = new ArrayList<>(); - for (OperatorFactory of : operators) { - operatorFactoryList.add(of); + List currentStage = new ArrayList<>(); + + for (int i = 0; i < operators.size(); i++) { + OperatorFactory of = operators.get(i); + currentStage.add(of); + if (of instanceof WindowOperatorFactory) { - operatorList.add(operatorFactoryList); - operatorFactoryList = new ArrayList<>(); - } else if (of instanceof NaivePartitioningOperatorFactory) { - if (((NaivePartitioningOperatorFactory) of).getPartitionColumns().isEmpty()) { - operatorList.clear(); - operatorList.add(originalQuery.getOperators()); - return operatorList; + // Process consecutive window operators + while (i + 1 < operators.size() && operators.get(i + 1) instanceof WindowOperatorFactory) { + i++; + currentStage.add(operators.get(i)); } + + // Finalize the current stage + operatorList.add(new ArrayList<>(currentStage)); + currentStage.clear(); } } + + // There shouldn't be any operators left in currentStage. The last operator should always be WindowOperatorFactory. + if (!currentStage.isEmpty()) { + throw new ISE( + "Found unexpected operators [%s] present in the list of operators [%s].", + currentStage, + operators + ); + } + return operatorList; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index c570fdc29b9..2104d1d40f4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -149,12 +149,6 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor @Override public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException { - final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter"); - - if (legacy) { - throw new ISE("Cannot use this engine in legacy mode"); - } - if (runningCountForLimit != null && runningCountForLimit.get() > query.getScanRowsOffset() + query.getScanRowsLimit()) { return ReturnOrAwait.returnObject(Unit.instance()); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/ArrayIngestMode.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/ArrayIngestMode.java index ff6b4718ad8..e065de077d3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/ArrayIngestMode.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/ArrayIngestMode.java @@ -25,11 +25,6 @@ package org.apache.druid.msq.util; */ public enum ArrayIngestMode { - /** - * Disables the ingestion of arrays via MSQ's INSERT queries. - */ - NONE, - /** * String arrays are ingested as MVDs. This is to preserve the legacy behaviour of Druid and will be removed in the * future, since MVDs are not true array types and the behaviour is incorrect. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java index 07a51821616..e08c0e5ded6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java @@ -131,19 +131,9 @@ public class DimensionSchemaUtils } else if (queryType.getType() == ValueType.ARRAY) { ValueType elementType = queryType.getElementType().getType(); if (elementType == ValueType.STRING) { - if (arrayIngestMode == ArrayIngestMode.NONE) { - throw InvalidInput.exception( - "String arrays can not be ingested when '%s' is set to '%s'. Set '%s' in query context " - + "to 'array' to ingest the string array as an array, or ingest it as an MVD by explicitly casting the " - + "array to an MVD with the ARRAY_TO_MV function.", - MultiStageQueryContext.CTX_ARRAY_INGEST_MODE, - StringUtils.toLowerCase(arrayIngestMode.name()), - MultiStageQueryContext.CTX_ARRAY_INGEST_MODE - ); - } else if (arrayIngestMode == ArrayIngestMode.MVD) { + if (arrayIngestMode == ArrayIngestMode.MVD) { return ColumnType.STRING; } else { - assert arrayIngestMode == ArrayIngestMode.ARRAY; return queryType; } } else if (elementType.isNumeric()) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index cbf9a1a905f..c3f5a065da7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -165,7 +165,7 @@ public class MultiStageQueryContext public static final boolean DEFAULT_USE_AUTO_SCHEMAS = false; public static final String CTX_ARRAY_INGEST_MODE = "arrayIngestMode"; - public static final ArrayIngestMode DEFAULT_ARRAY_INGEST_MODE = ArrayIngestMode.MVD; + public static final ArrayIngestMode DEFAULT_ARRAY_INGEST_MODE = ArrayIngestMode.ARRAY; public static final String NEXT_WINDOW_SHUFFLE_COL = "__windowShuffleCol"; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java index 06408af2a1b..f04a65f89a8 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/DataServerQueryHandlerTest.java @@ -132,7 +132,7 @@ public class DataServerQueryHandlerTest .build(); QueryToolChestWarehouse queryToolChestWarehouse = new MapQueryToolChestWarehouse( ImmutableMap., QueryToolChest>builder() - .put(ScanQuery.class, new ScanQueryQueryToolChest(null, null)) + .put(ScanQuery.class, new ScanQueryQueryToolChest(null)) .build() ); target = spy( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java index d3f7e6a1473..3cf4bf12ed1 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java @@ -122,30 +122,7 @@ public class MSQArraysTest extends MSQTestBase } /** - * Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest - * string arrays - */ - @MethodSource("data") - @ParameterizedTest(name = "{index}:with context {0}") - public void testInsertStringArrayWithArrayIngestModeNone(String contextName, Map context) - { - - final Map adjustedContext = new HashMap<>(context); - adjustedContext.put(MultiStageQueryContext.CTX_ARRAY_INGEST_MODE, "none"); - - testIngestQuery().setSql( - "INSERT INTO foo1 SELECT MV_TO_ARRAY(dim3) AS dim3 FROM foo GROUP BY 1 PARTITIONED BY ALL TIME") - .setQueryContext(adjustedContext) - .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(ISE.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "String arrays can not be ingested when 'arrayIngestMode' is set to 'none'")) - )) - .verifyExecutionError(); - } - - /** - * Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest + * Tests the behaviour of INSERT query when arrayIngestMode is set to default and the user tries to ingest * string arrays */ @MethodSource("data") @@ -172,7 +149,7 @@ public class MSQArraysTest extends MSQTestBase } /** - * Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest + * Tests the behaviour of INSERT query when arrayIngestMode is set to default and the user tries to ingest * string arrays */ @MethodSource("data") @@ -200,7 +177,7 @@ public class MSQArraysTest extends MSQTestBase } /** - * Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest + * Tests the behaviour of INSERT query when arrayIngestMode is set to default and the user tries to ingest * string arrays */ @MethodSource("data") @@ -228,7 +205,7 @@ public class MSQArraysTest extends MSQTestBase } /** - * Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest + * Tests the behaviour of INSERT query when arrayIngestMode is set to default and the user tries to ingest * string arrays */ @MethodSource("data") @@ -277,7 +254,7 @@ public class MSQArraysTest extends MSQTestBase } /** - * Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest + * Tests the behaviour of INSERT query when arrayIngestMode is set to default and the user tries to ingest * string arrays */ @MethodSource("data") @@ -316,8 +293,7 @@ public class MSQArraysTest extends MSQTestBase } /** - * Tests the behaviour of INSERT query when arrayIngestMode is set to mvd (default) and the only array type to be - * ingested is string array + * Tests the behaviour of INSERT query when arrayIngestMode is set to array (default) */ @MethodSource("data") @ParameterizedTest(name = "{index}:with context {0}") @@ -325,16 +301,32 @@ public class MSQArraysTest extends MSQTestBase { RowSignature rowSignature = RowSignature.builder() .add("__time", ColumnType.LONG) - .add("dim3", ColumnType.STRING) + .add("dim3", ColumnType.STRING_ARRAY) .build(); + List expectedRows = new ArrayList<>( + ImmutableList.of( + new Object[]{0L, null}, + new Object[]{0L, new Object[]{"a", "b"}} + ) + ); + if (!useDefault) { + expectedRows.add(new Object[]{0L, new Object[]{""}}); + } + expectedRows.addAll( + ImmutableList.of( + new Object[]{0L, new Object[]{"b", "c"}}, + new Object[]{0L, new Object[]{"d"}} + ) + ); + testIngestQuery().setSql( "INSERT INTO foo1 SELECT MV_TO_ARRAY(dim3) AS dim3 FROM foo GROUP BY 1 PARTITIONED BY ALL TIME") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) - .setExpectedResultRows(expectedMultiValueFooRowsToArray()) + .setExpectedResultRows(expectedRows) .verifyResults(); } @@ -603,13 +595,6 @@ public class MSQArraysTest extends MSQTestBase .verifyResults(); } - @MethodSource("data") - @ParameterizedTest(name = "{index}:with context {0}") - public void testSelectOnArraysWithArrayIngestModeAsNone(String contextName, Map context) - { - testSelectOnArrays(contextName, context, "none"); - } - @MethodSource("data") @ParameterizedTest(name = "{index}:with context {0}") public void testSelectOnArraysWithArrayIngestModeAsMVD(String contextName, Map context) @@ -1128,20 +1113,4 @@ public class MSQArraysTest extends MSQTestBase .setExpectedResultRows(expectedRows) .verifyResults(); } - - private List expectedMultiValueFooRowsToArray() - { - List expectedRows = new ArrayList<>(); - expectedRows.add(new Object[]{0L, null}); - if (!useDefault) { - expectedRows.add(new Object[]{0L, ""}); - } - - expectedRows.addAll(ImmutableList.of( - new Object[]{0L, ImmutableList.of("a", "b")}, - new Object[]{0L, ImmutableList.of("b", "c")}, - new Object[]{0L, "d"} - )); - return expectedRows; - } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index f6eb80b3282..1e8dc474f6e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -49,6 +49,7 @@ import org.apache.druid.msq.test.CounterSnapshotMatcher; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestTaskActionClient; import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.QueryContext; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -957,6 +958,13 @@ public class MSQReplaceTest extends MSQTestBase "Using RangeShardSpec to generate segments." ) ) + .setExpectedLastCompactionState(expectedCompactionState( + queryContext, + Collections.singletonList("dim1"), + Collections.singletonList(new StringDimensionSchema("dim1")), + GranularityType.ALL, + Intervals.ETERNITY + )) .verifyResults(); } @@ -1150,7 +1158,6 @@ public class MSQReplaceTest extends MSQTestBase + "CLUSTERED BY dim1" ) .setExpectedDataSource("foo1") - .setQueryContext(DEFAULT_MSQ_CONTEXT) .setExpectedShardSpec(DimensionRangeShardSpec.class) .setExpectedRowSignature(rowSignature) .setQueryContext(context) @@ -1165,6 +1172,19 @@ public class MSQReplaceTest extends MSQTestBase new Object[]{0L, "def", 5.0f, 1L} ) ) + .setExpectedLastCompactionState( + expectedCompactionState( + context, + Collections.singletonList("dim1"), + Arrays.asList( + new StringDimensionSchema("dim1"), + new FloatDimensionSchema("m1"), + new LongDimensionSchema("cnt") + ), + GranularityType.ALL, + Intervals.ETERNITY + ) + ) .verifyResults(); } @@ -1571,6 +1591,7 @@ public class MSQReplaceTest extends MSQTestBase + "GROUP BY 1, 2 " + "PARTITIONED by TIME_FLOOR(__time, 'P3M') " + "CLUSTERED by dim1") + .setQueryContext(context) .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setExpectedShardSpec(DimensionRangeShardSpec.class) @@ -1582,6 +1603,16 @@ public class MSQReplaceTest extends MSQTestBase ) ) .setExpectedResultRows(expectedResults) + .setExpectedLastCompactionState(expectedCompactionState( + context, + Collections.singletonList("dim1"), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("cnt") + ), + GranularityType.QUARTER, + Intervals.of("2000-01-01T00:00:00.000Z/2002-01-01T00:00:00.000Z") + )) .verifyResults(); } @@ -1652,6 +1683,16 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedResultRows(ImmutableList.of()) .setExpectedTombstoneIntervals(ImmutableSet.of(existingSegmentInterval)) + .setExpectedLastCompactionState(expectedCompactionState( + context, + Collections.emptyList(), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("cnt") + ), + GranularityType.DAY, + Intervals.ETERNITY + )) .verifyResults(); } @@ -1688,6 +1729,16 @@ public class MSQReplaceTest extends MSQTestBase Intervals.of("2016-06-27T01:00:00/2016-06-27T02:00:00") ) ) + .setExpectedLastCompactionState(expectedCompactionState( + context, + Collections.emptyList(), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("cnt") + ), + GranularityType.HOUR, + Intervals.of("2016-06-27T01:00:00/2016-06-27T02:00:00") + )) .verifyResults(); } @@ -1726,6 +1777,16 @@ public class MSQReplaceTest extends MSQTestBase Intervals.of("2016-06-30T/2016-07-01T") ) ) + .setExpectedLastCompactionState(expectedCompactionState( + context, + Collections.emptyList(), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("cnt") + ), + GranularityType.DAY, + Intervals.of("2016-06-29T00:00:00.000Z/2016-07-03T00:00:00.000Z") + )) .verifyResults(); } @@ -1764,6 +1825,16 @@ public class MSQReplaceTest extends MSQTestBase Intervals.of("2016-06-02T/2016-06-03T") ) ) + .setExpectedLastCompactionState(expectedCompactionState( + context, + Collections.emptyList(), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("cnt") + ), + GranularityType.DAY, + Intervals.of("2016-05-25T00:00:00.000Z/2016-06-03T00:00:00.000Z") + )) .verifyResults(); } @@ -1802,6 +1873,16 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedResultRows(ImmutableList.of()) .setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.ETERNITY)) + .setExpectedLastCompactionState(expectedCompactionState( + context, + Collections.emptyList(), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("cnt") + ), + GranularityType.DAY, + Intervals.ETERNITY + )) .verifyResults(); } @@ -1838,6 +1919,16 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedResultRows(ImmutableList.of()) .setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.of("2016-06-01T/2016-09-01T"))) + .setExpectedLastCompactionState(expectedCompactionState( + context, + Collections.emptyList(), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("cnt") + ), + GranularityType.ALL, + Intervals.ETERNITY + )) .verifyResults(); } @@ -1874,6 +1965,16 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedResultRows(ImmutableList.of()) .setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.ETERNITY)) + .setExpectedLastCompactionState(expectedCompactionState( + context, + Collections.emptyList(), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("cnt") + ), + GranularityType.ALL, + Intervals.ETERNITY + )) .verifyResults(); } @@ -1906,6 +2007,16 @@ public class MSQReplaceTest extends MSQTestBase .setExpectedDataSource("foo1") .setExpectedResultRows(ImmutableList.of()) .setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.ETERNITY)) + .setExpectedLastCompactionState(expectedCompactionState( + context, + Collections.emptyList(), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("cnt") + ), + GranularityType.ALL, + Intervals.ETERNITY + )) .verifyResults(); } @@ -1966,6 +2077,16 @@ public class MSQReplaceTest extends MSQTestBase Intervals.of("2016-06-02T/2016-06-03T") ) ) + .setExpectedLastCompactionState(expectedCompactionState( + context, + Collections.emptyList(), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("cnt") + ), + GranularityType.DAY, + Intervals.of("2016-06-01T00:00:00.000Z/2016-06-03T00:00:00.000Z") + )) .verifyResults(); } @@ -2049,11 +2170,16 @@ public class MSQReplaceTest extends MSQTestBase } PartitionsSpec partitionsSpec; if (partitionDimensions.isEmpty()) { - partitionsSpec = new DynamicPartitionsSpec(MultiStageQueryContext.DEFAULT_ROWS_PER_SEGMENT, Long.MAX_VALUE); - + partitionsSpec = new DynamicPartitionsSpec( + MultiStageQueryContext.getRowsPerSegment(QueryContext.of(context)), + Long.MAX_VALUE + ); } else { - partitionsSpec = new DimensionRangePartitionsSpec(MultiStageQueryContext.DEFAULT_ROWS_PER_SEGMENT, null, - partitionDimensions, false + partitionsSpec = new DimensionRangePartitionsSpec( + MultiStageQueryContext.getRowsPerSegment(QueryContext.of(context)), + null, + partitionDimensions, + false ); } DimensionsSpec dimensionsSpec = new DimensionsSpec.Builder() diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index 2d14e743497..c6d67f14428 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -2594,7 +2594,6 @@ public class MSQSelectTest extends MSQTestBase )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(defaultScanQueryContext( context, resultSignature @@ -2668,7 +2667,6 @@ public class MSQSelectTest extends MSQTestBase ) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .filters(equality("dim2", "a", ColumnType.STRING)) .columns("dim3") .context(defaultScanQueryContext( @@ -2683,7 +2681,6 @@ public class MSQSelectTest extends MSQTestBase )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(defaultScanQueryContext( context, resultSignature @@ -2751,7 +2748,6 @@ public class MSQSelectTest extends MSQTestBase )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(defaultScanQueryContext( context, rowSignature diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index b3f6ac20d53..5cc84ac6ee6 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java @@ -106,7 +106,7 @@ public class MSQWindowTest extends MSQTestBase .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "d0") }; @@ -196,7 +196,7 @@ public class MSQWindowTest extends MSQTestBase .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "d1") }; @@ -306,7 +306,7 @@ public class MSQWindowTest extends MSQTestBase .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "d1") }; @@ -419,7 +419,7 @@ public class MSQWindowTest extends MSQTestBase .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "d0") }; @@ -523,7 +523,7 @@ public class MSQWindowTest extends MSQTestBase .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "d0") }; @@ -589,7 +589,7 @@ public class MSQWindowTest extends MSQTestBase .add("cc", ColumnType.DOUBLE) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "m1") }; @@ -609,7 +609,6 @@ public class MSQWindowTest extends MSQTestBase .columns("m1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -655,7 +654,7 @@ public class MSQWindowTest extends MSQTestBase .add("cc", ColumnType.DOUBLE) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "m1") }; @@ -678,7 +677,6 @@ public class MSQWindowTest extends MSQTestBase .columns("m1", "m2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -727,7 +725,7 @@ public class MSQWindowTest extends MSQTestBase .add("cc", ColumnType.DOUBLE) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "m1") }; @@ -750,7 +748,6 @@ public class MSQWindowTest extends MSQTestBase .columns("m1", "m2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -796,7 +793,7 @@ public class MSQWindowTest extends MSQTestBase .add("cc", ColumnType.DOUBLE) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "d1") }; @@ -881,7 +878,7 @@ public class MSQWindowTest extends MSQTestBase .add("cc", ColumnType.DOUBLE) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "m1") }; @@ -896,7 +893,6 @@ public class MSQWindowTest extends MSQTestBase .virtualColumns(expressionVirtualColumn("v0", "strlen(\"dim1\")", ColumnType.LONG)) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -955,7 +951,7 @@ public class MSQWindowTest extends MSQTestBase .add("cc", ColumnType.DOUBLE) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "m1") }; @@ -969,7 +965,6 @@ public class MSQWindowTest extends MSQTestBase .columns("m1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -1033,17 +1028,7 @@ public class MSQWindowTest extends MSQTestBase .add("m2", ColumnType.DOUBLE) .build(); - final WindowFrame theFrame = new WindowFrame( - WindowFrame.PeerType.RANGE, - true, - 0, - false, - 0, - ImmutableList.of(new ColumnWithDirection( - "m1", - ColumnWithDirection.Direction.ASC - )) - ); + final WindowFrame theFrame = WindowFrame.forOrderBy("m1"); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "m1") }; @@ -1063,7 +1048,6 @@ public class MSQWindowTest extends MSQTestBase .columns("m2", "v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature1) - .legacy(false) .build() ), "j0.", @@ -1078,7 +1062,6 @@ public class MSQWindowTest extends MSQTestBase .columns("j0.m2", "m1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -1149,14 +1132,7 @@ public class MSQWindowTest extends MSQTestBase .add("m2", ColumnType.DOUBLE) .build(); - final WindowFrame theFrame = new WindowFrame( - WindowFrame.PeerType.ROWS, - true, - 0, - true, - 0, - null - ); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "m1") }; @@ -1176,7 +1152,6 @@ public class MSQWindowTest extends MSQTestBase .columns("m2", "v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature1) - .legacy(false) .build() ), "j0.", @@ -1191,7 +1166,6 @@ public class MSQWindowTest extends MSQTestBase .columns("j0.m2", "m1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -1242,7 +1216,7 @@ public class MSQWindowTest extends MSQTestBase .add("cc", ColumnType.DOUBLE) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "m1") }; @@ -1265,7 +1239,6 @@ public class MSQWindowTest extends MSQTestBase .columns("dim2", "m1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -1332,14 +1305,7 @@ public class MSQWindowTest extends MSQTestBase .add("d3", ColumnType.STRING) .build(); - final WindowFrame theFrame = new WindowFrame( - WindowFrame.PeerType.ROWS, - true, - 0, - true, - 0, - null - ); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "m1") }; @@ -1359,7 +1325,6 @@ public class MSQWindowTest extends MSQTestBase .columns("j0.unnest", "m1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -1423,14 +1388,7 @@ public class MSQWindowTest extends MSQTestBase .add("d3", ColumnType.STRING) .build(); - final WindowFrame theFrame = new WindowFrame( - WindowFrame.PeerType.ROWS, - true, - 0, - true, - 0, - null - ); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "m1") }; @@ -1450,7 +1408,6 @@ public class MSQWindowTest extends MSQTestBase .columns("j0.unnest", "m1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -1808,7 +1765,7 @@ public class MSQWindowTest extends MSQTestBase .add("cc", ColumnType.LONG) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new LongSumAggregatorFactory("w0", "added") }; @@ -1832,7 +1789,6 @@ public class MSQWindowTest extends MSQTestBase .columns("added", "cityName") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(contextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -1900,7 +1856,7 @@ public class MSQWindowTest extends MSQTestBase .add("cc", ColumnType.LONG) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new LongSumAggregatorFactory("w0", "added") }; @@ -1924,7 +1880,6 @@ public class MSQWindowTest extends MSQTestBase .columns("added", "cityName", "countryIsoCode") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(innerContextWithRowSignature) - .legacy(false) .build()), new LegacySegmentSpec(Intervals.ETERNITY), context, @@ -1954,7 +1909,6 @@ public class MSQWindowTest extends MSQTestBase .limit(5) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(outerContextWithRowSignature) - .legacy(false) .build(); testSelectQuery() @@ -2016,7 +1970,7 @@ public class MSQWindowTest extends MSQTestBase .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new LongSumAggregatorFactory("w0", "d1") }; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index b95243f7783..6c5d1957265 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -42,6 +42,7 @@ import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.math.expr.ExprMacroTable; @@ -54,10 +55,12 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.TransformSpec; @@ -73,6 +76,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public class MSQCompactionRunnerTest { @@ -127,7 +131,7 @@ public class MSQCompactionRunnerTest null, null ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -140,7 +144,7 @@ public class MSQCompactionRunnerTest null, null ); - Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -153,7 +157,7 @@ public class MSQCompactionRunnerTest null, null ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -166,7 +170,7 @@ public class MSQCompactionRunnerTest null, null ); - Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -179,7 +183,7 @@ public class MSQCompactionRunnerTest new ClientCompactionTaskGranularitySpec(null, Granularities.ALL, null), null ); - Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -192,28 +196,7 @@ public class MSQCompactionRunnerTest new ClientCompactionTaskGranularitySpec(null, null, false), AGGREGATORS.toArray(new AggregatorFactory[0]) ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); - } - - @Test - public void testMSQEngineWithUnsupportedMetricsSpecIsInValid() - { - // Aggregators having different input and ouput column names are unsupported. - final String inputColName = "added"; - final String outputColName = "sum_added"; - CompactionTask compactionTask = createCompactionTask( - new DynamicPartitionsSpec(3, null), - null, - Collections.emptyMap(), - new ClientCompactionTaskGranularitySpec(null, null, null), - new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)} - ); - CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask); - Assert.assertFalse(validationResult.isValid()); - Assert.assertEquals( - "Different name[sum_added] and fieldName(s)[[added]] for aggregator unsupported for MSQ engine.", - validationResult.getReason() - ); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); } @Test @@ -288,6 +271,10 @@ public class MSQCompactionRunnerTest ); Assert.assertNull(msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY)); Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); + Assert.assertEquals(PARTITION_DIMENSIONS.stream().map(col -> new ScanQuery.OrderBy( + col, + ScanQuery.Order.ASCENDING + )).collect(Collectors.toList()), ((ScanQuery) actualMSQSpec.getQuery()).getOrderBys()); } @Test @@ -358,6 +345,48 @@ public class MSQCompactionRunnerTest Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); } + @Test + public void testIntervalsWithRolledUpSegmentsAndNonIdempotentAggregatorFails() + { + final String inputColName = "added"; + final String outputColName = "sum_added"; + CompactionTask compactionTask = createCompactionTask( + null, + null, + Collections.emptyMap(), + null, + new AggregatorFactory[]{ + new LongSumAggregatorFactory( + outputColName, + inputColName + ) + } + ); + CombinedDataSchema dataSchema = new CombinedDataSchema( + DATA_SOURCE, + new TimestampSpec(TIMESTAMP_COLUMN, null, null), + new DimensionsSpec(DIMENSIONS), + new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)}, + new UniformGranularitySpec( + SEGMENT_GRANULARITY.getDefaultGranularity(), + null, + false, + Collections.singletonList(COMPACTION_INTERVAL) + ), + null, + true + ); + CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask( + compactionTask, + Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + ); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals(validationResult.getReason(), StringUtils.format( + "MSQ: Rolled-up segments in compaction interval[%s].", + COMPACTION_INTERVAL + )); + } + private CompactionTask createCompactionTask( @Nullable PartitionsSpec partitionsSpec, @Nullable DimFilter dimFilter, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java index 9de14610f19..9df6c38f30e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java @@ -62,7 +62,6 @@ public class MSQControllerTaskTest )) .query(new Druids.ScanQueryBuilder() .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .intervals(new MultipleIntervalSegmentSpec(INTERVALS)) .dataSource("target") .build() diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java index 2049c0194ed..802a4c52f17 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java @@ -28,7 +28,7 @@ public class WindowOperatorQueryFrameProcessorFactoryTest public void testEqualsAndHashcode() { EqualsVerifier.forClass(WindowOperatorQueryFrameProcessorFactory.class) - .withNonnullFields("query", "operatorList", "stageRowSignature", "isEmptyOver", "maxRowsMaterializedInWindow", "partitionColumnNames") + .withNonnullFields("query", "operatorList", "stageRowSignature", "maxRowsMaterializedInWindow", "partitionColumnNames") .usingGetClass() .verify(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java index a6844fcabff..e3272cbcee7 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java @@ -108,7 +108,6 @@ public class ScanQueryFrameProcessorTest extends InitializedNullHandlingTest .dataSource("test") .intervals(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)) .columns(adapter.getRowSignature().getColumnNames()) - .legacy(false) .build(); final StagePartition stagePartition = new StagePartition(new StageId("query", 0), 0); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java index 6d81da64aab..1b0483d0b5a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java @@ -301,16 +301,7 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase ), SqlStatementResourceTest.makeOkRequest()); Assert.assertEquals( - "{PLAN=[{\"query\":" - + "{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," - + "\"legacy\":false," - + "\"context\":{\"__resultFormat\":\"object\",\"executionMode\":\"ASYNC\",\"scanSignature\":\"[{\\\"name\\\":\\\"__time\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"cnt\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"dim1\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim2\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim3\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"m1\\\",\\\"type\\\":\\\"FLOAT\\\"},{\\\"name\\\":\\\"m2\\\",\\\"type\\\":\\\"DOUBLE\\\"},{\\\"name\\\":\\\"unique_dim1\\\",\\\"type\\\":\\\"COMPLEX\\\"}]\",\"sqlQueryId\":\"queryId\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]," - + " RESOURCES=[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]," - + " ATTRIBUTES={\"statementType\":\"SELECT\"}}", + "{PLAN=[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"__resultFormat\":\"object\",\"executionMode\":\"ASYNC\",\"scanSignature\":\"[{\\\"name\\\":\\\"__time\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"cnt\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"dim1\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim2\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim3\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"m1\\\",\\\"type\\\":\\\"FLOAT\\\"},{\\\"name\\\":\\\"m2\\\",\\\"type\\\":\\\"DOUBLE\\\"},{\\\"name\\\":\\\"unique_dim1\\\",\\\"type\\\":\\\"COMPLEX\\\"}]\",\"sqlQueryId\":\"queryId\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}], RESOURCES=[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}], ATTRIBUTES={\"statementType\":\"SELECT\"}}", String.valueOf(SqlStatementResourceTest.getResultRowsFromResponse(response).get(0)) ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java index 2da5fd42caf..a97ee01297f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlStatementResourceTest.java @@ -123,7 +123,6 @@ public class SqlStatementResourceTest extends MSQTestBase private static final Query QUERY = new Druids.ScanQueryBuilder().resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .intervals(new MultipleIntervalSegmentSpec( Collections.singletonList(Intervals.of( "2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z")))) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java index ad05c20b582..14f6f73b24a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java @@ -44,7 +44,7 @@ import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.NoopRowIngestionMeters; import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java index af08acf6ab7..bdd5270321a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java @@ -221,17 +221,12 @@ public class MultiStageQueryContextTest @Test public void arrayIngestMode_unset_returnsDefaultValue() { - Assert.assertEquals(ArrayIngestMode.MVD, MultiStageQueryContext.getArrayIngestMode(QueryContext.empty())); + Assert.assertEquals(ArrayIngestMode.ARRAY, MultiStageQueryContext.getArrayIngestMode(QueryContext.empty())); } @Test public void arrayIngestMode_set_returnsCorrectValue() { - Assert.assertEquals( - ArrayIngestMode.NONE, - MultiStageQueryContext.getArrayIngestMode(QueryContext.of(ImmutableMap.of(CTX_ARRAY_INGEST_MODE, "none"))) - ); - Assert.assertEquals( ArrayIngestMode.MVD, MultiStageQueryContext.getArrayIngestMode(QueryContext.of(ImmutableMap.of(CTX_ARRAY_INGEST_MODE, "mvd"))) diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/firehose/sql/MySQLFirehoseDatabaseConnector.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/input/MySQLInputSourceDatabaseConnector.java similarity index 91% rename from extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/firehose/sql/MySQLFirehoseDatabaseConnector.java rename to extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/input/MySQLInputSourceDatabaseConnector.java index 07f48f36cd3..4647434aba1 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/firehose/sql/MySQLFirehoseDatabaseConnector.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/input/MySQLInputSourceDatabaseConnector.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.firehose.sql; +package org.apache.druid.metadata.input; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; @@ -25,7 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.commons.dbcp2.BasicDataSource; import org.apache.druid.metadata.MetadataStorageConnectorConfig; -import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.SQLInputSourceDatabaseConnector; import org.apache.druid.metadata.storage.mysql.MySQLConnectorDriverConfig; import org.apache.druid.server.initialization.JdbcAccessSecurityConfig; import org.apache.druid.utils.ConnectionUriUtils; @@ -37,7 +37,7 @@ import java.util.Set; @JsonTypeName("mysql") -public class MySQLFirehoseDatabaseConnector extends SQLFirehoseDatabaseConnector +public class MySQLInputSourceDatabaseConnector extends SQLInputSourceDatabaseConnector { private final DBI dbi; private final MetadataStorageConnectorConfig connectorConfig; @@ -45,7 +45,7 @@ public class MySQLFirehoseDatabaseConnector extends SQLFirehoseDatabaseConnector private final String driverClassName; @JsonCreator - public MySQLFirehoseDatabaseConnector( + public MySQLInputSourceDatabaseConnector( @JsonProperty("connectorConfig") MetadataStorageConnectorConfig connectorConfig, @JsonProperty("driverClassName") @Nullable String driverClassName, @JacksonInject JdbcAccessSecurityConfig securityConfig, @@ -98,7 +98,7 @@ public class MySQLFirehoseDatabaseConnector extends SQLFirehoseDatabaseConnector if (o == null || getClass() != o.getClass()) { return false; } - MySQLFirehoseDatabaseConnector that = (MySQLFirehoseDatabaseConnector) o; + MySQLInputSourceDatabaseConnector that = (MySQLInputSourceDatabaseConnector) o; return connectorConfig.equals(that.connectorConfig) && Objects.equals( driverClassName, that.driverClassName diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java index 2cb8cc7dc79..3f62e1d7eb3 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLMetadataStorageModule.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; import com.google.inject.Key; -import org.apache.druid.firehose.sql.MySQLFirehoseDatabaseConnector; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.PolyBind; @@ -36,6 +35,7 @@ import org.apache.druid.metadata.MetadataStorageProvider; import org.apache.druid.metadata.MySQLMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.NoopMetadataStorageProvider; import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.metadata.input.MySQLInputSourceDatabaseConnector; import java.util.Collections; import java.util.List; @@ -55,7 +55,7 @@ public class MySQLMetadataStorageModule extends SQLMetadataStorageDruidModule im return Collections.singletonList( new SimpleModule() .registerSubtypes( - new NamedType(MySQLFirehoseDatabaseConnector.class, "mysql") + new NamedType(MySQLInputSourceDatabaseConnector.class, "mysql") ) ); } diff --git a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/firehose/sql/MySQLFirehoseDatabaseConnectorTest.java b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/input/MySQLInputSourceDatabaseConnectorTest.java similarity index 92% rename from extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/firehose/sql/MySQLFirehoseDatabaseConnectorTest.java rename to extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/input/MySQLInputSourceDatabaseConnectorTest.java index 43d96961c1b..1ed7214aa96 100644 --- a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/firehose/sql/MySQLFirehoseDatabaseConnectorTest.java +++ b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/input/MySQLInputSourceDatabaseConnectorTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.firehose.sql; +package org.apache.druid.metadata.input; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.InjectableValues; @@ -42,7 +42,7 @@ import org.mockito.junit.MockitoJUnitRunner; import java.util.Set; @RunWith(MockitoJUnitRunner.class) -public class MySQLFirehoseDatabaseConnectorTest +public class MySQLInputSourceDatabaseConnectorTest { private static final JdbcAccessSecurityConfig INJECTED_CONF = newSecurityConfigEnforcingAllowList(ImmutableSet.of()); @@ -76,33 +76,33 @@ public class MySQLFirehoseDatabaseConnectorTest return "jdbc:mysql://localhost:3306/test"; } }; - MySQLFirehoseDatabaseConnector connector = new MySQLFirehoseDatabaseConnector( + MySQLInputSourceDatabaseConnector connector = new MySQLInputSourceDatabaseConnector( connectorConfig, null, INJECTED_CONF, mySQLConnectorDriverConfig ); - MySQLFirehoseDatabaseConnector andBack = mapper.readValue( + MySQLInputSourceDatabaseConnector andBack = mapper.readValue( mapper.writeValueAsString(connector), - MySQLFirehoseDatabaseConnector.class + MySQLInputSourceDatabaseConnector.class ); Assert.assertEquals(connector, andBack); // test again with classname - connector = new MySQLFirehoseDatabaseConnector( + connector = new MySQLInputSourceDatabaseConnector( connectorConfig, "some.class.name.Driver", INJECTED_CONF, mySQLConnectorDriverConfig ); - andBack = mapper.readValue(mapper.writeValueAsString(connector), MySQLFirehoseDatabaseConnector.class); + andBack = mapper.readValue(mapper.writeValueAsString(connector), MySQLInputSourceDatabaseConnector.class); Assert.assertEquals(connector, andBack); } @Test public void testEqualsAndHashcode() { - EqualsVerifier.forClass(MySQLFirehoseDatabaseConnector.class) + EqualsVerifier.forClass(MySQLInputSourceDatabaseConnector.class) .usingGetClass() .withNonnullFields("connectorConfig") .withIgnoredFields("dbi") @@ -123,7 +123,7 @@ public class MySQLFirehoseDatabaseConnectorTest JdbcAccessSecurityConfig securityConfig = newSecurityConfigEnforcingAllowList(ImmutableSet.of()); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -145,7 +145,7 @@ public class MySQLFirehoseDatabaseConnectorTest JdbcAccessSecurityConfig securityConfig = newSecurityConfigEnforcingAllowList(ImmutableSet.of("user")); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -170,7 +170,7 @@ public class MySQLFirehoseDatabaseConnectorTest expectedException.expectMessage("The property [password] is not in the allowed list"); expectedException.expect(IllegalArgumentException.class); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -194,7 +194,7 @@ public class MySQLFirehoseDatabaseConnectorTest ImmutableSet.of("user", "password", "keyonly", "etc") ); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -218,7 +218,7 @@ public class MySQLFirehoseDatabaseConnectorTest ImmutableSet.of("user", "password", "keyonly", "etc") ); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -244,7 +244,7 @@ public class MySQLFirehoseDatabaseConnectorTest expectedException.expectMessage("The property [password] is not in the allowed list"); expectedException.expect(IllegalArgumentException.class); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -269,7 +269,7 @@ public class MySQLFirehoseDatabaseConnectorTest expectedException.expectMessage("The property [password] is not in the allowed list"); expectedException.expect(IllegalArgumentException.class); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -294,7 +294,7 @@ public class MySQLFirehoseDatabaseConnectorTest expectedException.expectMessage("The property [password] is not in the allowed list"); expectedException.expect(IllegalArgumentException.class); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -329,7 +329,7 @@ public class MySQLFirehoseDatabaseConnectorTest } }; - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, diff --git a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/firehose/PostgresqlFirehoseDatabaseConnector.java b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/input/PostgresqlInputSourceDatabaseConnector.java similarity index 87% rename from extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/firehose/PostgresqlFirehoseDatabaseConnector.java rename to extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/input/PostgresqlInputSourceDatabaseConnector.java index d9880d7acf2..9812b213f59 100644 --- a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/firehose/PostgresqlFirehoseDatabaseConnector.java +++ b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/input/PostgresqlInputSourceDatabaseConnector.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.firehose; +package org.apache.druid.metadata.input; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; @@ -25,7 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.commons.dbcp2.BasicDataSource; import org.apache.druid.metadata.MetadataStorageConnectorConfig; -import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.SQLInputSourceDatabaseConnector; import org.apache.druid.server.initialization.JdbcAccessSecurityConfig; import org.apache.druid.utils.ConnectionUriUtils; import org.skife.jdbi.v2.DBI; @@ -35,13 +35,13 @@ import java.util.Set; @JsonTypeName("postgresql") -public class PostgresqlFirehoseDatabaseConnector extends SQLFirehoseDatabaseConnector +public class PostgresqlInputSourceDatabaseConnector extends SQLInputSourceDatabaseConnector { private final DBI dbi; private final MetadataStorageConnectorConfig connectorConfig; @JsonCreator - public PostgresqlFirehoseDatabaseConnector( + public PostgresqlInputSourceDatabaseConnector( @JsonProperty("connectorConfig") MetadataStorageConnectorConfig connectorConfig, @JacksonInject JdbcAccessSecurityConfig securityConfig ) @@ -80,7 +80,7 @@ public class PostgresqlFirehoseDatabaseConnector extends SQLFirehoseDatabaseConn if (o == null || getClass() != o.getClass()) { return false; } - PostgresqlFirehoseDatabaseConnector that = (PostgresqlFirehoseDatabaseConnector) o; + PostgresqlInputSourceDatabaseConnector that = (PostgresqlInputSourceDatabaseConnector) o; return connectorConfig.equals(that.connectorConfig); } @@ -93,7 +93,7 @@ public class PostgresqlFirehoseDatabaseConnector extends SQLFirehoseDatabaseConn @Override public String toString() { - return "PostgresqlFirehoseDatabaseConnector{" + + return "PostgresqlInputSourceDatabaseConnector{" + "connectorConfig=" + connectorConfig + '}'; } diff --git a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLMetadataStorageModule.java b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLMetadataStorageModule.java index 9506eddd04b..31777d0f431 100644 --- a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLMetadataStorageModule.java +++ b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLMetadataStorageModule.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; import com.google.inject.Key; -import org.apache.druid.firehose.PostgresqlFirehoseDatabaseConnector; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.PolyBind; @@ -36,6 +35,7 @@ import org.apache.druid.metadata.MetadataStorageProvider; import org.apache.druid.metadata.NoopMetadataStorageProvider; import org.apache.druid.metadata.PostgreSQLMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.metadata.input.PostgresqlInputSourceDatabaseConnector; import java.util.Collections; import java.util.List; @@ -56,7 +56,7 @@ public class PostgreSQLMetadataStorageModule extends SQLMetadataStorageDruidModu return Collections.singletonList( new SimpleModule() .registerSubtypes( - new NamedType(PostgresqlFirehoseDatabaseConnector.class, "postgresql") + new NamedType(PostgresqlInputSourceDatabaseConnector.class, "postgresql") ) ); } diff --git a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/firehose/PostgresqlFirehoseDatabaseConnectorTest.java b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/input/PostgresqlInputSourceDatabaseConnectorTest.java similarity index 90% rename from extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/firehose/PostgresqlFirehoseDatabaseConnectorTest.java rename to extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/input/PostgresqlInputSourceDatabaseConnectorTest.java index 9b93f0102c4..0d67d72aa84 100644 --- a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/firehose/PostgresqlFirehoseDatabaseConnectorTest.java +++ b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/input/PostgresqlInputSourceDatabaseConnectorTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.firehose; +package org.apache.druid.metadata.input; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.InjectableValues; @@ -35,7 +35,7 @@ import org.junit.rules.ExpectedException; import java.util.Set; -public class PostgresqlFirehoseDatabaseConnectorTest +public class PostgresqlInputSourceDatabaseConnectorTest { private static final ObjectMapper MAPPER = new DefaultObjectMapper(); private static final JdbcAccessSecurityConfig INJECTED_CONF = newSecurityConfigEnforcingAllowList(ImmutableSet.of()); @@ -60,13 +60,13 @@ public class PostgresqlFirehoseDatabaseConnectorTest return "jdbc:postgresql://localhost:3306/test"; } }; - PostgresqlFirehoseDatabaseConnector connector = new PostgresqlFirehoseDatabaseConnector( + PostgresqlInputSourceDatabaseConnector connector = new PostgresqlInputSourceDatabaseConnector( connectorConfig, INJECTED_CONF ); - PostgresqlFirehoseDatabaseConnector andBack = MAPPER.readValue( + PostgresqlInputSourceDatabaseConnector andBack = MAPPER.readValue( MAPPER.writeValueAsString(connector), - PostgresqlFirehoseDatabaseConnector.class + PostgresqlInputSourceDatabaseConnector.class ); Assert.assertEquals(connector, andBack); } @@ -74,7 +74,7 @@ public class PostgresqlFirehoseDatabaseConnectorTest @Test public void testEqualsAndHashcode() { - EqualsVerifier.forClass(PostgresqlFirehoseDatabaseConnector.class) + EqualsVerifier.forClass(PostgresqlInputSourceDatabaseConnector.class) .usingGetClass() .withNonnullFields("connectorConfig") .withIgnoredFields("dbi") @@ -95,7 +95,7 @@ public class PostgresqlFirehoseDatabaseConnectorTest JdbcAccessSecurityConfig securityConfig = newSecurityConfigEnforcingAllowList(ImmutableSet.of()); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -115,7 +115,7 @@ public class PostgresqlFirehoseDatabaseConnectorTest JdbcAccessSecurityConfig securityConfig = newSecurityConfigEnforcingAllowList(ImmutableSet.of("user")); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -138,7 +138,7 @@ public class PostgresqlFirehoseDatabaseConnectorTest expectedException.expectMessage("is not in the allowed list"); expectedException.expect(IllegalArgumentException.class); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -160,7 +160,7 @@ public class PostgresqlFirehoseDatabaseConnectorTest ImmutableSet.of("user", "password", "keyonly", "etc") ); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -183,7 +183,7 @@ public class PostgresqlFirehoseDatabaseConnectorTest expectedException.expectMessage("is not in the allowed list"); expectedException.expect(IllegalArgumentException.class); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -206,7 +206,7 @@ public class PostgresqlFirehoseDatabaseConnectorTest expectedException.expectMessage("is not in the allowed list"); expectedException.expect(IllegalArgumentException.class); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -239,7 +239,7 @@ public class PostgresqlFirehoseDatabaseConnectorTest } }; - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); 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 687e26e9ff2..9c62e2969ca 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 @@ -29,6 +29,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.DoubleDimensionSchema; import org.apache.druid.data.input.impl.FloatDimensionSchema; import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; @@ -64,6 +65,7 @@ import org.apache.druid.sql.calcite.SqlTestFrameworkConfig; import org.apache.druid.sql.calcite.TempDirProducer; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.run.EngineFeature; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier; import org.apache.druid.sql.calcite.util.TestDataBuilder; @@ -73,6 +75,9 @@ import org.junit.jupiter.api.Test; import java.util.List; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + @SqlTestFrameworkConfig.ComponentSupplier(VarianceComponentSupplier.class) public class VarianceSqlAggregatorTest extends BaseCalciteQueryTest { @@ -724,4 +729,23 @@ public class VarianceSqlAggregatorTest extends BaseCalciteQueryTest )) .run(); } + + @Test + public void testStddevNotSupportedOverWindow() + { + assumeFeatureAvailable(EngineFeature.WINDOW_FUNCTIONS); + + DruidException e = assertThrows( + DruidException.class, + () -> testBuilder() + .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) + .sql("SELECT stddev(m1) OVER () from numfoo") + .run() + ); + + assertEquals( + "Query could not be planned. A possible reason is [Aggregation [STDDEV] is currently not supported for window functions]", + e.getMessage() + ); + } } diff --git a/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java b/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java index 43c2fb557a1..528d82bd965 100644 --- a/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java +++ b/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java @@ -74,7 +74,6 @@ public class SleepSqlTest extends BaseCalciteQueryTest .columns("v0") .filters(range("m1", ColumnType.DOUBLE, null, 2.0, false, true)) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java index 1e5da8af8ed..40a67f1236e 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java @@ -20,36 +20,62 @@ package org.apache.druid.indexer.hadoop; import com.google.common.base.Function; +import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.io.Closeables; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.Row; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.HadoopDruidIndexerConfig; import org.apache.druid.indexer.JobHelper; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.segment.BaseLongColumnValueSelector; +import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.realtime.firehose.IngestSegmentFirehose; -import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.segment.transform.Transformer; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import javax.annotation.Nullable; +import java.io.Closeable; import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; +import java.util.Map; public class DatasourceRecordReader extends RecordReader { private static final Logger logger = new Logger(DatasourceRecordReader.class); private DatasourceIngestionSpec spec; - private IngestSegmentFirehose firehose; + private SegmentReader segmentReader; private long rowNum; private Row currRow; @@ -108,7 +134,7 @@ public class DatasourceRecordReader extends RecordReader } ); - firehose = new IngestSegmentFirehose( + segmentReader = new SegmentReader( adapters, spec.getTransformSpec(), spec.getDimensions(), @@ -120,8 +146,8 @@ public class DatasourceRecordReader extends RecordReader @Override public boolean nextKeyValue() { - if (firehose.hasMore()) { - currRow = firehose.nextRow(); + if (segmentReader.hasMore()) { + currRow = segmentReader.nextRow(); rowNum++; return true; } else { @@ -154,7 +180,7 @@ public class DatasourceRecordReader extends RecordReader @Override public void close() throws IOException { - Closeables.close(firehose, true); + Closeables.close(segmentReader, true); for (QueryableIndex qi : indexes) { Closeables.close(qi, true); } @@ -163,4 +189,157 @@ public class DatasourceRecordReader extends RecordReader FileUtils.deleteDirectory(dir); } } + + public static class SegmentReader implements Closeable + { + private final Transformer transformer; + private Yielder rowYielder; + + public SegmentReader( + final List adapters, + final TransformSpec transformSpec, + final List dims, + final List metrics, + final DimFilter dimFilter + ) + { + this.transformer = transformSpec.toTransformer(); + + Sequence rows = Sequences.concat( + Iterables.transform( + adapters, + new Function>() + { + @Nullable + @Override + public Sequence apply(WindowedStorageAdapter adapter) + { + return Sequences.concat( + Sequences.map( + adapter.getAdapter().makeCursors( + Filters.toFilter(dimFilter), + adapter.getInterval(), + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), new Function>() + { + @Nullable + @Override + public Sequence apply(final Cursor cursor) + { + final BaseLongColumnValueSelector timestampColumnSelector = + cursor.getColumnSelectorFactory().makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME); + + final Map dimSelectors = new HashMap<>(); + for (String dim : dims) { + final DimensionSelector dimSelector = cursor + .getColumnSelectorFactory() + .makeDimensionSelector(new DefaultDimensionSpec(dim, dim)); + // dimSelector is null if the dimension is not present + if (dimSelector != null) { + dimSelectors.put(dim, dimSelector); + } + } + + final Map metSelectors = new HashMap<>(); + for (String metric : metrics) { + final BaseObjectColumnValueSelector metricSelector = + cursor.getColumnSelectorFactory().makeColumnValueSelector(metric); + metSelectors.put(metric, metricSelector); + } + + return Sequences.simple( + new Iterable() + { + @Override + public Iterator iterator() + { + return new Iterator() + { + @Override + public boolean hasNext() + { + return !cursor.isDone(); + } + + @Override + public InputRow next() + { + final Map theEvent = Maps.newLinkedHashMap(); + final long timestamp = timestampColumnSelector.getLong(); + theEvent.put(TimestampSpec.DEFAULT_COLUMN, DateTimes.utc(timestamp)); + + for (Map.Entry dimSelector : + dimSelectors.entrySet()) { + final String dim = dimSelector.getKey(); + final DimensionSelector selector = dimSelector.getValue(); + final IndexedInts vals = selector.getRow(); + + int valsSize = vals.size(); + if (valsSize == 1) { + final String dimVal = selector.lookupName(vals.get(0)); + theEvent.put(dim, dimVal); + } else if (valsSize > 1) { + List dimVals = new ArrayList<>(valsSize); + for (int i = 0; i < valsSize; ++i) { + dimVals.add(selector.lookupName(vals.get(i))); + } + theEvent.put(dim, dimVals); + } + } + + for (Map.Entry metSelector : + metSelectors.entrySet()) { + final String metric = metSelector.getKey(); + final BaseObjectColumnValueSelector selector = metSelector.getValue(); + Object value = selector.getObject(); + if (value != null) { + theEvent.put(metric, value); + } + } + cursor.advance(); + return new MapBasedInputRow(timestamp, dims, theEvent); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException("Remove Not Supported"); + } + }; + } + } + ); + } + } + ) + ); + } + } + ) + ); + rowYielder = Yielders.each(rows); + } + + public boolean hasMore() + { + return !rowYielder.isDone(); + } + + @Nullable + public InputRow nextRow() + { + final InputRow inputRow = rowYielder.get(); + rowYielder = rowYielder.next(null); + return transformer.transform(inputRow); + } + + @Override + public void close() throws IOException + { + rowYielder.close(); + } + } } diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java index ed8b8c0bb09..6516b0a0e00 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java @@ -26,13 +26,13 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import org.apache.commons.io.FileUtils; -import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.hll.HyperLogLogCollector; +import org.apache.druid.indexer.hadoop.DatasourceRecordReader; import org.apache.druid.indexer.hadoop.WindowedDataSegment; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -49,8 +49,7 @@ import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.LocalDataSegmentPuller; -import org.apache.druid.segment.realtime.firehose.IngestSegmentFirehose; -import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedStorageAdapter; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; @@ -404,7 +403,7 @@ public class BatchDeltaIngestionTest QueryableIndex index = INDEX_IO.loadIndex(tmpUnzippedSegmentDir); StorageAdapter adapter = new QueryableIndexStorageAdapter(index); - Firehose firehose = new IngestSegmentFirehose( + DatasourceRecordReader.SegmentReader segmentReader = new DatasourceRecordReader.SegmentReader( ImmutableList.of(new WindowedStorageAdapter(adapter, windowedDataSegment.getInterval())), TransformSpec.NONE, expectedDimensions, @@ -413,11 +412,12 @@ public class BatchDeltaIngestionTest ); List rows = new ArrayList<>(); - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); + while (segmentReader.hasMore()) { + rows.add(segmentReader.nextRow()); } verifyRows(expectedRowsGenerated, rows, expectedDimensions, expectedMetrics); + segmentReader.close(); } private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig(Map inputSpec, File tmpDir) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java similarity index 92% rename from server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java rename to indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java index 44f24f879a2..042aa5e9550 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.indexer.hadoop; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -47,6 +47,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; +import org.apache.druid.segment.realtime.WindowedStorageAdapter; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -66,7 +67,7 @@ import java.util.List; /** */ @RunWith(Parameterized.class) -public class IngestSegmentFirehoseTest +public class DatasourceRecordReaderSegmentReaderTest { private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( ImmutableList.of( @@ -107,7 +108,7 @@ public class IngestSegmentFirehoseTest private final IndexIO indexIO; private final IndexMerger indexMerger; - public IngestSegmentFirehoseTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + public DatasourceRecordReaderSegmentReaderTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { indexIO = TestHelper.getTestIndexIO(); indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); @@ -135,7 +136,7 @@ public class IngestSegmentFirehoseTest ) { final StorageAdapter sa = new QueryableIndexStorageAdapter(qi); final WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval()); - final IngestSegmentFirehose firehose = new IngestSegmentFirehose( + final DatasourceRecordReader.SegmentReader segmentReader = new DatasourceRecordReader.SegmentReader( ImmutableList.of(wsa, wsa), TransformSpec.NONE, ImmutableList.of("host", "spatial"), @@ -144,8 +145,8 @@ public class IngestSegmentFirehoseTest ); int count = 0; - while (firehose.hasMore()) { - final InputRow row = firehose.nextRow(); + while (segmentReader.hasMore()) { + final InputRow row = segmentReader.nextRow(); Assert.assertNotNull(row); if (count == 0) { Assert.assertEquals(DateTimes.of("2014-10-22T00Z"), row.getTimestamp()); @@ -168,15 +169,15 @@ public class IngestSegmentFirehoseTest Assert.assertEquals(ImmutableList.of("visited_sum", "unique_hosts"), queryable.getAvailableMetrics()); // Do a spatial filter - final IngestSegmentFirehose firehose2 = new IngestSegmentFirehose( + final DatasourceRecordReader.SegmentReader segmentReader2 = new DatasourceRecordReader.SegmentReader( ImmutableList.of(new WindowedStorageAdapter(queryable, Intervals.of("2000/3000"))), TransformSpec.NONE, ImmutableList.of("host", "spatial"), ImmutableList.of("visited_sum", "unique_hosts"), new SpatialDimFilter("spatial", new RadiusBound(new float[]{1, 0}, 0.1f)) ); - final InputRow row = firehose2.nextRow(); - Assert.assertFalse(firehose2.hasMore()); + final InputRow row = segmentReader2.nextRow(); + Assert.assertFalse(segmentReader2.hasMore()); Assert.assertEquals(DateTimes.of("2014-10-22T00Z"), row.getTimestamp()); Assert.assertEquals("host2", row.getRaw("host")); Assert.assertEquals("1,0", row.getRaw("spatial")); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 46de3064f03..9732d72eaf5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -58,9 +58,9 @@ import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 669a30dc5de..7190c38849f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -59,8 +59,8 @@ import org.apache.druid.segment.loading.DataSegmentMover; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java index db48d6f07f7..b8025e415a8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.EnumUtils; import org.apache.druid.common.config.Configs; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.java.util.common.ISE; @@ -65,16 +64,6 @@ public class TaskConfig } } - // This enum controls processing mode of batch ingestion "segment creation" phase (i.e. appenderator logic) - public enum BatchProcessingMode - { - OPEN_SEGMENTS, /* mmap segments, legacy code */ - CLOSED_SEGMENTS, /* Do not mmap segments but keep most other legacy code */ - CLOSED_SEGMENTS_SINKS /* Most aggressive memory optimization, do not mmap segments and eliminate sinks, etc. */ - } - - public static final BatchProcessingMode BATCH_PROCESSING_MODE_DEFAULT = BatchProcessingMode.CLOSED_SEGMENTS; - private static final Period DEFAULT_DIRECTORY_LOCK_TIMEOUT = new Period("PT10M"); private static final Period DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT = new Period("PT5M"); private static final boolean DEFAULT_STORE_EMPTY_COLUMNS = true; @@ -110,12 +99,6 @@ public class TaskConfig @JsonProperty private final boolean ignoreTimestampSpecForDruidInputSource; - @JsonProperty - private final boolean batchMemoryMappedIndex; - - @JsonProperty - private final BatchProcessingMode batchProcessingMode; - @JsonProperty private final boolean storeEmptyColumns; @@ -137,9 +120,6 @@ public class TaskConfig @JsonProperty("directoryLockTimeout") Period directoryLockTimeout, @JsonProperty("shuffleDataLocations") List shuffleDataLocations, @JsonProperty("ignoreTimestampSpecForDruidInputSource") boolean ignoreTimestampSpecForDruidInputSource, - @JsonProperty("batchMemoryMappedIndex") boolean batchMemoryMappedIndex, - // deprecated, only set to true to fall back to older behavior - @JsonProperty("batchProcessingMode") String batchProcessingMode, @JsonProperty("storeEmptyColumns") @Nullable Boolean storeEmptyColumns, @JsonProperty("encapsulatedTask") boolean enableTaskLevelLogPush, @JsonProperty("tmpStorageBytesPerTask") @Nullable Long tmpStorageBytesPerTask @@ -171,26 +151,8 @@ public class TaskConfig ); this.ignoreTimestampSpecForDruidInputSource = ignoreTimestampSpecForDruidInputSource; - this.batchMemoryMappedIndex = batchMemoryMappedIndex; this.encapsulatedTask = enableTaskLevelLogPush; - // Conflict resolution. Assume that if batchMemoryMappedIndex is set (since false is the default) that - // the user changed it intentionally to use legacy, in this case oveeride batchProcessingMode and also - // set it to legacy else just use batchProcessingMode and don't pay attention to batchMemoryMappedIndexMode: - if (batchMemoryMappedIndex) { - this.batchProcessingMode = BatchProcessingMode.OPEN_SEGMENTS; - } else if (EnumUtils.isValidEnum(BatchProcessingMode.class, batchProcessingMode)) { - this.batchProcessingMode = BatchProcessingMode.valueOf(batchProcessingMode); - } else { - // batchProcessingMode input string is invalid, log & use the default. - this.batchProcessingMode = BatchProcessingMode.CLOSED_SEGMENTS; // Default - log.warn( - "Batch processing mode argument value is null or not valid:[%s], defaulting to[%s] ", - batchProcessingMode, this.batchProcessingMode - ); - } - log.debug("Batch processing mode:[%s]", this.batchProcessingMode); - this.storeEmptyColumns = Configs.valueOrDefault(storeEmptyColumns, DEFAULT_STORE_EMPTY_COLUMNS); this.tmpStorageBytesPerTask = Configs.valueOrDefault(tmpStorageBytesPerTask, DEFAULT_TMP_STORAGE_BYTES_PER_TASK); } @@ -206,8 +168,6 @@ public class TaskConfig Period directoryLockTimeout, List shuffleDataLocations, boolean ignoreTimestampSpecForDruidInputSource, - boolean batchMemoryMappedIndex, - BatchProcessingMode batchProcessingMode, boolean storeEmptyColumns, boolean encapsulatedTask, long tmpStorageBytesPerTask @@ -223,8 +183,6 @@ public class TaskConfig this.directoryLockTimeout = directoryLockTimeout; this.shuffleDataLocations = shuffleDataLocations; this.ignoreTimestampSpecForDruidInputSource = ignoreTimestampSpecForDruidInputSource; - this.batchMemoryMappedIndex = batchMemoryMappedIndex; - this.batchProcessingMode = batchProcessingMode; this.storeEmptyColumns = storeEmptyColumns; this.encapsulatedTask = encapsulatedTask; this.tmpStorageBytesPerTask = tmpStorageBytesPerTask; @@ -310,22 +268,6 @@ public class TaskConfig return ignoreTimestampSpecForDruidInputSource; } - @JsonProperty - public BatchProcessingMode getBatchProcessingMode() - { - return batchProcessingMode; - } - - /** - * Do not use in code! use {@link TaskConfig#getBatchProcessingMode() instead} - */ - @Deprecated - @JsonProperty - public boolean getbatchMemoryMappedIndex() - { - return batchMemoryMappedIndex; - } - @JsonProperty public boolean isStoreEmptyColumns() { @@ -366,8 +308,6 @@ public class TaskConfig directoryLockTimeout, shuffleDataLocations, ignoreTimestampSpecForDruidInputSource, - batchMemoryMappedIndex, - batchProcessingMode, storeEmptyColumns, encapsulatedTask, tmpStorageBytesPerTask @@ -387,8 +327,6 @@ public class TaskConfig directoryLockTimeout, shuffleDataLocations, ignoreTimestampSpecForDruidInputSource, - batchMemoryMappedIndex, - batchProcessingMode, storeEmptyColumns, encapsulatedTask, tmpStorageBytesPerTask diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java index 087464b48ac..6af4402c35a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java @@ -21,8 +21,6 @@ package org.apache.druid.indexing.common.task; import org.apache.druid.indexing.appenderator.ActionBasedPublishedSegmentRetriever; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; @@ -75,54 +73,20 @@ public final class BatchAppenderators boolean useMaxMemoryEstimates ) { - if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.OPEN_SEGMENTS) { - return appenderatorsManager.createOpenSegmentsOfflineAppenderatorForTask( - taskId, - dataSchema, - appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()), - metrics, - segmentPusher, - toolbox.getJsonMapper(), - toolbox.getIndexIO(), - toolbox.getIndexMergerV9(), - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - toolbox.getCentralizedTableSchemaConfig() - ); - } else if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS) { - return appenderatorsManager.createClosedSegmentsOfflineAppenderatorForTask( - taskId, - dataSchema, - appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()), - metrics, - segmentPusher, - toolbox.getJsonMapper(), - toolbox.getIndexIO(), - toolbox.getIndexMergerV9(), - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - toolbox.getCentralizedTableSchemaConfig() - ); - } else if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS_SINKS) { - return appenderatorsManager.createOfflineAppenderatorForTask( - taskId, - dataSchema, - appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()), - metrics, - segmentPusher, - toolbox.getJsonMapper(), - toolbox.getIndexIO(), - toolbox.getIndexMergerV9(), - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - toolbox.getCentralizedTableSchemaConfig() - ); - } else { - throw new IAE("Invalid batchProcesingMode[%s]", toolbox.getConfig().getBatchProcessingMode()); - } + return appenderatorsManager.createBatchAppenderatorForTask( + taskId, + dataSchema, + appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + segmentPusher, + toolbox.getJsonMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + rowIngestionMeters, + parseExceptionHandler, + useMaxMemoryEstimates, + toolbox.getCentralizedTableSchemaConfig() + ); } public static BatchAppenderatorDriver newDriver( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index 8d30a60d04e..0abaeed8eb2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -57,6 +57,9 @@ public interface CompactionRunner * Checks if the provided compaction config is supported by the runner. * The same validation is done at {@link org.apache.druid.msq.indexing.MSQCompactionRunner#validateCompactionTask} */ - CompactionConfigValidationResult validateCompactionTask(CompactionTask compactionTask); + CompactionConfigValidationResult validateCompactionTask( + CompactionTask compactionTask, + Map intervalToDataSchemaMap + ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 8659eb0f397..68320387845 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -77,6 +77,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -459,11 +460,13 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg transformSpec, metricsSpec, granularitySpec, - getMetricBuilder() + getMetricBuilder(), + compactionRunner ); registerResourceCloserOnAbnormalExit(compactionRunner.getCurrentSubTaskHolder()); - CompactionConfigValidationResult supportsCompactionConfig = compactionRunner.validateCompactionTask(this); + CompactionConfigValidationResult supportsCompactionConfig = + compactionRunner.validateCompactionTask(this, intervalDataSchemas); if (!supportsCompactionConfig.isValid()) { throw InvalidInput.exception("Compaction spec not supported. Reason[%s].", supportsCompactionConfig.getReason()); } @@ -485,7 +488,8 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg @Nullable final ClientCompactionTaskTransformSpec transformSpec, @Nullable final AggregatorFactory[] metricsSpec, @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, - final ServiceMetricEvent.Builder metricBuilder + final ServiceMetricEvent.Builder metricBuilder, + CompactionRunner compactionRunner ) throws IOException { final Iterable timelineSegments = retrieveRelevantTimelineHolders( @@ -549,7 +553,8 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg metricsSpec, granularitySpec == null ? new ClientCompactionTaskGranularitySpec(segmentGranularityToUse, null, null) - : granularitySpec.withSegmentGranularity(segmentGranularityToUse) + : granularitySpec.withSegmentGranularity(segmentGranularityToUse), + compactionRunner ); intervalDataSchemaMap.put(interval, dataSchema); } @@ -574,7 +579,8 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg dimensionsSpec, transformSpec, metricsSpec, - granularitySpec + granularitySpec, + compactionRunner ); return Collections.singletonMap(segmentProvider.interval, dataSchema); } @@ -604,13 +610,17 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg @Nullable DimensionsSpec dimensionsSpec, @Nullable ClientCompactionTaskTransformSpec transformSpec, @Nullable AggregatorFactory[] metricsSpec, - @Nonnull ClientCompactionTaskGranularitySpec granularitySpec + @Nonnull ClientCompactionTaskGranularitySpec granularitySpec, + @Nullable CompactionRunner compactionRunner ) { // Check index metadata & decide which values to propagate (i.e. carry over) for rollup & queryGranularity final ExistingSegmentAnalyzer existingSegmentAnalyzer = new ExistingSegmentAnalyzer( segments, - granularitySpec.isRollup() == null, + // For MSQ, always need rollup to check if there are some rollup segments already present. + compactionRunner instanceof NativeCompactionRunner + ? (granularitySpec.isRollup() == null) + : true, granularitySpec.getQueryGranularity() == null, dimensionsSpec == null, metricsSpec == null @@ -665,13 +675,14 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg finalMetricsSpec = metricsSpec; } - return new DataSchema( + return new CombinedDataSchema( dataSource, new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null), finalDimensionsSpec, finalMetricsSpec, uniformGranularitySpec, - transformSpec == null ? null : new TransformSpec(transformSpec.getFilter(), null) + transformSpec == null ? null : new TransformSpec(transformSpec.getFilter(), null), + existingSegmentAnalyzer.hasRolledUpSegments() ); } @@ -748,6 +759,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg // For processRollup: private boolean rollup = true; + private boolean hasRolledUpSegments = false; // For processQueryGranularity: private Granularity queryGranularity; @@ -815,6 +827,11 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg return rollup; } + public boolean hasRolledUpSegments() + { + return hasRolledUpSegments; + } + public Granularity getQueryGranularity() { if (!needQueryGranularity) { @@ -904,6 +921,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg // Pick rollup value if all segments being compacted have the same, non-null, value otherwise set it to false final Boolean isIndexRollup = index.getMetadata().isRollup(); rollup = rollup && Boolean.valueOf(true).equals(isIndexRollup); + hasRolledUpSegments = hasRolledUpSegments || Boolean.valueOf(true).equals(isIndexRollup); } private void processQueryGranularity(final QueryableIndex index) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index bc9d5aca2b6..f1f96d8ca34 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -59,8 +59,8 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.realtime.firehose.ChatHandler; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; +import org.apache.druid.segment.realtime.ChatHandler; +import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.Resource; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index dc6c07b6b83..a8f60cca572 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -21,8 +21,6 @@ package org.apache.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonInclude; -import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.databind.ObjectMapper; @@ -34,7 +32,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputSource; @@ -87,6 +84,7 @@ import org.apache.druid.segment.indexing.IngestionSpec; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; @@ -95,7 +93,6 @@ import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; -import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizerMapper; @@ -312,9 +309,6 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler, Pe @Override public Set getInputSourceResources() { - if (ingestionSchema.getIOConfig().firehoseFactory != null) { - throw getInputSecurityOnFirehoseUnsupportedError(); - } return getIngestionSchema().getIOConfig().getInputSource() != null ? getIngestionSchema().getIOConfig().getInputSource().getTypes() .stream() @@ -1123,8 +1117,6 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler, Pe @JsonTypeName("index") public static class IndexIOConfig implements BatchIOConfig { - - private final FirehoseFactory firehoseFactory; private final InputSource inputSource; private final AtomicReference inputSourceWithToolbox = new AtomicReference<>(); private final InputFormat inputFormat; @@ -1133,42 +1125,18 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler, Pe @JsonCreator public IndexIOConfig( - @Deprecated @JsonProperty("firehose") @Nullable FirehoseFactory firehoseFactory, @JsonProperty("inputSource") @Nullable InputSource inputSource, @JsonProperty("inputFormat") @Nullable InputFormat inputFormat, @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting, @JsonProperty("dropExisting") @Nullable Boolean dropExisting ) { - Checks.checkOneNotNullOrEmpty( - ImmutableList.of(new Property<>("firehose", firehoseFactory), new Property<>("inputSource", inputSource)) - ); - if (firehoseFactory != null && inputFormat != null) { - throw new IAE("Cannot use firehose and inputFormat together. Try using inputSource instead of firehose."); - } - this.firehoseFactory = firehoseFactory; this.inputSource = inputSource; this.inputFormat = inputFormat; this.appendToExisting = appendToExisting == null ? BatchIOConfig.DEFAULT_APPEND_EXISTING : appendToExisting; this.dropExisting = dropExisting == null ? BatchIOConfig.DEFAULT_DROP_EXISTING : dropExisting; } - // old constructor for backward compatibility - @Deprecated - public IndexIOConfig(FirehoseFactory firehoseFactory, @Nullable Boolean appendToExisting, @Nullable Boolean dropExisting) - { - this(firehoseFactory, null, null, appendToExisting, dropExisting); - } - - @Nullable - @JsonProperty("firehose") - @JsonInclude(Include.NON_NULL) - @Deprecated - public FirehoseFactory getFirehoseFactory() - { - return firehoseFactory; - } - @Nullable @Override @JsonProperty diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index f2eacb8c1c6..5aa7af71451 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -85,10 +85,11 @@ public class NativeCompactionRunner implements CompactionRunner @Override public CompactionConfigValidationResult validateCompactionTask( - CompactionTask compactionTask + CompactionTask compactionTask, + Map intervalToDataSchemaMap ) { - return new CompactionConfigValidationResult(true, null); + return CompactionConfigValidationResult.success(); } /** @@ -157,7 +158,6 @@ public class NativeCompactionRunner implements CompactionRunner } return new ParallelIndexIOConfig( - null, new DruidInputSource( dataSchema.getDataSource(), interval, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 4fb4bb7852c..9b882e2e8d2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -152,8 +152,7 @@ public interface Task * the task does not use any. Users can be given permission to access particular types of * input sources but not others, using the * {@link org.apache.druid.server.security.AuthConfig#enableInputSourceSecurity} config. - * @throws UnsupportedOperationException if the given task type does not suppoert input source based security. Such - * would be the case, if the task uses firehose. + * @throws UnsupportedOperationException if the given task type does not suppoert input source based security */ @JsonIgnore @Nonnull @@ -165,15 +164,6 @@ public interface Task )); } - default UOE getInputSecurityOnFirehoseUnsupportedError() - { - throw new UOE(StringUtils.format( - "Input source based security cannot be performed '%s' task because it uses firehose." - + " Change the tasks configuration, or disable `isEnableInputSourceSecurity`", - getType() - )); - } - /** * Returns query runners for this task. If this task is not meant to answer queries over its datasource, this method * should return null. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java index 286325186b2..e6097678b5a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.SplittableInputSource; @@ -81,14 +80,11 @@ abstract class InputSourceSplitParallelIndexTaskRunner newTaskSpec(InputSplit split) { - final FirehoseFactory firehoseFactory; final InputSource inputSource; - firehoseFactory = null; inputSource = baseInputSource.withSplit(split); final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec( ingestionSchema.getDataSchema(), new ParallelIndexIOConfig( - firehoseFactory, inputSource, ingestionSchema.getIOConfig().getInputFormat(), ingestionSchema.getIOConfig().isAppendToExisting(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/LegacySinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/LegacySinglePhaseSubTask.java index 1daa9999427..27a242885c3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/LegacySinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/LegacySinglePhaseSubTask.java @@ -71,9 +71,6 @@ public class LegacySinglePhaseSubTask extends SinglePhaseSubTask @Override public Set getInputSourceResources() { - if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) { - throw getInputSecurityOnFirehoseUnsupportedError(); - } return getIngestionSchema().getIOConfig().getInputSource() != null ? getIngestionSchema().getIOConfig().getInputSource().getTypes() .stream() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java index c86446d9130..ff10e556de8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java @@ -22,7 +22,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; @@ -37,26 +36,12 @@ public class ParallelIndexIOConfig extends IndexIOConfig { @JsonCreator public ParallelIndexIOConfig( - @JsonProperty("firehose") @Nullable FirehoseFactory firehoseFactory, @JsonProperty("inputSource") @Nullable InputSource inputSource, @JsonProperty("inputFormat") @Nullable InputFormat inputFormat, @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting, @JsonProperty("dropExisting") @Nullable Boolean dropExisting ) { - super(firehoseFactory, inputSource, inputFormat, appendToExisting, dropExisting); - } - - // old constructor for backward compatibility - @Deprecated - public ParallelIndexIOConfig(FirehoseFactory firehoseFactory, @Nullable Boolean appendToExisting) - { - this(firehoseFactory, null, null, appendToExisting, null); - } - - @Deprecated - public ParallelIndexIOConfig(FirehoseFactory firehoseFactory, @Nullable Boolean appendToExisting, boolean dropExisting) - { - this(firehoseFactory, null, null, appendToExisting, dropExisting); + super(inputSource, inputFormat, appendToExisting, dropExisting); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 4ca0f1ff80d..6039a62bc98 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -75,10 +75,10 @@ import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandler; +import org.apache.druid.segment.realtime.ChatHandlers; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; -import org.apache.druid.segment.realtime.firehose.ChatHandler; -import org.apache.druid.segment.realtime.firehose.ChatHandlers; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.Resource; @@ -290,9 +290,6 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask @Override public Set getInputSourceResources() { - if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) { - throw getInputSecurityOnFirehoseUnsupportedError(); - } return getIngestionSchema().getIOConfig().getInputSource() != null ? getIngestionSchema().getIOConfig().getInputSource().getTypes() .stream() @@ -556,7 +553,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask } else { if (!baseInputSource.isSplittable()) { LOG.warn( - "firehoseFactory[%s] is not splittable. Running sequentially.", + "inputSource[%s] is not splittable. Running sequentially.", baseInputSource.getClass().getSimpleName() ); } else if (ingestionSchema.getTuningConfig().getMaxNumConcurrentSubTasks() <= 1) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java index 6724a2ebb90..d75304f38c0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java @@ -139,9 +139,6 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask @Override public Set getInputSourceResources() { - if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) { - throw getInputSecurityOnFirehoseUnsupportedError(); - } return getIngestionSchema().getIOConfig().getInputSource() != null ? getIngestionSchema().getIOConfig().getInputSource().getTypes() .stream() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java index cb361a204ea..4bb395420b3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java @@ -180,9 +180,6 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask @Override public Set getInputSourceResources() { - if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) { - throw getInputSecurityOnFirehoseUnsupportedError(); - } return getIngestionSchema().getIOConfig().getInputSource() != null ? getIngestionSchema().getIOConfig().getInputSource().getTypes() .stream() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java index 383fc7afb2e..4c224e396d1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -135,9 +135,6 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask getInputSourceResources() { - if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) { - throw getInputSecurityOnFirehoseUnsupportedError(); - } return getIngestionSchema().getIOConfig().getInputSource() != null ? getIngestionSchema().getIOConfig().getInputSource().getTypes() .stream() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java index cf7ae15a9a5..933df9ee778 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java @@ -153,9 +153,6 @@ public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask< @Override public Set getInputSourceResources() { - if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) { - throw getInputSecurityOnFirehoseUnsupportedError(); - } return getIngestionSchema().getIOConfig().getInputSource() != null ? getIngestionSchema().getIOConfig().getInputSource().getTypes() .stream() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index c53557c6655..620d593467a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -37,8 +37,8 @@ import org.apache.druid.indexing.common.task.SequenceNameFunction; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskInputRowIteratorBuilder; -import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.indexing.input.WindowedSegmentId; import org.apache.druid.indexing.worker.shuffle.ShuffleDataSegmentPusher; import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.SegmentSchemaMapping; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index ce6aee98af3..a2a29b3cdd3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -21,7 +21,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang3.mutable.MutableObject; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.SplittableInputSource; @@ -174,9 +173,7 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner @VisibleForTesting SubTaskSpec newTaskSpec(InputSplit split) { - final FirehoseFactory firehoseFactory; final InputSource inputSource; - firehoseFactory = null; inputSource = baseInputSource.withSplit(split); final Map subtaskContext = new HashMap<>(getContext()); @@ -187,7 +184,6 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner new ParallelIndexIngestionSpec( ingestionSchema.getDataSchema(), new ParallelIndexIOConfig( - firehoseFactory, inputSource, ingestionSchema.getIOConfig().getInputFormat(), ingestionSchema.getIOConfig().isAppendToExisting(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index cd05e50ff1c..8d49a7f3dbe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -60,13 +60,13 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; -import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.Resource; @@ -198,9 +198,6 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand @Override public Set getInputSourceResources() { - if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) { - throw getInputSecurityOnFirehoseUnsupportedError(); - } return getIngestionSchema().getIOConfig().getInputSource() != null ? getIngestionSchema().getIOConfig().getInputSource().getTypes() .stream() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java index 89afdbb64d0..3e3d653e3c3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task.batch.parallel.iterator; -import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.HandlingInputRowIterator; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -34,7 +33,6 @@ public interface IndexTaskInputRowIteratorBuilder /** * @param granularitySpec {@link GranularitySpec} for the {@link org.apache.druid.segment.indexing.DataSchema} - * associated with the {@link Firehose}. */ IndexTaskInputRowIteratorBuilder granularitySpec(GranularitySpec granularitySpec); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java index dd1998645b3..84c17f1a8fa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java @@ -51,7 +51,6 @@ import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -133,7 +132,7 @@ public class DruidInputSource extends AbstractInputSource implements SplittableI private final String dataSource; // Exactly one of interval and segmentIds should be non-null. Typically 'interval' is specified directly - // by the user creating this firehose and 'segmentIds' is used for sub-tasks if it is split for parallel + // by the user creating this input source and 'segmentIds' is used for sub-tasks if it is split for parallel // batch ingestion. @Nullable private final Interval interval; @@ -164,7 +163,7 @@ public class DruidInputSource extends AbstractInputSource implements SplittableI public DruidInputSource( @JsonProperty("dataSource") final String dataSource, @JsonProperty("interval") @Nullable Interval interval, - // Specifying "segments" is intended only for when this FirehoseFactory has split itself, + // Specifying "segments" is intended only for when this input source has split itself, // not for direct end user use. @JsonProperty("segments") @Nullable List segmentIds, @JsonProperty("filter") DimFilter dimFilter, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index d048cf4d8b9..42618556d00 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java @@ -56,7 +56,7 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedStorageAdapter; import org.apache.druid.utils.CloseableUtils; import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/WindowedSegmentId.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/WindowedSegmentId.java similarity index 98% rename from indexing-service/src/main/java/org/apache/druid/indexing/firehose/WindowedSegmentId.java rename to indexing-service/src/main/java/org/apache/druid/indexing/input/WindowedSegmentId.java index b55510a7e21..8232aba0446 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/WindowedSegmentId.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/WindowedSegmentId.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.firehose; +package org.apache.druid.indexing.input; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java new file mode 100644 index 00000000000..f99189d3589 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java @@ -0,0 +1,261 @@ +/* + * 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.indexing.overlord; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; +import com.google.inject.Inject; +import org.apache.druid.client.indexing.IndexingService; +import org.apache.druid.curator.discovery.ServiceAnnouncer; +import org.apache.druid.discovery.DruidLeaderSelector; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexing.common.actions.SegmentAllocationQueue; +import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.task.TaskContextEnricher; +import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; +import org.apache.druid.indexing.overlord.config.TaskLockConfig; +import org.apache.druid.indexing.overlord.config.TaskQueueConfig; +import org.apache.druid.indexing.overlord.duty.OverlordDutyExecutor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; + +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Encapsulates the leadership lifecycle of the Druid Overlord service. + * No classes other than Resource endpoints should have this class as a dependency. + * To query the current state of the Overlord, use {@link TaskMaster} instead. + */ +public class DruidOverlord +{ + private static final EmittingLogger log = new EmittingLogger(DruidOverlord.class); + + private final DruidLeaderSelector overlordLeaderSelector; + private final DruidLeaderSelector.Listener leadershipListener; + + private final ReentrantLock giant = new ReentrantLock(true); + + private final AtomicReference leaderLifecycleRef = new AtomicReference<>(null); + + /** + * Indicates that all services have been started and the node can now announce + * itself with {@link ServiceAnnouncer#announce}. This must be set to false + * as soon as {@link DruidLeaderSelector.Listener#stopBeingLeader()} is + * called. + */ + private volatile boolean initialized; + + @Inject + public DruidOverlord( + final TaskMaster taskMaster, + final TaskLockConfig taskLockConfig, + final TaskQueueConfig taskQueueConfig, + final DefaultTaskConfig defaultTaskConfig, + final TaskLockbox taskLockbox, + final TaskStorage taskStorage, + final TaskActionClientFactory taskActionClientFactory, + @Self final DruidNode selfNode, + final TaskRunnerFactory runnerFactory, + final ServiceAnnouncer serviceAnnouncer, + final CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, + final ServiceEmitter emitter, + final SupervisorManager supervisorManager, + final OverlordDutyExecutor overlordDutyExecutor, + @IndexingService final DruidLeaderSelector overlordLeaderSelector, + final SegmentAllocationQueue segmentAllocationQueue, + final ObjectMapper mapper, + final TaskContextEnricher taskContextEnricher + ) + { + this.overlordLeaderSelector = overlordLeaderSelector; + + final DruidNode node = coordinatorOverlordServiceConfig.getOverlordService() == null ? selfNode : + selfNode.withService(coordinatorOverlordServiceConfig.getOverlordService()); + + this.leadershipListener = new DruidLeaderSelector.Listener() + { + @Override + public void becomeLeader() + { + giant.lock(); + + // I AM THE MASTER OF THE UNIVERSE. + log.info("By the power of Grayskull, I have the power!"); + + try { + final TaskRunner taskRunner = runnerFactory.build(); + final TaskQueue taskQueue = new TaskQueue( + taskLockConfig, + taskQueueConfig, + defaultTaskConfig, + taskStorage, + taskRunner, + taskActionClientFactory, + taskLockbox, + emitter, + mapper, + taskContextEnricher + ); + + // Sensible order to start stuff: + final Lifecycle leaderLifecycle = new Lifecycle("task-master"); + if (leaderLifecycleRef.getAndSet(leaderLifecycle) != null) { + log.makeAlert("TaskMaster set a new Lifecycle without the old one being cleared! Race condition") + .emit(); + } + + leaderLifecycle.addManagedInstance(taskRunner); + leaderLifecycle.addManagedInstance(taskQueue); + leaderLifecycle.addManagedInstance(supervisorManager); + leaderLifecycle.addManagedInstance(overlordDutyExecutor); + leaderLifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() + { + segmentAllocationQueue.becomeLeader(); + taskMaster.becomeLeader(taskRunner, taskQueue); + + // Announce the node only after all the services have been initialized + initialized = true; + serviceAnnouncer.announce(node); + } + + @Override + public void stop() + { + serviceAnnouncer.unannounce(node); + taskMaster.stopBeingLeader(); + segmentAllocationQueue.stopBeingLeader(); + } + } + ); + + leaderLifecycle.start(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + finally { + giant.unlock(); + } + } + + @Override + public void stopBeingLeader() + { + giant.lock(); + try { + initialized = false; + final Lifecycle leaderLifecycle = leaderLifecycleRef.getAndSet(null); + + if (leaderLifecycle != null) { + leaderLifecycle.stop(); + } + } + finally { + giant.unlock(); + } + } + }; + } + + /** + * Starts waiting for leadership. + * Should be called only once throughout the life of the service. + */ + @LifecycleStart + public void start() + { + giant.lock(); + + try { + overlordLeaderSelector.registerListener(leadershipListener); + } + finally { + giant.unlock(); + } + } + + /** + * Stops forever (not just this particular leadership session). + * Should be called only once throughout the life of the service. + */ + @LifecycleStop + public void stop() + { + giant.lock(); + + try { + gracefulStopLeaderLifecycle(); + overlordLeaderSelector.unregisterListener(); + } + finally { + giant.unlock(); + } + } + + /** + * @return true if it's the leader and all its services have been initialized. + */ + public boolean isLeader() + { + return overlordLeaderSelector.isLeader() && initialized; + } + + public String getCurrentLeader() + { + return overlordLeaderSelector.getCurrentLeader(); + } + + public Optional getRedirectLocation() + { + String leader = overlordLeaderSelector.getCurrentLeader(); + // do not redirect when + // leader is not elected + // leader is the current node + if (leader == null || leader.isEmpty() || overlordLeaderSelector.isLeader()) { + return Optional.absent(); + } else { + return Optional.of(leader); + } + } + + private void gracefulStopLeaderLifecycle() + { + try { + if (isLeader()) { + leadershipListener.stopBeingLeader(); + } + } + catch (Exception ex) { + // fail silently since we are stopping anyway + } + } + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 2155ac2c265..bebb52157d6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -42,6 +42,7 @@ import org.apache.druid.indexing.common.task.PendingSegmentAllocatingTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; @@ -49,6 +50,7 @@ import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.metadata.ReplaceTaskLock; +import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -992,50 +994,76 @@ public class TaskLockbox } /** - * Gets a List of Intervals locked by higher priority tasks for each datasource. - * Here, Segment Locks are being treated the same as Time Chunk Locks i.e. - * a Task with a Segment Lock is assumed to lock a whole Interval and not just - * the corresponding Segment. - * - * @param minTaskPriority Minimum task priority for each datasource. Only the - * Intervals that are locked by Tasks with equal or - * higher priority than this are returned. Locked intervals - * for datasources that are not present in this Map are - * not returned. - * @return Map from Datasource to List of Intervals locked by Tasks that have - * priority greater than or equal to the {@code minTaskPriority} for that datasource. + * @param lockFilterPolicies Lock filters for the given datasources + * @return Map from datasource to list of non-revoked locks with at least as much priority and an overlapping interval */ - public Map> getLockedIntervals(Map minTaskPriority) + public Map> getActiveLocks(List lockFilterPolicies) { - final Map> datasourceToIntervals = new HashMap<>(); + final Map> datasourceToLocks = new HashMap<>(); // Take a lock and populate the maps giant.lock(); + try { - running.forEach( - (datasource, datasourceLocks) -> { - // If this datasource is not requested, do not proceed - if (!minTaskPriority.containsKey(datasource)) { + lockFilterPolicies.forEach( + lockFilter -> { + final String datasource = lockFilter.getDatasource(); + if (!running.containsKey(datasource)) { return; } - datasourceLocks.forEach( + final int priority = lockFilter.getPriority(); + final List intervals; + if (lockFilter.getIntervals() != null) { + intervals = lockFilter.getIntervals(); + } else { + intervals = Collections.singletonList(Intervals.ETERNITY); + } + + final Map context = lockFilter.getContext(); + final boolean ignoreAppendLocks; + final Boolean useConcurrentLocks = QueryContexts.getAsBoolean( + Tasks.USE_CONCURRENT_LOCKS, + context.get(Tasks.USE_CONCURRENT_LOCKS) + ); + if (useConcurrentLocks == null) { + TaskLockType taskLockType = QueryContexts.getAsEnum( + Tasks.TASK_LOCK_TYPE, + context.get(Tasks.TASK_LOCK_TYPE), + TaskLockType.class + ); + if (taskLockType == null) { + ignoreAppendLocks = Tasks.DEFAULT_USE_CONCURRENT_LOCKS; + } else { + ignoreAppendLocks = taskLockType == TaskLockType.APPEND; + } + } else { + ignoreAppendLocks = useConcurrentLocks; + } + + running.get(datasource).forEach( (startTime, startTimeLocks) -> startTimeLocks.forEach( (interval, taskLockPosses) -> taskLockPosses.forEach( taskLockPosse -> { if (taskLockPosse.getTaskLock().isRevoked()) { - // Do not proceed if the lock is revoked - return; + // do nothing } else if (taskLockPosse.getTaskLock().getPriority() == null - || taskLockPosse.getTaskLock().getPriority() < minTaskPriority.get(datasource)) { - // Do not proceed if the lock has a priority strictly less than the minimum - return; + || taskLockPosse.getTaskLock().getPriority() < priority) { + // do nothing + } else if (ignoreAppendLocks + && taskLockPosse.getTaskLock().getType() == TaskLockType.APPEND) { + // do nothing + } else { + for (Interval filterInterval : intervals) { + if (interval.overlaps(filterInterval)) { + datasourceToLocks.computeIfAbsent(datasource, ds -> new ArrayList<>()) + .add(taskLockPosse.getTaskLock()); + break; + } + } } - - datasourceToIntervals - .computeIfAbsent(datasource, k -> new HashSet<>()) - .add(interval); - }) + } + ) ) ); } @@ -1045,11 +1073,7 @@ public class TaskLockbox giant.unlock(); } - return datasourceToIntervals.entrySet().stream() - .collect(Collectors.toMap( - Map.Entry::getKey, - entry -> new ArrayList<>(entry.getValue()) - )); + return datasourceToLocks; } public void unlock(final Task task, final Interval interval) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java index 5103f9bd87e..1de95352504 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java @@ -19,254 +19,61 @@ package org.apache.druid.indexing.overlord; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.inject.Inject; -import org.apache.druid.client.indexing.IndexingService; -import org.apache.druid.curator.discovery.ServiceAnnouncer; -import org.apache.druid.discovery.DruidLeaderSelector; -import org.apache.druid.discovery.DruidLeaderSelector.Listener; -import org.apache.druid.guice.annotations.Self; -import org.apache.druid.indexing.common.actions.SegmentAllocationQueue; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.common.task.TaskContextEnricher; import org.apache.druid.indexing.overlord.autoscaling.ScalingStats; -import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; -import org.apache.druid.indexing.overlord.config.TaskLockConfig; -import org.apache.druid.indexing.overlord.config.TaskQueueConfig; -import org.apache.druid.indexing.overlord.duty.OverlordDutyExecutor; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; -import org.apache.druid.java.util.common.lifecycle.Lifecycle; -import org.apache.druid.java.util.common.lifecycle.LifecycleStart; -import org.apache.druid.java.util.common.lifecycle.LifecycleStop; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.metrics.TaskCountStatsProvider; import org.apache.druid.server.metrics.TaskSlotCountStatsProvider; import javax.annotation.Nullable; import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.atomic.AtomicBoolean; /** - * Encapsulates the indexer leadership lifecycle. + * Encapsulates various Overlord classes that allow querying and updating the + * current state of the Overlord leader. */ public class TaskMaster implements TaskCountStatsProvider, TaskSlotCountStatsProvider { - private static final EmittingLogger log = new EmittingLogger(TaskMaster.class); - - private final DruidLeaderSelector overlordLeaderSelector; - private final DruidLeaderSelector.Listener leadershipListener; - - private final ReentrantLock giant = new ReentrantLock(true); private final TaskActionClientFactory taskActionClientFactory; private final SupervisorManager supervisorManager; - - private final AtomicReference leaderLifecycleRef = new AtomicReference<>(null); - private volatile TaskRunner taskRunner; private volatile TaskQueue taskQueue; - /** - * This flag indicates that all services has been started and should be true before calling - * {@link ServiceAnnouncer#announce}. This is set to false immediately once {@link Listener#stopBeingLeader()} is - * called. - */ - private volatile boolean initialized; + private final AtomicBoolean isLeader = new AtomicBoolean(false); @Inject public TaskMaster( - final TaskLockConfig taskLockConfig, - final TaskQueueConfig taskQueueConfig, - final DefaultTaskConfig defaultTaskConfig, - final TaskLockbox taskLockbox, - final TaskStorage taskStorage, - final TaskActionClientFactory taskActionClientFactory, - @Self final DruidNode selfNode, - final TaskRunnerFactory runnerFactory, - final ServiceAnnouncer serviceAnnouncer, - final CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, - final ServiceEmitter emitter, - final SupervisorManager supervisorManager, - final OverlordDutyExecutor overlordDutyExecutor, - @IndexingService final DruidLeaderSelector overlordLeaderSelector, - final SegmentAllocationQueue segmentAllocationQueue, - final ObjectMapper mapper, - final TaskContextEnricher taskContextEnricher + TaskActionClientFactory taskActionClientFactory, + SupervisorManager supervisorManager ) { - this.supervisorManager = supervisorManager; this.taskActionClientFactory = taskActionClientFactory; - - this.overlordLeaderSelector = overlordLeaderSelector; - - final DruidNode node = coordinatorOverlordServiceConfig.getOverlordService() == null ? selfNode : - selfNode.withService(coordinatorOverlordServiceConfig.getOverlordService()); - - this.leadershipListener = new DruidLeaderSelector.Listener() - { - @Override - public void becomeLeader() - { - giant.lock(); - - // I AM THE MASTER OF THE UNIVERSE. - log.info("By the power of Grayskull, I have the power!"); - - try { - taskRunner = runnerFactory.build(); - taskQueue = new TaskQueue( - taskLockConfig, - taskQueueConfig, - defaultTaskConfig, - taskStorage, - taskRunner, - taskActionClientFactory, - taskLockbox, - emitter, - mapper, - taskContextEnricher - ); - - // Sensible order to start stuff: - final Lifecycle leaderLifecycle = new Lifecycle("task-master"); - if (leaderLifecycleRef.getAndSet(leaderLifecycle) != null) { - log.makeAlert("TaskMaster set a new Lifecycle without the old one being cleared! Race condition") - .emit(); - } - - leaderLifecycle.addManagedInstance(taskRunner); - leaderLifecycle.addManagedInstance(taskQueue); - leaderLifecycle.addManagedInstance(supervisorManager); - leaderLifecycle.addManagedInstance(overlordDutyExecutor); - leaderLifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() - { - segmentAllocationQueue.becomeLeader(); - } - - @Override - public void stop() - { - segmentAllocationQueue.stopBeingLeader(); - } - } - ); - - leaderLifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() - { - initialized = true; - serviceAnnouncer.announce(node); - } - - @Override - public void stop() - { - serviceAnnouncer.unannounce(node); - } - } - ); - - leaderLifecycle.start(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - finally { - giant.unlock(); - } - } - - @Override - public void stopBeingLeader() - { - giant.lock(); - try { - initialized = false; - final Lifecycle leaderLifecycle = leaderLifecycleRef.getAndSet(null); - - if (leaderLifecycle != null) { - leaderLifecycle.stop(); - } - } - finally { - giant.unlock(); - } - } - }; + this.supervisorManager = supervisorManager; } - /** - * Starts waiting for leadership. Should only be called once throughout the life of the program. - */ - @LifecycleStart - public void start() + public void becomeLeader(TaskRunner taskRunner, TaskQueue taskQueue) { - giant.lock(); - - try { - overlordLeaderSelector.registerListener(leadershipListener); - } - finally { - giant.unlock(); - } + this.taskRunner = taskRunner; + this.taskQueue = taskQueue; + isLeader.set(true); } - /** - * Stops forever (not just this particular leadership session). Should only be called once throughout the life of - * the program. - */ - @LifecycleStop - public void stop() + public void stopBeingLeader() { - giant.lock(); - - try { - gracefulStopLeaderLifecycle(); - overlordLeaderSelector.unregisterListener(); - } - finally { - giant.unlock(); - } + isLeader.set(false); + this.taskQueue = null; + this.taskRunner = null; } - /** - * Returns true if it's the leader and all its services have been initialized. - */ - public boolean isLeader() + private boolean isLeader() { - return overlordLeaderSelector.isLeader() && initialized; - } - - public String getCurrentLeader() - { - return overlordLeaderSelector.getCurrentLeader(); - } - - public Optional getRedirectLocation() - { - String leader = overlordLeaderSelector.getCurrentLeader(); - // do not redirect when - // leader is not elected - // leader is the current node - if (leader == null || leader.isEmpty() || overlordLeaderSelector.isLeader()) { - return Optional.absent(); - } else { - return Optional.of(leader); - } + return isLeader.get(); } public Optional getTaskRunner() @@ -380,18 +187,6 @@ public class TaskMaster implements TaskCountStatsProvider, TaskSlotCountStatsPro } } - private void gracefulStopLeaderLifecycle() - { - try { - if (isLeader()) { - leadershipListener.stopBeingLeader(); - } - } - catch (Exception ex) { - // fail silently since we are stopping anyway - } - } - @Override @Nullable public Map getTotalTaskSlotCount() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java index 29ca16f5aa9..b25bde067c7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java @@ -20,19 +20,38 @@ package org.apache.druid.indexing.overlord; import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; +import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.autoscaling.ProvisioningStrategy; +import org.apache.druid.indexing.overlord.http.TaskStateLookup; +import org.apache.druid.indexing.overlord.http.TotalWorkerCapacityResponse; +import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TaskLookup.TaskLookupType; +import org.joda.time.Duration; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** * Provides read-only methods to fetch information related to tasks. @@ -42,16 +61,28 @@ import java.util.Map; */ public class TaskQueryTool { + private static final Logger log = new Logger(TaskQueryTool.class); + private final TaskStorage storage; private final TaskLockbox taskLockbox; private final TaskMaster taskMaster; + private final JacksonConfigManager configManager; + private final ProvisioningStrategy provisioningStrategy; @Inject - public TaskQueryTool(TaskStorage storage, TaskLockbox taskLockbox, TaskMaster taskMaster) + public TaskQueryTool( + TaskStorage storage, + TaskLockbox taskLockbox, + TaskMaster taskMaster, + ProvisioningStrategy provisioningStrategy, + JacksonConfigManager configManager + ) { this.storage = storage; this.taskLockbox = taskLockbox; this.taskMaster = taskMaster; + this.configManager = configManager; + this.provisioningStrategy = provisioningStrategy; } /** @@ -64,30 +95,20 @@ public class TaskQueryTool } /** - * Gets a List of Intervals locked by higher priority tasks for each datasource. - * - * @param minTaskPriority Minimum task priority for each datasource. Only the - * Intervals that are locked by Tasks with equal or - * higher priority than this are returned. Locked intervals - * for datasources that are not present in this Map are - * not returned. - * @return Map from Datasource to List of Intervals locked by Tasks that have - * priority greater than or equal to the {@code minTaskPriority} for that datasource. + * @param lockFilterPolicies Requests for active locks for various datasources + * @return Map from datasource to conflicting lock infos */ - public Map> getLockedIntervals(Map minTaskPriority) + public Map> getActiveLocks(List lockFilterPolicies) { - return taskLockbox.getLockedIntervals(minTaskPriority); + return taskLockbox.getActiveLocks(lockFilterPolicies); } public List> getActiveTaskInfo(@Nullable String dataSource) { - return storage.getTaskInfos( - TaskLookup.activeTasksOnly(), - dataSource - ); + return storage.getTaskInfos(TaskLookup.activeTasksOnly(), dataSource); } - public List getTaskStatusPlusList( + private List getTaskStatusPlusList( Map taskLookups, @Nullable String dataSource ) @@ -107,9 +128,14 @@ public class TaskQueryTool return storage.getTask(taskId); } - public Optional getStatus(final String taskId) + public Optional getTaskStatus(final String taskId) { - return storage.getStatus(taskId); + final Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + return taskQueue.get().getTaskStatus(taskId); + } else { + return storage.getStatus(taskId); + } } @Nullable @@ -118,4 +144,235 @@ public class TaskQueryTool return storage.getTaskInfo(taskId); } + public List getTaskStatusPlusList( + TaskStateLookup state, + @Nullable String dataSource, + @Nullable String createdTimeInterval, + @Nullable Integer maxCompletedTasks, + @Nullable String type + ) + { + Optional taskRunnerOptional = taskMaster.getTaskRunner(); + if (!taskRunnerOptional.isPresent()) { + return Collections.emptyList(); + } + final TaskRunner taskRunner = taskRunnerOptional.get(); + + final Duration createdTimeDuration; + if (createdTimeInterval != null) { + final Interval theInterval = Intervals.of(StringUtils.replace(createdTimeInterval, "_", "/")); + createdTimeDuration = theInterval.toDuration(); + } else { + createdTimeDuration = null; + } + + // Ideally, snapshotting in taskStorage and taskRunner should be done atomically, + // but there is no way to do it today. + // Instead, we first gets a snapshot from taskStorage and then one from taskRunner. + // This way, we can use the snapshot from taskStorage as the source of truth for the set of tasks to process + // and use the snapshot from taskRunner as a reference for potential task state updates happened + // after the first snapshotting. + Stream taskStatusPlusStream = getTaskStatusPlusList( + state, + dataSource, + createdTimeDuration, + maxCompletedTasks, + type + ); + final Map runnerWorkItems = getTaskRunnerWorkItems( + taskRunner, + state, + dataSource, + type + ); + + if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING) { + // We are interested in only those tasks which are in taskRunner. + taskStatusPlusStream = taskStatusPlusStream + .filter(statusPlus -> runnerWorkItems.containsKey(statusPlus.getId())); + } + final List taskStatusPlusList = taskStatusPlusStream.collect(Collectors.toList()); + + // Separate complete and active tasks from taskStorage. + // Note that taskStorage can return only either complete tasks or active tasks per TaskLookupType. + final List completeTaskStatusPlusList = new ArrayList<>(); + final List activeTaskStatusPlusList = new ArrayList<>(); + for (TaskStatusPlus statusPlus : taskStatusPlusList) { + if (statusPlus.getStatusCode().isComplete()) { + completeTaskStatusPlusList.add(statusPlus); + } else { + activeTaskStatusPlusList.add(statusPlus); + } + } + + final List taskStatuses = new ArrayList<>(completeTaskStatusPlusList); + + activeTaskStatusPlusList.forEach(statusPlus -> { + final TaskRunnerWorkItem runnerWorkItem = runnerWorkItems.get(statusPlus.getId()); + if (runnerWorkItem == null) { + // a task is assumed to be a waiting task if it exists in taskStorage but not in taskRunner. + if (state == TaskStateLookup.WAITING || state == TaskStateLookup.ALL) { + taskStatuses.add(statusPlus); + } + } else { + if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING || state == TaskStateLookup.ALL) { + taskStatuses.add( + new TaskStatusPlus( + statusPlus.getId(), + statusPlus.getGroupId(), + statusPlus.getType(), + statusPlus.getCreatedTime(), + runnerWorkItem.getQueueInsertionTime(), + statusPlus.getStatusCode(), + taskRunner.getRunnerTaskState(statusPlus.getId()), // this is racy for remoteTaskRunner + statusPlus.getDuration(), + runnerWorkItem.getLocation(), // location in taskInfo is only updated after the task is done. + statusPlus.getDataSource(), + statusPlus.getErrorMsg() + ) + ); + } + } + }); + + return taskStatuses; + } + + private Stream getTaskStatusPlusList( + TaskStateLookup state, + @Nullable String dataSource, + Duration createdTimeDuration, + @Nullable Integer maxCompletedTasks, + @Nullable String type + ) + { + final Map taskLookups; + switch (state) { + case ALL: + taskLookups = ImmutableMap.of( + TaskLookupType.ACTIVE, + TaskLookup.ActiveTaskLookup.getInstance(), + TaskLookupType.COMPLETE, + TaskLookup.CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) + ); + break; + case COMPLETE: + taskLookups = ImmutableMap.of( + TaskLookupType.COMPLETE, + TaskLookup.CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) + ); + break; + case WAITING: + case PENDING: + case RUNNING: + taskLookups = ImmutableMap.of( + TaskLookupType.ACTIVE, + TaskLookup.ActiveTaskLookup.getInstance() + ); + break; + default: + throw new IAE("Unknown state: [%s]", state); + } + + final Stream taskStatusPlusStream = getTaskStatusPlusList( + taskLookups, + dataSource + ).stream(); + if (type != null) { + return taskStatusPlusStream.filter( + statusPlus -> type.equals(statusPlus == null ? null : statusPlus.getType()) + ); + } else { + return taskStatusPlusStream; + } + } + + private Map getTaskRunnerWorkItems( + TaskRunner taskRunner, + TaskStateLookup state, + @Nullable String dataSource, + @Nullable String type + ) + { + Stream runnerWorkItemsStream; + switch (state) { + case ALL: + case WAITING: + // waiting tasks can be found by (all tasks in taskStorage - all tasks in taskRunner) + runnerWorkItemsStream = taskRunner.getKnownTasks().stream(); + break; + case PENDING: + runnerWorkItemsStream = taskRunner.getPendingTasks().stream(); + break; + case RUNNING: + runnerWorkItemsStream = taskRunner.getRunningTasks().stream(); + break; + case COMPLETE: + runnerWorkItemsStream = Stream.empty(); + break; + default: + throw new IAE("Unknown state: [%s]", state); + } + if (dataSource != null) { + runnerWorkItemsStream = runnerWorkItemsStream.filter(item -> dataSource.equals(item.getDataSource())); + } + if (type != null) { + runnerWorkItemsStream = runnerWorkItemsStream.filter(item -> type.equals(item.getTaskType())); + } + return runnerWorkItemsStream + .collect(Collectors.toMap(TaskRunnerWorkItem::getTaskId, item -> item)); + } + + public TotalWorkerCapacityResponse getTotalWorkerCapacity() + { + Optional taskRunnerOptional = taskMaster.getTaskRunner(); + if (!taskRunnerOptional.isPresent()) { + return null; + } + TaskRunner taskRunner = taskRunnerOptional.get(); + + Collection workers = taskRunner instanceof WorkerTaskRunner ? + ((WorkerTaskRunner) taskRunner).getWorkers() : ImmutableList.of(); + + int currentCapacity = taskRunner.getTotalCapacity(); + int usedCapacity = taskRunner.getUsedCapacity(); + // Calculate maximum capacity with auto scale + int maximumCapacity; + WorkerBehaviorConfig workerBehaviorConfig = getLatestWorkerConfig(); + if (workerBehaviorConfig == null) { + // Auto scale not setup + log.debug("Cannot calculate maximum worker capacity as worker behavior config is not configured"); + maximumCapacity = -1; + } else if (workerBehaviorConfig instanceof DefaultWorkerBehaviorConfig) { + DefaultWorkerBehaviorConfig defaultWorkerBehaviorConfig = (DefaultWorkerBehaviorConfig) workerBehaviorConfig; + if (defaultWorkerBehaviorConfig.getAutoScaler() == null) { + // Auto scale not setup + log.debug("Cannot calculate maximum worker capacity as auto scaler not configured"); + maximumCapacity = -1; + } else { + int maxWorker = defaultWorkerBehaviorConfig.getAutoScaler().getMaxNumWorkers(); + int expectedWorkerCapacity = provisioningStrategy.getExpectedWorkerCapacity(workers); + maximumCapacity = expectedWorkerCapacity == -1 ? -1 : maxWorker * expectedWorkerCapacity; + } + } else { + // Auto-scale is not using DefaultWorkerBehaviorConfig + log.debug( + "Cannot calculate maximum worker capacity as WorkerBehaviorConfig [%s] of type [%s] does not support getting max capacity", + workerBehaviorConfig, + workerBehaviorConfig.getClass().getSimpleName() + ); + maximumCapacity = -1; + } + + return new TotalWorkerCapacityResponse(currentCapacity, maximumCapacity, usedCapacity); + } + + public WorkerBehaviorConfig getLatestWorkerConfig() + { + return configManager.watch( + WorkerBehaviorConfig.CONFIG_KEY, + WorkerBehaviorConfig.class + ).get(); + } + } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordRedirectInfo.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordRedirectInfo.java index 4e332f599df..41e1ef8b7c9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordRedirectInfo.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordRedirectInfo.java @@ -22,7 +22,7 @@ package org.apache.druid.indexing.overlord.http; import com.google.common.base.Optional; import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; -import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.http.RedirectInfo; @@ -38,25 +38,25 @@ public class OverlordRedirectInfo implements RedirectInfo "/druid/indexer/v1/isLeader" ); - private final TaskMaster taskMaster; + private final DruidOverlord overlord; @Inject - public OverlordRedirectInfo(TaskMaster taskMaster) + public OverlordRedirectInfo(DruidOverlord overlord) { - this.taskMaster = taskMaster; + this.overlord = overlord; } @Override public boolean doLocal(String requestURI) { - return (requestURI != null && LOCAL_PATHS.contains(requestURI)) || taskMaster.isLeader(); + return (requestURI != null && LOCAL_PATHS.contains(requestURI)) || overlord.isLeader(); } @Override public URL getRedirectURL(String queryString, String requestURI) { try { - final Optional redirectLocation = taskMaster.getRedirectLocation(); + final Optional redirectLocation = overlord.getRedirectLocation(); if (!redirectLocation.isPresent()) { return null; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index f945cc95c1e..b62e1de055f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -33,6 +33,7 @@ import org.apache.druid.audit.AuditEntry; import org.apache.druid.audit.AuditManager; import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.common.config.ConfigManager.SetResult; +import org.apache.druid.common.config.Configs; import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.error.DruidException; import org.apache.druid.indexer.RunnerTaskState; @@ -44,31 +45,23 @@ import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionHolder; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.ImmutableWorkerInfo; +import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; -import org.apache.druid.indexing.overlord.TaskQueue; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.WorkerTaskRunner; import org.apache.druid.indexing.overlord.WorkerTaskRunnerQueryAdapter; -import org.apache.druid.indexing.overlord.autoscaling.ProvisioningStrategy; import org.apache.druid.indexing.overlord.autoscaling.ScalingStats; import org.apache.druid.indexing.overlord.http.security.TaskResourceFilter; -import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.LockFilterPolicy; -import org.apache.druid.metadata.TaskLookup; -import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; -import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup; -import org.apache.druid.metadata.TaskLookup.TaskLookupType; import org.apache.druid.server.http.HttpMediaType; import org.apache.druid.server.http.ServletResourceUtils; import org.apache.druid.server.http.security.ConfigResourceFilter; @@ -84,7 +77,7 @@ import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceType; import org.apache.druid.tasklogs.TaskLogStreamer; -import org.joda.time.Duration; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -104,17 +97,12 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; import java.io.InputStream; -import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; -import java.util.stream.Stream; /** * @@ -125,6 +113,7 @@ public class OverlordResource private static final Logger log = new Logger(OverlordResource.class); private final TaskMaster taskMaster; + private final DruidOverlord overlord; private final TaskQueryTool taskQueryTool; private final IndexerMetadataStorageAdapter indexerMetadataStorageAdapter; private final TaskLogStreamer taskLogStreamer; @@ -132,35 +121,16 @@ public class OverlordResource private final AuditManager auditManager; private final AuthorizerMapper authorizerMapper; private final WorkerTaskRunnerQueryAdapter workerTaskRunnerQueryAdapter; - private final ProvisioningStrategy provisioningStrategy; private final AuthConfig authConfig; - private AtomicReference workerConfigRef = null; private static final List API_TASK_STATES = ImmutableList.of("pending", "waiting", "running", "complete"); private static final Set AUDITED_TASK_TYPES = ImmutableSet.of("index", "index_parallel", "compact", "index_hadoop", "kill"); - private enum TaskStateLookup - { - ALL, - WAITING, - PENDING, - RUNNING, - COMPLETE; - - private static TaskStateLookup fromString(@Nullable String state) - { - if (state == null) { - return ALL; - } else { - return TaskStateLookup.valueOf(StringUtils.toUpperCase(state)); - } - } - } - @Inject public OverlordResource( + DruidOverlord overlord, TaskMaster taskMaster, TaskQueryTool taskQueryTool, IndexerMetadataStorageAdapter indexerMetadataStorageAdapter, @@ -169,10 +139,10 @@ public class OverlordResource AuditManager auditManager, AuthorizerMapper authorizerMapper, WorkerTaskRunnerQueryAdapter workerTaskRunnerQueryAdapter, - ProvisioningStrategy provisioningStrategy, AuthConfig authConfig ) { + this.overlord = overlord; this.taskMaster = taskMaster; this.taskQueryTool = taskQueryTool; this.indexerMetadataStorageAdapter = indexerMetadataStorageAdapter; @@ -181,7 +151,6 @@ public class OverlordResource this.auditManager = auditManager; this.authorizerMapper = authorizerMapper; this.workerTaskRunnerQueryAdapter = workerTaskRunnerQueryAdapter; - this.provisioningStrategy = provisioningStrategy; this.authConfig = authConfig; } @@ -252,7 +221,7 @@ public class OverlordResource @Produces(MediaType.APPLICATION_JSON) public Response getLeader() { - return Response.ok(taskMaster.getCurrentLeader()).build(); + return Response.ok(overlord.getCurrentLeader()).build(); } /** @@ -263,7 +232,7 @@ public class OverlordResource @Produces(MediaType.APPLICATION_JSON) public Response isLeader() { - final boolean leading = taskMaster.isLeader(); + final boolean leading = overlord.isLeader(); final Map response = ImmutableMap.of("leader", leading); if (leading) { return Response.ok(response).build(); @@ -272,26 +241,11 @@ public class OverlordResource } } - @Deprecated - @POST - @Path("/lockedIntervals") - @Produces(MediaType.APPLICATION_JSON) - @ResourceFilters(StateResourceFilter.class) - public Response getDatasourceLockedIntervals(Map minTaskPriority) - { - if (minTaskPriority == null || minTaskPriority.isEmpty()) { - return Response.status(Status.BAD_REQUEST).entity("No Datasource provided").build(); - } - - // Build the response - return Response.ok(taskQueryTool.getLockedIntervals(minTaskPriority)).build(); - } - @POST @Path("/lockedIntervals/v2") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(StateResourceFilter.class) - public Response getDatasourceLockedIntervalsV2(List lockFilterPolicies) + public Response getDatasourceLockedIntervals(List lockFilterPolicies) { if (lockFilterPolicies == null || lockFilterPolicies.isEmpty()) { return Response.status(Status.BAD_REQUEST).entity("No filter provided").build(); @@ -301,6 +255,20 @@ public class OverlordResource return Response.ok(taskQueryTool.getLockedIntervals(lockFilterPolicies)).build(); } + @POST + @Path("/activeLocks") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(StateResourceFilter.class) + public Response getActiveLocks(List lockFilterPolicies) + { + if (lockFilterPolicies == null || lockFilterPolicies.isEmpty()) { + return Response.status(Status.BAD_REQUEST).entity("No filter provided").build(); + } + + // Build the response + return Response.ok(new TaskLockResponse(taskQueryTool.getActiveLocks(lockFilterPolicies))).build(); + } + @GET @Path("/task/{taskid}") @Produces(MediaType.APPLICATION_JSON) @@ -373,9 +341,7 @@ public class OverlordResource taskInfo.getStatus().getStatusCode(), RunnerTaskState.WAITING, taskInfo.getStatus().getDuration(), - taskInfo.getStatus().getLocation() == null - ? TaskLocation.unknown() - : taskInfo.getStatus().getLocation(), + Configs.valueOrDefault(taskInfo.getStatus().getLocation(), TaskLocation.unknown()), taskInfo.getDataSource(), taskInfo.getStatus().getErrorMsg() ) @@ -415,14 +381,9 @@ public class OverlordResource { return asLeaderWith( taskMaster.getTaskQueue(), - new Function() - { - @Override - public Response apply(TaskQueue taskQueue) - { - taskQueue.shutdown(taskid, "Shutdown request from user"); - return Response.ok(ImmutableMap.of("task", taskid)).build(); - } + taskQueue -> { + taskQueue.shutdown(taskid, "Shutdown request from user"); + return Response.ok(ImmutableMap.of("task", taskid)).build(); } ); } @@ -435,20 +396,15 @@ public class OverlordResource { return asLeaderWith( taskMaster.getTaskQueue(), - new Function() - { - @Override - public Response apply(TaskQueue taskQueue) - { - final List> tasks = taskQueryTool.getActiveTaskInfo(dataSource); - if (tasks.isEmpty()) { - return Response.status(Status.NOT_FOUND).build(); - } else { - for (final TaskInfo task : tasks) { - taskQueue.shutdown(task.getId(), "Shutdown request from user"); - } - return Response.ok(ImmutableMap.of("dataSource", dataSource)).build(); + taskQueue -> { + final List> tasks = taskQueryTool.getActiveTaskInfo(dataSource); + if (tasks.isEmpty()) { + return Response.status(Status.NOT_FOUND).build(); + } else { + for (final TaskInfo task : tasks) { + taskQueue.shutdown(task.getId(), "Shutdown request from user"); } + return Response.ok(ImmutableMap.of("dataSource", dataSource)).build(); } } ); @@ -460,19 +416,13 @@ public class OverlordResource @ResourceFilters(StateResourceFilter.class) public Response getMultipleTaskStatuses(Set taskIds) { - if (taskIds == null || taskIds.size() == 0) { - return Response.status(Response.Status.BAD_REQUEST).entity("No TaskIds provided.").build(); + if (CollectionUtils.isNullOrEmpty(taskIds)) { + return Response.status(Response.Status.BAD_REQUEST).entity("No Task IDs provided.").build(); } - final Optional taskQueue = taskMaster.getTaskQueue(); - Map result = Maps.newHashMapWithExpectedSize(taskIds.size()); + final Map result = Maps.newHashMapWithExpectedSize(taskIds.size()); for (String taskId : taskIds) { - final Optional optional; - if (taskQueue.isPresent()) { - optional = taskQueue.get().getTaskStatus(taskId); - } else { - optional = taskQueryTool.getStatus(taskId); - } + final Optional optional = taskQueryTool.getTaskStatus(taskId); if (optional.isPresent()) { result.put(taskId, optional.get()); } @@ -487,11 +437,7 @@ public class OverlordResource @ResourceFilters(ConfigResourceFilter.class) public Response getWorkerConfig() { - if (workerConfigRef == null) { - workerConfigRef = configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class); - } - - return Response.ok(workerConfigRef.get()).build(); + return Response.ok(taskQueryTool.getLatestWorkerConfig()).build(); } /** @@ -503,49 +449,11 @@ public class OverlordResource @ResourceFilters(ConfigResourceFilter.class) public Response getTotalWorkerCapacity() { - // Calculate current cluster capacity - Optional taskRunnerOptional = taskMaster.getTaskRunner(); - if (!taskRunnerOptional.isPresent()) { - // Cannot serve call as not leader + if (overlord.isLeader()) { + return Response.ok(taskQueryTool.getTotalWorkerCapacity()).build(); + } else { return Response.status(Response.Status.SERVICE_UNAVAILABLE).build(); } - TaskRunner taskRunner = taskRunnerOptional.get(); - Collection workers = taskRunner instanceof WorkerTaskRunner ? - ((WorkerTaskRunner) taskRunner).getWorkers() : ImmutableList.of(); - - int currentCapacity = taskRunner.getTotalCapacity(); - int usedCapacity = taskRunner.getUsedCapacity(); - // Calculate maximum capacity with auto scale - int maximumCapacity; - if (workerConfigRef == null) { - workerConfigRef = configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class); - } - WorkerBehaviorConfig workerBehaviorConfig = workerConfigRef.get(); - if (workerBehaviorConfig == null) { - // Auto scale not setup - log.debug("Cannot calculate maximum worker capacity as worker behavior config is not configured"); - maximumCapacity = -1; - } else if (workerBehaviorConfig instanceof DefaultWorkerBehaviorConfig) { - DefaultWorkerBehaviorConfig defaultWorkerBehaviorConfig = (DefaultWorkerBehaviorConfig) workerBehaviorConfig; - if (defaultWorkerBehaviorConfig.getAutoScaler() == null) { - // Auto scale not setup - log.debug("Cannot calculate maximum worker capacity as auto scaler not configured"); - maximumCapacity = -1; - } else { - int maxWorker = defaultWorkerBehaviorConfig.getAutoScaler().getMaxNumWorkers(); - int expectedWorkerCapacity = provisioningStrategy.getExpectedWorkerCapacity(workers); - maximumCapacity = expectedWorkerCapacity == -1 ? -1 : maxWorker * expectedWorkerCapacity; - } - } else { - // Auto scale is not using DefaultWorkerBehaviorConfig - log.debug( - "Cannot calculate maximum worker capacity as WorkerBehaviorConfig [%s] of type [%s] does not support getting max capacity", - workerBehaviorConfig, - workerBehaviorConfig.getClass().getSimpleName() - ); - maximumCapacity = -1; - } - return Response.ok(new TotalWorkerCapacityResponse(currentCapacity, maximumCapacity, usedCapacity)).build(); } // default value is used for backwards compatibility @@ -693,9 +601,13 @@ public class OverlordResource //check for valid state if (state != null) { if (!API_TASK_STATES.contains(StringUtils.toLowerCase(state))) { + String errorMessage = StringUtils.format( + "Invalid task state[%s]. Must be one of %s.", + state, API_TASK_STATES + ); return Response.status(Status.BAD_REQUEST) .type(MediaType.TEXT_PLAIN) - .entity(StringUtils.format("Invalid state : %s, valid values are: %s", state, API_TASK_STATES)) + .entity(errorMessage) .build(); } } @@ -725,8 +637,7 @@ public class OverlordResource taskMaster.getTaskRunner(), taskRunner -> { final List authorizedList = securedTaskStatusPlus( - getTaskStatusPlusList( - taskRunner, + taskQueryTool.getTaskStatusPlusList( TaskStateLookup.fromString(state), dataSource, createdTimeInterval, @@ -741,180 +652,6 @@ public class OverlordResource ); } - private List getTaskStatusPlusList( - TaskRunner taskRunner, - TaskStateLookup state, - @Nullable String dataSource, - @Nullable String createdTimeInterval, - @Nullable Integer maxCompletedTasks, - @Nullable String type - ) - { - final Duration createdTimeDuration; - if (createdTimeInterval != null) { - final Interval theInterval = Intervals.of(StringUtils.replace(createdTimeInterval, "_", "/")); - createdTimeDuration = theInterval.toDuration(); - } else { - createdTimeDuration = null; - } - - // Ideally, snapshotting in taskStorage and taskRunner should be done atomically, - // but there is no way to do it today. - // Instead, we first gets a snapshot from taskStorage and then one from taskRunner. - // This way, we can use the snapshot from taskStorage as the source of truth for the set of tasks to process - // and use the snapshot from taskRunner as a reference for potential task state updates happened - // after the first snapshotting. - Stream taskStatusPlusStream = getTaskStatusPlusList( - state, - dataSource, - createdTimeDuration, - maxCompletedTasks, - type - ); - final Map runnerWorkItems = getTaskRunnerWorkItems( - taskRunner, - state, - dataSource, - type - ); - - if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING) { - // We are interested in only those tasks which are in taskRunner. - taskStatusPlusStream = taskStatusPlusStream - .filter(statusPlus -> runnerWorkItems.containsKey(statusPlus.getId())); - } - final List taskStatusPlusList = taskStatusPlusStream.collect(Collectors.toList()); - - // Separate complete and active tasks from taskStorage. - // Note that taskStorage can return only either complete tasks or active tasks per TaskLookupType. - final List completeTaskStatusPlusList = new ArrayList<>(); - final List activeTaskStatusPlusList = new ArrayList<>(); - for (TaskStatusPlus statusPlus : taskStatusPlusList) { - if (statusPlus.getStatusCode().isComplete()) { - completeTaskStatusPlusList.add(statusPlus); - } else { - activeTaskStatusPlusList.add(statusPlus); - } - } - - final List taskStatuses = new ArrayList<>(completeTaskStatusPlusList); - - activeTaskStatusPlusList.forEach(statusPlus -> { - final TaskRunnerWorkItem runnerWorkItem = runnerWorkItems.get(statusPlus.getId()); - if (runnerWorkItem == null) { - // a task is assumed to be a waiting task if it exists in taskStorage but not in taskRunner. - if (state == TaskStateLookup.WAITING || state == TaskStateLookup.ALL) { - taskStatuses.add(statusPlus); - } - } else { - if (state == TaskStateLookup.PENDING || state == TaskStateLookup.RUNNING || state == TaskStateLookup.ALL) { - taskStatuses.add( - new TaskStatusPlus( - statusPlus.getId(), - statusPlus.getGroupId(), - statusPlus.getType(), - statusPlus.getCreatedTime(), - runnerWorkItem.getQueueInsertionTime(), - statusPlus.getStatusCode(), - taskRunner.getRunnerTaskState(statusPlus.getId()), // this is racy for remoteTaskRunner - statusPlus.getDuration(), - runnerWorkItem.getLocation(), // location in taskInfo is only updated after the task is done. - statusPlus.getDataSource(), - statusPlus.getErrorMsg() - ) - ); - } - } - }); - - return taskStatuses; - } - - private Stream getTaskStatusPlusList( - TaskStateLookup state, - @Nullable String dataSource, - Duration createdTimeDuration, - @Nullable Integer maxCompletedTasks, - @Nullable String type - ) - { - final Map taskLookups; - switch (state) { - case ALL: - taskLookups = ImmutableMap.of( - TaskLookupType.ACTIVE, - ActiveTaskLookup.getInstance(), - TaskLookupType.COMPLETE, - CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) - ); - break; - case COMPLETE: - taskLookups = ImmutableMap.of( - TaskLookupType.COMPLETE, - CompleteTaskLookup.of(maxCompletedTasks, createdTimeDuration) - ); - break; - case WAITING: - case PENDING: - case RUNNING: - taskLookups = ImmutableMap.of( - TaskLookupType.ACTIVE, - ActiveTaskLookup.getInstance() - ); - break; - default: - throw new IAE("Unknown state: [%s]", state); - } - - final Stream taskStatusPlusStream = taskQueryTool.getTaskStatusPlusList( - taskLookups, - dataSource - ).stream(); - if (type != null) { - return taskStatusPlusStream.filter( - statusPlus -> type.equals(statusPlus == null ? null : statusPlus.getType()) - ); - } else { - return taskStatusPlusStream; - } - } - - private Map getTaskRunnerWorkItems( - TaskRunner taskRunner, - TaskStateLookup state, - @Nullable String dataSource, - @Nullable String type - ) - { - Stream runnerWorkItemsStream; - switch (state) { - case ALL: - case WAITING: - // waiting tasks can be found by (all tasks in taskStorage - all tasks in taskRunner) - runnerWorkItemsStream = taskRunner.getKnownTasks().stream(); - break; - case PENDING: - runnerWorkItemsStream = taskRunner.getPendingTasks().stream(); - break; - case RUNNING: - runnerWorkItemsStream = taskRunner.getRunningTasks().stream(); - break; - case COMPLETE: - runnerWorkItemsStream = Stream.empty(); - break; - default: - throw new IAE("Unknown state: [%s]", state); - } - if (dataSource != null) { - runnerWorkItemsStream = runnerWorkItemsStream.filter(item -> dataSource.equals(item.getDataSource())); - } - if (type != null) { - runnerWorkItemsStream = runnerWorkItemsStream.filter(item -> type.equals(item.getTaskType())); - } - return runnerWorkItemsStream - .collect(Collectors.toMap(TaskRunnerWorkItem::getTaskId, item -> item)); - } - @DELETE @Path("/pendingSegments/{dataSource}") @Produces(MediaType.APPLICATION_JSON) @@ -939,7 +676,7 @@ public class OverlordResource throw new ForbiddenException(authResult.getMessage()); } - if (taskMaster.isLeader()) { + if (overlord.isLeader()) { try { final int numDeleted = indexerMetadataStorageAdapter.deletePendingSegments(dataSource, deleteInterval); return Response.ok().entity(ImmutableMap.of("numDeleted", numDeleted)).build(); @@ -970,23 +707,17 @@ public class OverlordResource { return asLeaderWith( taskMaster.getTaskRunner(), - new Function() - { - @Override - public Response apply(TaskRunner taskRunner) - { - if (taskRunner instanceof WorkerTaskRunner) { - return Response.ok(((WorkerTaskRunner) taskRunner).getWorkers()).build(); - } else { - log.debug( - "Task runner [%s] of type [%s] does not support listing workers", - taskRunner, - taskRunner.getClass().getName() - ); - return Response.serverError() - .entity(ImmutableMap.of("error", "Task Runner does not support worker listing")) - .build(); - } + taskRunner -> { + if (taskRunner instanceof WorkerTaskRunner) { + return Response.ok(((WorkerTaskRunner) taskRunner).getWorkers()).build(); + } else { + log.debug( + "Task runner[%s] of type[%s] does not support listing workers", + taskRunner, taskRunner.getClass().getName() + ); + return Response.serverError() + .entity(ImmutableMap.of("error", "Task Runner does not support worker listing")) + .build(); } } ); diff --git a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskLockResponse.java similarity index 51% rename from indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java rename to indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskLockResponse.java index 8e495221099..df4a5d8b03c 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskLockResponse.java @@ -17,33 +17,38 @@ * under the License. */ -package org.apache.druid.guice; +package org.apache.druid.indexing.overlord.http; -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.common.collect.ImmutableList; -import com.google.inject.Binder; -import org.apache.druid.initialization.DruidModule; -import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.common.TaskLock; import java.util.List; +import java.util.Map; -public class IndexingServiceFirehoseModule implements DruidModule +public class TaskLockResponse { - @Override - public List getJacksonModules() + private final Map> datasourceToLocks; + + @JsonCreator + public TaskLockResponse( + @JsonProperty("datasourceToLocks") final Map> datasourceToLocks + ) { - return ImmutableList.of( - new SimpleModule("IndexingServiceFirehoseModule") - .registerSubtypes( - new NamedType(EventReceiverFirehoseFactory.class, "receiver") - ) - ); + this.datasourceToLocks = datasourceToLocks; + } + + @JsonProperty + public Map> getDatasourceToLocks() + { + return datasourceToLocks; } @Override - public void configure(Binder binder) + public String toString() { + return "TaskLockResponse{" + + "datasourceToLocks='" + datasourceToLocks + + '}'; } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiver.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskStateLookup.java similarity index 65% rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiver.java rename to indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskStateLookup.java index e2b776fa4fa..0236afdd057 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiver.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskStateLookup.java @@ -17,12 +17,26 @@ * under the License. */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.indexing.overlord.http; -import java.util.Collection; -import java.util.Map; +import org.apache.druid.java.util.common.StringUtils; -public interface EventReceiver +import javax.annotation.Nullable; + +public enum TaskStateLookup { - void addAll(Collection> events); + ALL, + WAITING, + PENDING, + RUNNING, + COMPLETE; + + public static TaskStateLookup fromString(@Nullable String state) + { + if (state == null) { + return ALL; + } else { + return TaskStateLookup.valueOf(StringUtils.toUpperCase(state)); + } + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java index a84cb4e1979..5018795cbb4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java @@ -76,8 +76,8 @@ public class InputSourceSampler null ); - // We want to be able to sort the list of processed results back into the same order that we read them from the - // firehose so that the rows in the data loader are not always changing. To do this, we add a temporary column to the + // We want to be able to sort the list of processed results back into the same order that we read them from the input + // source so that the rows in the data loader are not always changing. To do this, we add a temporary column to the // InputRow (in SamplerInputRow) and tag each row with a sortKey. We use an aggregator so that it will not affect // rollup, and we use a longMin aggregator so that as rows get rolled up, the earlier rows stay stable and later // rows may get rolled into these rows. After getting the results back from the IncrementalIndex, we sort by this diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index ee4eed572df..41cd084cd96 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -49,10 +49,10 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; -import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.partition.NumberedPartialShardSpec; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index fad7b923ca9..d347fd81503 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -81,13 +81,13 @@ import org.apache.druid.metadata.PendingSegmentRecord; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; -import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizerMapper; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index e1ac9482436..78cbb88c3cf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -51,9 +51,9 @@ import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; @@ -114,7 +114,6 @@ public class TaskToolboxTest TaskConfig taskConfig = new TaskConfigBuilder() .setBaseDir(temporaryFolder.newFile().toString()) .setDefaultRowFlushBoundary(50000) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); taskToolbox = new TaskToolboxFactory( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java deleted file mode 100644 index 1572373151b..00000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java +++ /dev/null @@ -1,118 +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.indexing.common; - -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.java.util.common.parsers.ParseException; - -import java.io.File; -import java.util.ArrayDeque; -import java.util.Collections; -import java.util.Deque; -import java.util.List; -import java.util.Optional; - -public class TestFirehose implements Firehose -{ - public static class TestFirehoseFactory implements FirehoseFactory - { - private boolean waitForClose = true; - - @Override - @SuppressWarnings("unchecked") - public Firehose connect(InputRowParser parser, File temporaryDirectory) throws ParseException - { - return new TestFirehose(parser, waitForClose, Collections.emptyList()); - } - } - - public static final String FAIL_DIM = "__fail__"; - - private final Deque> queue = new ArrayDeque<>(); - - private InputRowParser parser; - private boolean closed; - - private TestFirehose(InputRowParser parser, boolean waitForClose, List seedRows) - { - this.parser = parser; - this.closed = !waitForClose; - - if (parser instanceof StringInputRowParser) { - ((StringInputRowParser) parser).startFileFromBeginning(); - } - - if (seedRows != null) { - seedRows.stream().map(Optional::ofNullable).forEach(queue::add); - } - } - - public void addRows(List rows) - { - synchronized (this) { - rows.stream().map(Optional::ofNullable).forEach(queue::add); - notifyAll(); - } - } - - @Override - public boolean hasMore() - { - try { - synchronized (this) { - while (queue.isEmpty() && !closed) { - wait(); - } - return !queue.isEmpty(); - } - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - - @Override - public InputRow nextRow() - { - synchronized (this) { - final InputRow row = parser instanceof StringInputRowParser - ? ((StringInputRowParser) parser).parse((String) queue.removeFirst().orElse(null)) - : (InputRow) parser.parseBatch(queue.removeFirst().orElse(null)).get(0); - if (row != null && row.getRaw(FAIL_DIM) != null) { - throw new ParseException(null, FAIL_DIM); - } - return row; - } - } - - @Override - public void close() - { - synchronized (this) { - closed = true; - notifyAll(); - } - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java index c69ac562821..f6732f68a6c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java @@ -64,7 +64,6 @@ public class TestIndexTask extends IndexTask new IndexIngestionSpec( new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper), new IndexTask.IndexIOConfig( - null, new LocalInputSource(new File("lol"), "rofl"), new JsonInputFormat(null, null, null, null, null), false, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java index 7dda3b8ff62..ae5ba44f7f0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.common; import com.fasterxml.jackson.databind.InjectableValues; -import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; @@ -30,7 +29,6 @@ import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.NoopInputSource; import org.apache.druid.guice.DruidSecondaryModule; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; @@ -49,15 +47,14 @@ import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; -import java.util.List; import java.util.concurrent.TimeUnit; /** @@ -122,9 +119,6 @@ public class TestUtils } ); DruidSecondaryModule.setupAnnotationIntrospector(jsonMapper, TestHelper.makeAnnotationIntrospector()); - - List firehoseModules = new FirehoseModule().getJacksonModules(); - firehoseModules.forEach(jsonMapper::registerModule); } public ObjectMapper getTestObjectMapper() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java index af920ebbeb7..1213b552514 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java @@ -36,8 +36,6 @@ public class TaskConfigBuilder private Period directoryLockTimeout; private List shuffleDataLocations; private boolean ignoreTimestampSpecForDruidInputSource; - private boolean batchMemoryMappedIndex; // deprecated; only set to true to fall back to older behavior - private String batchProcessingMode; private Boolean storeEmptyColumns; private boolean enableTaskLevelLogPush; private Long tmpStorageBytesPerTask; @@ -102,18 +100,6 @@ public class TaskConfigBuilder return this; } - public TaskConfigBuilder setBatchMemoryMappedIndex(boolean batchMemoryMappedIndex) - { - this.batchMemoryMappedIndex = batchMemoryMappedIndex; - return this; - } - - public TaskConfigBuilder setBatchProcessingMode(String batchProcessingMode) - { - this.batchProcessingMode = batchProcessingMode; - return this; - } - public TaskConfigBuilder setStoreEmptyColumns(Boolean storeEmptyColumns) { this.storeEmptyColumns = storeEmptyColumns; @@ -145,8 +131,6 @@ public class TaskConfigBuilder directoryLockTimeout, shuffleDataLocations, ignoreTimestampSpecForDruidInputSource, - batchMemoryMappedIndex, - batchProcessingMode, storeEmptyColumns, enableTaskLevelLogPush, tmpStorageBytesPerTask diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java deleted file mode 100644 index f789c1e641f..00000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java +++ /dev/null @@ -1,604 +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.indexing.common.task; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.config.TaskConfigBuilder; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.core.NoopEmitter; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMerger; -import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.incremental.AppendableIndexSpec; -import org.apache.druid.segment.incremental.ParseExceptionHandler; -import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.join.NoopJoinableFactory; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.SegmentGenerationMetrics; -import org.apache.druid.segment.realtime.appenderator.Appenderator; -import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; -import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; -import org.apache.druid.segment.realtime.appenderator.BatchAppenderator; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; -import org.apache.druid.server.security.AuthTestUtils; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.LinearShardSpec; -import org.joda.time.Period; -import org.junit.Assert; -import org.junit.Test; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.CopyOnWriteArrayList; - - -public class BatchAppenderatorsTest -{ - @Test - public void testLegacyOfflineAppenderator() throws Exception - { - try (final AppenderatorTester tester = new AppenderatorTester("OPEN_SEGMENTS")) { - Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); - AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; - Assert.assertTrue(appenderator.isOpenSegments()); - } - } - - @Test - public void testClosedSegmentsOfflineAppenderator() throws Exception - { - try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS")) { - Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); - AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; - Assert.assertFalse(appenderator.isOpenSegments()); - } - } - - @Test - public void testClosedSegmentsSinksOfflineAppenderator() throws Exception - { - try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS_SINKS")) { - Assert.assertTrue(tester.appenderator instanceof BatchAppenderator); - } - } - - - private static class AppenderatorTester implements AutoCloseable - { - public static final String DATASOURCE = "foo"; - - private final DataSchema schema; - private final AppenderatorConfig tuningConfig; - private final SegmentGenerationMetrics metrics; - private final ObjectMapper objectMapper; - private final Appenderator appenderator; - private final ServiceEmitter emitter; - - private final List pushedSegments = new CopyOnWriteArrayList<>(); - - - public AppenderatorTester( - final String batchMode - ) - { - this(100, 100, null, false, new SimpleRowIngestionMeters(), - false, batchMode - ); - } - - public AppenderatorTester( - final int maxRowsInMemory, - final long maxSizeInBytes, - @Nullable final File basePersistDirectory, - final boolean enablePushFailure, - final RowIngestionMeters rowIngestionMeters, - final boolean skipBytesInMemoryOverheadCheck, - String batchMode - ) - { - objectMapper = new DefaultObjectMapper(); - objectMapper.registerSubtypes(LinearShardSpec.class); - - final Map parserMap = objectMapper.convertValue( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec("ts", "auto", null), - DimensionsSpec.EMPTY, - null, - null, - null - ) - ), - Map.class - ); - - schema = new DataSchema( - DATASOURCE, - null, - null, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null, - parserMap, - objectMapper - ); - - tuningConfig = new TestIndexTuningConfig( - TuningConfig.DEFAULT_APPENDABLE_INDEX, - maxRowsInMemory, - maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, - skipBytesInMemoryOverheadCheck, - IndexSpec.DEFAULT, - 0, - false, - 0L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, - basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory, - null - ); - metrics = new SegmentGenerationMetrics(); - - IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); - IndexMergerV9 indexMerger = new IndexMergerV9( - objectMapper, - indexIO, - OffHeapMemorySegmentWriteOutMediumFactory.instance() - ); - - emitter = new ServiceEmitter( - "test", - "test", - new NoopEmitter() - ); - emitter.start(); - EmittingLogger.registerEmitter(emitter); - DataSegmentPusher dataSegmentPusher = new DataSegmentPusher() - { - private boolean mustFail = true; - - @Deprecated - @Override - public String getPathForHadoop(String dataSource) - { - return getPathForHadoop(); - } - - @Override - public String getPathForHadoop() - { - throw new UnsupportedOperationException(); - } - - @Override - public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException - { - if (enablePushFailure && mustFail) { - mustFail = false; - throw new IOException("Push failure test"); - } else if (enablePushFailure) { - mustFail = true; - } - pushedSegments.add(segment); - return segment; - } - - @Override - public Map makeLoadSpec(URI uri) - { - throw new UnsupportedOperationException(); - } - }; - - switch (batchMode) { - case "OPEN_SEGMENTS": - appenderator = BatchAppenderators.newAppenderator( - "foo", - new TestAppenderatorsManager(), - metrics, - makeTaskToolbox( - objectMapper, - indexMerger, - TaskConfig.BatchProcessingMode.OPEN_SEGMENTS - ), - schema, - tuningConfig, - dataSegmentPusher, - rowIngestionMeters, - new ParseExceptionHandler( - rowIngestionMeters, - false, - Integer.MAX_VALUE, - 0 - ), - false - ); - break; - case "CLOSED_SEGMENTS": - appenderator = BatchAppenderators.newAppenderator( - "foo", - new TestAppenderatorsManager(), - metrics, - makeTaskToolbox( - objectMapper, - indexMerger, - TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS - ), - schema, - tuningConfig, - dataSegmentPusher, - rowIngestionMeters, - new ParseExceptionHandler( - rowIngestionMeters, - false, - Integer.MAX_VALUE, - 0 - ), - false - ); - - break; - case "CLOSED_SEGMENTS_SINKS": - appenderator = BatchAppenderators.newAppenderator( - "foo", - new TestAppenderatorsManager(), - metrics, - makeTaskToolbox( - objectMapper, - indexMerger, - TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS_SINKS - ), - schema, - tuningConfig, - dataSegmentPusher, - rowIngestionMeters, - new ParseExceptionHandler( - rowIngestionMeters, - false, - Integer.MAX_VALUE, - 0 - ), - false - ); - break; - default: - throw new IllegalArgumentException("Unrecognized batchMode: " + batchMode); - } - } - - private long getDefaultMaxBytesInMemory() - { - return (Runtime.getRuntime().totalMemory()) / 3; - } - - public DataSchema getSchema() - { - return schema; - } - - public AppenderatorConfig getTuningConfig() - { - return tuningConfig; - } - - public SegmentGenerationMetrics getMetrics() - { - return metrics; - } - - public ObjectMapper getObjectMapper() - { - return objectMapper; - } - - public Appenderator getAppenderator() - { - return appenderator; - } - - public List getPushedSegments() - { - return pushedSegments; - } - - @Override - public void close() throws Exception - { - appenderator.close(); - emitter.close(); - FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory()); - } - - private static File createNewBasePersistDirectory() - { - return FileUtils.createTempDir("druid-batch-persist"); - } - - - static class TestIndexTuningConfig implements AppenderatorConfig - { - private final AppendableIndexSpec appendableIndexSpec; - private final int maxRowsInMemory; - private final long maxBytesInMemory; - private final boolean skipBytesInMemoryOverheadCheck; - private final int maxColumnsToMerge; - private final PartitionsSpec partitionsSpec; - private final IndexSpec indexSpec; - private final File basePersistDirectory; - private final int maxPendingPersists; - private final boolean reportParseExceptions; - private final long pushTimeout; - private final IndexSpec indexSpecForIntermediatePersists; - @Nullable - private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - private final int numPersistThreads; - - public TestIndexTuningConfig( - AppendableIndexSpec appendableIndexSpec, - Integer maxRowsInMemory, - Long maxBytesInMemory, - Boolean skipBytesInMemoryOverheadCheck, - IndexSpec indexSpec, - Integer maxPendingPersists, - Boolean reportParseExceptions, - Long pushTimeout, - @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - Integer maxColumnsToMerge, - File basePersistDirectory, - Integer numPersistThreads - ) - { - this.appendableIndexSpec = appendableIndexSpec; - this.maxRowsInMemory = maxRowsInMemory; - this.maxBytesInMemory = maxBytesInMemory; - this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck; - this.indexSpec = indexSpec; - this.maxPendingPersists = maxPendingPersists; - this.reportParseExceptions = reportParseExceptions; - this.pushTimeout = pushTimeout; - this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; - this.maxColumnsToMerge = maxColumnsToMerge; - this.basePersistDirectory = basePersistDirectory; - - this.partitionsSpec = null; - this.indexSpecForIntermediatePersists = this.indexSpec; - - this.numPersistThreads = numPersistThreads == null ? DEFAULT_NUM_PERSIST_THREADS : numPersistThreads; - } - - @Override - public TestIndexTuningConfig withBasePersistDirectory(File dir) - { - return this; - } - - @Override - public AppendableIndexSpec getAppendableIndexSpec() - { - return appendableIndexSpec; - } - - @Override - public int getMaxRowsInMemory() - { - return maxRowsInMemory; - } - - @Override - public long getMaxBytesInMemory() - { - return maxBytesInMemory; - } - - @Override - public boolean isSkipBytesInMemoryOverheadCheck() - { - return skipBytesInMemoryOverheadCheck; - } - - @Nullable - @Override - public PartitionsSpec getPartitionsSpec() - { - return partitionsSpec; - } - - @Override - public IndexSpec getIndexSpec() - { - return indexSpec; - } - - @Override - public IndexSpec getIndexSpecForIntermediatePersists() - { - return indexSpecForIntermediatePersists; - } - - @Override - public int getMaxPendingPersists() - { - return maxPendingPersists; - } - - @Override - public boolean isReportParseExceptions() - { - return reportParseExceptions; - } - - @Nullable - @Override - public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() - { - return segmentWriteOutMediumFactory; - } - - @Override - public int getMaxColumnsToMerge() - { - return maxColumnsToMerge; - } - - @Override - public File getBasePersistDirectory() - { - return basePersistDirectory; - } - - @Override - public Period getIntermediatePersistPeriod() - { - return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs - } - - @Override - public int getNumPersistThreads() - { - return numPersistThreads; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - TestIndexTuningConfig that = (TestIndexTuningConfig) o; - return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) && - maxRowsInMemory == that.maxRowsInMemory && - maxBytesInMemory == that.maxBytesInMemory && - skipBytesInMemoryOverheadCheck == that.skipBytesInMemoryOverheadCheck && - maxColumnsToMerge == that.maxColumnsToMerge && - maxPendingPersists == that.maxPendingPersists && - reportParseExceptions == that.reportParseExceptions && - pushTimeout == that.pushTimeout && - numPersistThreads == that.numPersistThreads && - Objects.equals(partitionsSpec, that.partitionsSpec) && - Objects.equals(indexSpec, that.indexSpec) && - Objects.equals(indexSpecForIntermediatePersists, that.indexSpecForIntermediatePersists) && - Objects.equals(basePersistDirectory, that.basePersistDirectory) && - Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); - } - - @Override - public int hashCode() - { - return Objects.hash( - appendableIndexSpec, - maxRowsInMemory, - maxBytesInMemory, - skipBytesInMemoryOverheadCheck, - maxColumnsToMerge, - partitionsSpec, - indexSpec, - indexSpecForIntermediatePersists, - basePersistDirectory, - maxPendingPersists, - reportParseExceptions, - pushTimeout, - segmentWriteOutMediumFactory, - numPersistThreads - ); - } - - @Override - public String toString() - { - return "IndexTuningConfig{" + - "maxRowsInMemory=" + maxRowsInMemory + - ", maxBytesInMemory=" + maxBytesInMemory + - ", skipBytesInMemoryOverheadCheck=" + skipBytesInMemoryOverheadCheck + - ", maxColumnsToMerge=" + maxColumnsToMerge + - ", partitionsSpec=" + partitionsSpec + - ", indexSpec=" + indexSpec + - ", indexSpecForIntermediatePersists=" + indexSpecForIntermediatePersists + - ", basePersistDirectory=" + basePersistDirectory + - ", maxPendingPersists=" + maxPendingPersists + - ", reportParseExceptions=" + reportParseExceptions + - ", pushTimeout=" + pushTimeout + - ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + - ", numPersistThreads=" + numPersistThreads + - '}'; - } - } - - private static TaskToolbox makeTaskToolbox( - ObjectMapper mapper, - IndexMergerV9 indexMergerV9, - TaskConfig.BatchProcessingMode mode - ) - { - TaskConfig config = new TaskConfigBuilder() - .setBatchProcessingMode(mode.name()) - .build(); - return new TaskToolbox.Builder() - .config(config) - .joinableFactory(NoopJoinableFactory.INSTANCE) - .jsonMapper(mapper) - .indexIO(new IndexIO(new ObjectMapper(), ColumnConfig.DEFAULT)) - .indexMergerV9(indexMergerV9) - .taskReportFileWriter(new NoopTestTaskReportFileWriter()) - .authorizerMapper(AuthTestUtils.TEST_AUTHORIZER_MAPPER) - .chatHandlerProvider(new NoopChatHandlerProvider()) - .appenderatorsManager(new TestAppenderatorsManager()) - .taskLogPusher(null) - .attemptId("1") - .centralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig.create()) - .build(); - - } - } -} - diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index cd942a9a766..55898357d66 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -60,9 +60,9 @@ import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.AuthTestUtils; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index 188ea3cdd07..1b742971eb9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -49,8 +49,8 @@ import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIOConfi import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIngestionSpec; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; -import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.indexing.input.WindowedSegmentId; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -922,7 +922,6 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis private void runIndexTask(@Nullable PartitionsSpec partitionsSpec, boolean appendToExisting) { ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDir, "druid*"), new CsvInputFormat( Arrays.asList("ts", "dim", "val"), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 0c274a01966..7ebde700bee 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -94,8 +94,8 @@ import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.loading.TombstoneLoadSpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.WindowedStorageAdapter; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; @@ -2030,7 +2030,6 @@ public class CompactionTaskRunTest extends IngestionTestBase ); final TaskConfig config = new TaskConfigBuilder() - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = CentralizedDatasourceSchemaConfig.create(); centralizedDatasourceSchemaConfig.setEnabled(true); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 25dc84d98b7..3a386bc4aa7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -123,9 +123,9 @@ import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.selector.settable.SettableColumnValueSelector; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; @@ -749,7 +749,8 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -810,7 +811,8 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -872,7 +874,8 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -935,7 +938,8 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1005,7 +1009,8 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1055,7 +1060,8 @@ public class CompactionTaskTest null, customMetricsSpec, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1098,7 +1104,8 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1148,7 +1155,8 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); NativeCompactionRunner.createIngestionSpecs( @@ -1178,7 +1186,8 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); NativeCompactionRunner.createIngestionSpecs( @@ -1219,7 +1228,8 @@ public class CompactionTaskTest null, null, new ClientCompactionTaskGranularitySpec(new PeriodGranularity(Period.months(3), null, null), null, null), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1263,7 +1273,8 @@ public class CompactionTaskTest null, null, new ClientCompactionTaskGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null), null), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, @@ -1308,7 +1319,8 @@ public class CompactionTaskTest new PeriodGranularity(Period.months(3), null, null), null ), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1355,7 +1367,8 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1400,7 +1413,8 @@ public class CompactionTaskTest null, null, new ClientCompactionTaskGranularitySpec(null, null, null), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1445,7 +1459,8 @@ public class CompactionTaskTest null, null, new ClientCompactionTaskGranularitySpec(null, null, true), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1475,7 +1490,8 @@ public class CompactionTaskTest null, null, new ClientCompactionTaskGranularitySpec(null, null, null), - METRIC_BUILDER + METRIC_BUILDER, + null ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1769,7 +1785,6 @@ public class CompactionTaskTest }; final TaskConfig config = new TaskConfigBuilder() - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); return new TaskToolbox.Builder() .config(config) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java index ceca135c810..130258e85d4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java @@ -111,7 +111,6 @@ public class HadoopTaskTest new TaskConfigBuilder() .setBaseDir(temporaryFolder.newFolder().toString()) .setDefaultHadoopCoordinates(ImmutableList.of("something:hadoop:1")) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build() ).once(); EasyMock.replay(toolbox); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java index 7555c6b1e37..ab953ba954a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java @@ -54,7 +54,6 @@ public class IndexIngestionSpecTest null ), new IndexIOConfig( - null, new NoopInputSource(), new NoopInputFormat(), null, @@ -79,7 +78,6 @@ public class IndexIngestionSpecTest null ), new IndexIOConfig( - null, new NoopInputSource(), null, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 40be3844585..3ec5b8d0aa8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -93,7 +93,7 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.segment.loading.StorageLocationConfig; -import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedStorageAdapter; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -244,7 +244,6 @@ public class IndexTaskTest extends IngestionTestBase null ), new IndexIOConfig( - null, new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, false, @@ -295,7 +294,6 @@ public class IndexTaskTest extends IngestionTestBase null ), new IndexIOConfig( - null, new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, false, @@ -358,7 +356,6 @@ public class IndexTaskTest extends IngestionTestBase null ), new IndexIOConfig( - null, new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, false, @@ -2727,7 +2724,6 @@ public class IndexTaskTest extends IngestionTestBase transformSpec ), new IndexIOConfig( - null, new LocalInputSource(baseDir, "druid*"), inputFormat, appendToExisting, @@ -2755,7 +2751,6 @@ public class IndexTaskTest extends IngestionTestBase objectMapper ), new IndexIOConfig( - null, new LocalInputSource(baseDir, "druid*"), createInputFormatFromParseSpec(parseSpec), appendToExisting, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index ca4d7ee7fca..67a0c518f57 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -23,13 +23,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DelimitedInputFormat; import org.apache.druid.data.input.impl.DelimitedParseSpec; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.ParseSpec; @@ -85,7 +83,7 @@ import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.AuthTestUtils; @@ -300,8 +298,7 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest } /** - * Converts ParseSpec to InputFormat for indexing tests. To be used until {@link FirehoseFactory} - * & {@link InputRowParser} is deprecated and removed. + * Converts ParseSpec to InputFormat for indexing tests. Used for backwards compatibility */ public static InputFormat createInputFormatFromParseSpec(ParseSpec parseSpec) { @@ -446,7 +443,6 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest ); final TaskConfig config = new TaskConfigBuilder() - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); centralizedDatasourceSchemaConfig.setEnabled(true); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index 9cf6c0102fb..e6ea0e1329a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -19,19 +19,14 @@ package org.apache.druid.indexing.common.task; -import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIngestionSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; @@ -70,14 +65,9 @@ public class TaskSerdeTest { TestUtils testUtils = new TestUtils(); jsonMapper = testUtils.getTestObjectMapper(); - - for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { - jsonMapper.registerModule(jacksonModule); - } jsonMapper.registerSubtypes( new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), - new NamedType(IndexTuningConfig.class, "index"), - new NamedType(MockFirehoseFactory.class, "mock") + new NamedType(IndexTuningConfig.class, "index") ); } @@ -242,7 +232,7 @@ public class TaskSerdeTest ), null ), - new IndexIOConfig(null, new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), + new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), TuningConfigBuilder.forIndexTask() .withMaxRowsInMemory(10) .withPartitionsSpec(new DynamicPartitionsSpec(10000, null)) @@ -310,7 +300,7 @@ public class TaskSerdeTest ), null ), - new IndexIOConfig(null, new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), + new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), TuningConfigBuilder.forIndexTask() .withMaxRowsInMemory(10) .withForceGuaranteedRollup(false) @@ -322,10 +312,6 @@ public class TaskSerdeTest null ); - for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { - jsonMapper.registerModule(jacksonModule); - } - final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -462,15 +448,6 @@ public class TaskSerdeTest Assert.assertEquals("blah", task2.getClasspathPrefix()); } - private static class MockFirehoseFactory implements FirehoseFactory - { - @Override - public Firehose connect(InputRowParser parser, File temporaryDirectory) - { - return null; - } - } - @Test public void testHadoopIndexTaskWithContextSerde() throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java index e6186329126..417195326fb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java @@ -101,7 +101,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager } @Override - public Appenderator createOpenSegmentsOfflineAppenderatorForTask( + public Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, @@ -116,71 +116,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { - return Appenderators.createOpenSegmentsOffline( - taskId, - schema, - config, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - } - - @Override - public Appenderator createClosedSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - return Appenderators.createClosedSegmentsOffline( - taskId, - schema, - config, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - } - - @Override - public Appenderator createOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - return Appenderators.createOffline( + return Appenderators.createBatch( taskId, schema, config, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index be322ad06d3..44b8284f407 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -74,10 +74,7 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn protected static final Granularity SEGMENT_GRANULARITY = Granularities.DAY; private static final ScanQueryRunnerFactory SCAN_QUERY_RUNNER_FACTORY = new ScanQueryRunnerFactory( - new ScanQueryQueryToolChest( - new ScanQueryConfig().setLegacy(false), - DefaultGenericQueryMetricsFactory.instance() - ), + new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()), new ScanQueryEngine(), new ScanQueryConfig() ); @@ -218,7 +215,6 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn if (useInputFormatApi) { Preconditions.checkArgument(parseSpec == null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDirectory, filter), inputFormat, appendToExisting, @@ -239,7 +235,6 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn } else { Preconditions.checkArgument(inputFormat == null && parseSpec != null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDirectory, filter), createInputFormatFromParseSpec(parseSpec), appendToExisting, @@ -293,7 +288,6 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn null, null, columns, - false, null, null ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index ea6f9d963fe..1cd6a29e10d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -96,10 +96,10 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.NoopDataSegmentKiller; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthTestUtils; @@ -225,7 +225,6 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase indexingServiceClient = new LocalOverlordClient(objectMapper, taskRunner); final TaskConfig taskConfig = new TaskConfigBuilder() .setShuffleDataLocations(ImmutableList.of(new StorageLocationConfig(temporaryFolder.newFolder(), null, null))) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); intermediaryDataManager = new LocalIntermediaryDataManager(new WorkerConfig(), taskConfig, null); remoteApiExecutor = Execs.singleThreaded("coordinator-api-executor"); @@ -616,7 +615,6 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase public void prepareObjectMapper(ObjectMapper objectMapper, IndexIO indexIO) { final TaskConfig taskConfig = new TaskConfigBuilder() - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); objectMapper.setInjectableValues( @@ -652,7 +650,6 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase protected TaskToolbox createTaskToolbox(Task task, TaskActionClient actionClient) throws IOException { TaskConfig config = new TaskConfigBuilder() - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = CentralizedDatasourceSchemaConfig.create(true); return new TaskToolbox.Builder() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java index e71dc6db36d..b8c59d042a3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java @@ -244,7 +244,6 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin if (useInputFormatApi) { Preconditions.checkArgument(parseSpec == null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDir, filter), inputFormat, appendToExisting, @@ -265,7 +264,6 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin } else { Preconditions.checkArgument(inputFormat == null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDir, filter), createInputFormatFromParseSpec(parseSpec), appendToExisting, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java index 1448a5723f3..0d19cd86e03 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java @@ -140,7 +140,6 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP null ), new ParallelIndexIOConfig( - null, getInputSource(), JSON_FORMAT, false, @@ -191,7 +190,6 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP null ), new ParallelIndexIOConfig( - null, getInputSource(), new JsonInputFormat( new JSONPathSpec(true, null), @@ -252,7 +250,6 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP null ), new ParallelIndexIOConfig( - null, getInputSource(), new JsonInputFormat( new JSONPathSpec( @@ -321,7 +318,6 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP null ), new ParallelIndexIOConfig( - null, getInputSource(), JSON_FORMAT, false, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index e85df4fe5dd..cbf71146973 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -76,7 +76,6 @@ public class ParallelIndexSupervisorTaskKillTest extends AbstractParallelIndexSu final ParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), new ParallelIndexIOConfig( - null, // Sub-tasks would run forever new TestInputSource(Pair.of(new TestInput(Integer.MAX_VALUE, TaskState.SUCCESS), 4)), new NoopInputFormat(), @@ -103,7 +102,6 @@ public class ParallelIndexSupervisorTaskKillTest extends AbstractParallelIndexSu final ParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), new ParallelIndexIOConfig( - null, new TestInputSource( Pair.of(new TestInput(10L, TaskState.FAILED), 1), Pair.of(new TestInput(Integer.MAX_VALUE, TaskState.FAILED), 3) @@ -282,7 +280,6 @@ public class ParallelIndexSupervisorTaskKillTest extends AbstractParallelIndexSu new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), new ParallelIndexIOConfig( - null, baseInputSource.withSplit(split), getIngestionSchema().getIOConfig().getInputFormat(), getIngestionSchema().getIOConfig().isAppendToExisting(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 50939a99d8a..c9858b80847 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -131,7 +131,6 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd task = newTask( Intervals.of("2017/2018"), new ParallelIndexIOConfig( - null, new TestInputSource(IntStream.range(0, NUM_SUB_TASKS).boxed().collect(Collectors.toList())), new NoopInputFormat(), false, @@ -519,7 +518,6 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), new ParallelIndexIOConfig( - null, baseInputSource.withSplit(split), getIngestionSchema().getIOConfig().getInputFormat(), getIngestionSchema().getIOConfig().isAppendToExisting(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index cd588d0b6ee..e11fb2ef001 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -182,7 +182,6 @@ public class ParallelIndexSupervisorTaskSerdeTest ); private final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(new File("tmp"), "test_*"), new CsvInputFormat(Arrays.asList("ts", "dim", "val"), null, null, false, 0), false, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index 699476dbc16..57dbafa173f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -231,7 +231,6 @@ public class ParallelIndexSupervisorTaskTest final boolean appendToExisting = true; final boolean forceGuaranteedRollup = true; final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new InlineInputSource("test"), new JsonInputFormat(null, null, null, null, null), appendToExisting, @@ -291,7 +290,6 @@ public class ParallelIndexSupervisorTaskTest { final ObjectMapper mapper = new DefaultObjectMapper(); final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new InlineInputSource("test"), null, false, @@ -528,7 +526,6 @@ public class ParallelIndexSupervisorTaskTest final boolean appendToExisting = false; final boolean forceGuaranteedRollup = true; final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new InlineInputSource("test"), new JsonInputFormat(null, null, null, null, null), appendToExisting, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index 16976e827d2..2455ce692b9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -116,7 +116,7 @@ class ParallelIndexTestingFactory DataSchema dataSchema ) { - ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(null, inputSource, inputFormat, false, false); + ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(inputSource, inputFormat, false, false); return new ParallelIndexIngestionSpec(dataSchema, ioConfig, tuningConfig); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java index 97e4f54c06e..96494b8ac79 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java @@ -323,7 +323,6 @@ public class RangePartitionTaskKillTest extends AbstractMultiPhaseParallelIndexi Preconditions.checkArgument(parseSpec == null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDir, filter), inputFormat, appendToExisting, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index de060e521a9..ac8371acaa0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -406,7 +406,6 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), DEFAULT_INPUT_FORMAT, false, @@ -460,7 +459,6 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), DEFAULT_INPUT_FORMAT, false, @@ -803,7 +801,6 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, "*.json", true), new JsonInputFormat( new JSONPathSpec(true, null), @@ -887,7 +884,6 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, "*.json", true), new JsonInputFormat( new JSONPathSpec(true, null), @@ -967,7 +963,6 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, inputSourceFilter, splittableInputSource), DEFAULT_INPUT_FORMAT, appendToExisting, @@ -990,7 +985,6 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv null ), new ParallelIndexIOConfig( - null, new LocalInputSource(inputDir, inputSourceFilter), createInputFormatFromParseSpec(DEFAULT_PARSE_SPEC), appendToExisting, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java index b70cc7a99e6..4acc3d3f591 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java @@ -57,7 +57,6 @@ public class SinglePhaseSubTaskSpecTest null ), new ParallelIndexIOConfig( - null, new LocalInputSource(new File("baseDir"), "filter"), new JsonInputFormat(null, null, null, null, null), null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java index 6c6db80c1c7..4fc77934f60 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidInputSourceTest.java @@ -34,7 +34,6 @@ import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.TestHelper; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/WindowedSegmentIdTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/WindowedSegmentIdTest.java similarity index 96% rename from indexing-service/src/test/java/org/apache/druid/indexing/firehose/WindowedSegmentIdTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/input/WindowedSegmentIdTest.java index 0586e98c11b..add3ccc0a3c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/WindowedSegmentIdTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/WindowedSegmentIdTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.firehose; +package org.apache.druid.indexing.input; import nl.jqno.equalsverifier.EqualsVerifier; import org.junit.Test; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java index 56695e5cf27..0c490856363 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java @@ -389,7 +389,6 @@ public class ForkingTaskRunnerTest + " \"runTime\" : 2500,\n" + " \"isReadyTime\" : 0,\n" + " \"isReadyResult\" : \"YES\",\n" - + " \"firehose\" : null,\n" + " \"context\" : {\n" + " \"druid.indexer.runner.javaOptsArray\" : [ \"-Xmx10g\", \"-Xms10g\" ],\n" + " \"druid.indexer.fork.property.druid.processing.numThreads\" : 4,\n" @@ -447,7 +446,6 @@ public class ForkingTaskRunnerTest + " \"runTime\" : 2500,\n" + " \"isReadyTime\" : 0,\n" + " \"isReadyResult\" : \"YES\",\n" - + " \"firehose\" : null,\n" + " \"context\" : {\n" + " \"druid.indexer.runner.javaOptsArray\" : \"not a string array\",\n" + " \"druid.indexer.runner.javaOpts\" : \"-Xmx1g -Xms1g\"\n" @@ -543,8 +541,7 @@ public class ForkingTaskRunnerTest .setDefaultHadoopCoordinates(ImmutableList.of()) .setGracefulShutdownTimeout(new Period("PT0S")) .setDirectoryLockTimeout(new Period("PT10S")) - .setShuffleDataLocations(ImmutableList.of()) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()); + .setShuffleDataLocations(ImmutableList.of()); } @Nonnull diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/OverlordBlinkLeadershipTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/OverlordBlinkLeadershipTest.java index 7924dcb6e32..3824cc20f85 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/OverlordBlinkLeadershipTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/OverlordBlinkLeadershipTest.java @@ -65,7 +65,7 @@ public class OverlordBlinkLeadershipTest /** * Test that we can start taskRunner, then stop it (emulating "losing leadership", see {@link - * TaskMaster#stop()}), then creating a new taskRunner from {@link + * TaskMaster#stopBeingLeader()}), then creating a new taskRunner from {@link * org.apache.curator.framework.recipes.leader.LeaderSelectorListener#takeLeadership} implementation in * {@link TaskMaster} and start it again. */ diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index ad34aa10ebe..3acf9ff6f0e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -52,7 +52,7 @@ import org.apache.druid.segment.loading.NoopDataSegmentKiller; import org.apache.druid.segment.loading.NoopDataSegmentMover; import org.apache.druid.segment.loading.NoopDataSegmentPusher; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.SetAndVerifyContextQueryRunner; import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; @@ -94,7 +94,6 @@ public class SingleTaskBackgroundRunnerTest .setBaseDir(temporaryFolder.newFile().toString()) .setDefaultRowFlushBoundary(50000) .setRestoreTasksOnRestart(true) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); final ServiceEmitter emitter = new NoopServiceEmitter(); EmittingLogger.registerEmitter(emitter); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 7b1209e7929..ddbed6be7c7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -37,8 +37,6 @@ import org.apache.druid.client.cache.MapCache; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.AbstractInputSource; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; @@ -46,7 +44,6 @@ import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.discovery.DataNodeService; @@ -130,9 +127,9 @@ import org.apache.druid.segment.loading.LocalDataSegmentKiller; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.NoopDataSegmentArchiver; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; @@ -368,37 +365,6 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest } } - private static class MockFirehoseFactory implements FirehoseFactory - { - @Override - public Firehose connect(InputRowParser parser, File temporaryDirectory) - { - final Iterator inputRowIterator = REALTIME_IDX_TASK_INPUT_ROWS.iterator(); - - return new Firehose() - { - @Override - public boolean hasMore() - { - return inputRowIterator.hasNext(); - } - - @Nullable - @Override - public InputRow nextRow() - { - return inputRowIterator.next(); - } - - @Override - public void close() - { - - } - }; - } - } - @Before public void setUp() throws Exception { @@ -420,17 +386,11 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest // For creating a customized TaskQueue see testRealtimeIndexTaskFailure test taskStorage = setUpTaskStorage(); - handoffNotifierFactory = setUpSegmentHandOffNotifierFactory(); - dataSegmentPusher = setUpDataSegmentPusher(); - mdc = setUpMetadataStorageCoordinator(); - tb = setUpTaskToolboxFactory(dataSegmentPusher, handoffNotifierFactory, mdc); - taskRunner = setUpThreadPoolTaskRunner(tb); - taskQueue = setUpTaskQueue(taskStorage, taskRunner); } @@ -452,7 +412,6 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest case METADATA_TASK_STORAGE: { TestDerbyConnector testDerbyConnector = derbyConnectorRule.getConnector(); mapper.registerSubtypes( - new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory"), new NamedType(MockInputSource.class, "mockInputSource"), new NamedType(NoopInputFormat.class, "noopInputFormat") ); @@ -477,7 +436,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest TaskMaster taskMaster = EasyMock.createMock(TaskMaster.class); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.absent()).anyTimes(); EasyMock.replay(taskMaster); - tsqa = new TaskQueryTool(taskStorage, taskLockbox, taskMaster); + tsqa = new TaskQueryTool(taskStorage, taskLockbox, taskMaster, null, null); return taskStorage; } @@ -603,7 +562,6 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest taskConfig = new TaskConfigBuilder() .setBaseDir(temporaryFolder.newFolder().toString()) .setDefaultRowFlushBoundary(50000) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .setTmpStorageBytesPerTask(-1L) .build(); @@ -726,7 +684,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest ), null ), - new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), + new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) .withMaxRowsInMemory(100) @@ -738,7 +696,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest null ); - final Optional preRunTaskStatus = tsqa.getStatus(indexTask.getId()); + final Optional preRunTaskStatus = tsqa.getTaskStatus(indexTask.getId()); Assert.assertTrue("pre run task status not present", !preRunTaskStatus.isPresent()); final TaskStatus mergedStatus = runTask(indexTask); @@ -789,7 +747,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest null, mapper ), - new IndexIOConfig(null, new MockExceptionInputSource(), new NoopInputFormat(), false, false), + new IndexIOConfig(new MockExceptionInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) .withMaxRowsInMemory(10) @@ -1219,7 +1177,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest ), null ), - new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), + new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) .withMaxRowsInMemory(10) @@ -1235,7 +1193,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest taskQueue.start(); taskStorage.insert(indexTask, TaskStatus.running(indexTask.getId())); - while (tsqa.getStatus(indexTask.getId()).get().isRunnable()) { + while (tsqa.getTaskStatus(indexTask.getId()).get().isRunnable()) { if (System.currentTimeMillis() > startTime + 10 * 1000) { throw new ISE("Where did the task go?!: %s", indexTask.getId()); } @@ -1307,7 +1265,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest ), null ), - new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), + new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) .withMaxRowsInMemory(10) @@ -1319,7 +1277,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest null ); - final Optional preRunTaskStatus = tsqa.getStatus(indexTask.getId()); + final Optional preRunTaskStatus = tsqa.getTaskStatus(indexTask.getId()); Assert.assertTrue("pre run task status not present", !preRunTaskStatus.isPresent()); final TaskStatus mergedStatus = runTask(indexTask); @@ -1409,7 +1367,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest TaskStatus retVal = null; try { TaskStatus status; - while ((status = tsqa.getStatus(taskId).get()).isRunnable()) { + while ((status = tsqa.getTaskStatus(taskId).get()).isRunnable()) { if (taskRunDuration.millisElapsed() > 10_000) { throw new ISE("Where did the task go?!: %s", task.getId()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index a02b5108767..a8c4b5117b1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -75,7 +75,6 @@ import org.junit.Rule; import org.junit.Test; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -1174,99 +1173,6 @@ public class TaskLockboxTest ); } - @Test - public void testGetLockedIntervals() - { - // Acquire locks for task1 - final Task task1 = NoopTask.forDatasource("ds1"); - lockbox.add(task1); - - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - task1, - Intervals.of("2017-01-01/2017-02-01") - ); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - task1, - Intervals.of("2017-04-01/2017-05-01") - ); - - // Acquire locks for task2 - final Task task2 = NoopTask.forDatasource("ds2"); - lockbox.add(task2); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - task2, - Intervals.of("2017-03-01/2017-04-01") - ); - - // Verify the locked intervals - final Map minTaskPriority = new HashMap<>(); - minTaskPriority.put(task1.getDataSource(), 10); - minTaskPriority.put(task2.getDataSource(), 10); - final Map> lockedIntervals = lockbox.getLockedIntervals(minTaskPriority); - Assert.assertEquals(2, lockedIntervals.size()); - - Assert.assertEquals( - Arrays.asList( - Intervals.of("2017-01-01/2017-02-01"), - Intervals.of("2017-04-01/2017-05-01") - ), - lockedIntervals.get(task1.getDataSource()) - ); - - Assert.assertEquals( - Collections.singletonList( - Intervals.of("2017-03-01/2017-04-01")), - lockedIntervals.get(task2.getDataSource()) - ); - } - - @Test - public void testGetLockedIntervalsForLowPriorityTask() - { - // Acquire lock for a low priority task - final Task lowPriorityTask = NoopTask.ofPriority(5); - lockbox.add(lowPriorityTask); - taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - lowPriorityTask, - Intervals.of("2017/2018") - ); - - final Map minTaskPriority = new HashMap<>(); - minTaskPriority.put(lowPriorityTask.getDataSource(), 10); - - Map> lockedIntervals = lockbox.getLockedIntervals(minTaskPriority); - Assert.assertTrue(lockedIntervals.isEmpty()); - } - - @Test - public void testGetLockedIntervalsForEqualPriorityTask() - { - // Acquire lock for a low priority task - final Task task = NoopTask.ofPriority(5); - lockbox.add(task); - taskStorage.insert(task, TaskStatus.running(task.getId())); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - task, - Intervals.of("2017/2018") - ); - - final Map minTaskPriority = new HashMap<>(); - minTaskPriority.put(task.getDataSource(), 5); - - Map> lockedIntervals = lockbox.getLockedIntervals(minTaskPriority); - Assert.assertEquals(1, lockedIntervals.size()); - Assert.assertEquals( - Collections.singletonList(Intervals.of("2017/2018")), - lockedIntervals.get(task.getDataSource()) - ); - } - @Test public void testGetLockedIntervalsForHigherPriorityExclusiveLock() { @@ -1282,6 +1188,7 @@ public class TaskLockboxTest LockFilterPolicy requestForExclusiveLowerPriorityLock = new LockFilterPolicy( task.getDataSource(), 75, + null, null ); @@ -1305,6 +1212,7 @@ public class TaskLockboxTest LockFilterPolicy requestForExclusiveLowerPriorityLock = new LockFilterPolicy( task.getDataSource(), 25, + null, null ); @@ -1332,6 +1240,7 @@ public class TaskLockboxTest LockFilterPolicy requestForReplaceLowerPriorityLock = new LockFilterPolicy( task.getDataSource(), 25, + null, ImmutableMap.of(Tasks.TASK_LOCK_TYPE, TaskLockType.REPLACE.name()) ); @@ -1355,6 +1264,7 @@ public class TaskLockboxTest LockFilterPolicy requestForReplaceLowerPriorityLock = new LockFilterPolicy( task.getDataSource(), 25, + null, ImmutableMap.of( Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name(), @@ -1369,6 +1279,171 @@ public class TaskLockboxTest } + @Test + public void testGetActiveLocks() + { + final Set expectedLocks = new HashSet<>(); + final TaskLock overlappingReplaceLock = + validator.expectLockCreated(TaskLockType.REPLACE, Intervals.of("2024/2025"), 50); + expectedLocks.add(overlappingReplaceLock); + + //Lower priority + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024/2025"), 25); + + final TaskLock overlappingAppendLock = + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-01-01/2024-02-01"), 75); + expectedLocks.add(overlappingAppendLock); + + // Non-overlapping interval + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-12-01/2025-01-01"), 75); + + final TaskLock overlappingExclusiveLock = + validator.expectLockCreated(TaskLockType.EXCLUSIVE, Intervals.of("2020/2021"), 50); + expectedLocks.add(overlappingExclusiveLock); + + LockFilterPolicy policy = new LockFilterPolicy( + "none", + 50, + ImmutableList.of(Intervals.of("2020/2021"), Intervals.of("2024-01-01/2024-07-01")), + null + ); + + LockFilterPolicy policyForNonExistentDatasource = new LockFilterPolicy( + "nonExistent", + 0, + null, + null + ); + + Map> activeLocks = + lockbox.getActiveLocks(ImmutableList.of(policy, policyForNonExistentDatasource)); + Assert.assertEquals(1, activeLocks.size()); + Assert.assertEquals(expectedLocks, new HashSet<>(activeLocks.get("none"))); + } + + @Test + public void testGetActiveLocksWithAppendLockIgnoresAppendLocks() + { + final Set expectedLocks = new HashSet<>(); + final TaskLock overlappingReplaceLock = + validator.expectLockCreated(TaskLockType.REPLACE, Intervals.of("2024/2025"), 50); + expectedLocks.add(overlappingReplaceLock); + + //Lower priority + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024/2025"), 25); + + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-01-01/2024-02-01"), 75); + + // Non-overlapping interval + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-12-01/2025-01-01"), 75); + + final TaskLock overlappingExclusiveLock = + validator.expectLockCreated(TaskLockType.EXCLUSIVE, Intervals.of("2020/2021"), 50); + expectedLocks.add(overlappingExclusiveLock); + + LockFilterPolicy policy = new LockFilterPolicy( + "none", + 50, + ImmutableList.of(Intervals.of("2020/2021"), Intervals.of("2024-01-01/2024-07-01")), + ImmutableMap.of(Tasks.TASK_LOCK_TYPE, TaskLockType.APPEND.name()) + ); + + LockFilterPolicy policyForNonExistentDatasource = new LockFilterPolicy( + "nonExistent", + 0, + null, + null + ); + + Map> activeLocks = + lockbox.getActiveLocks(ImmutableList.of(policy, policyForNonExistentDatasource)); + Assert.assertEquals(1, activeLocks.size()); + Assert.assertEquals(expectedLocks, new HashSet<>(activeLocks.get("none"))); + } + + @Test + public void testGetActiveLocksWithConcurrentLocksIgnoresAppendLocks() + { + final Set expectedLocks = new HashSet<>(); + final TaskLock overlappingReplaceLock = + validator.expectLockCreated(TaskLockType.REPLACE, Intervals.of("2024/2025"), 50); + expectedLocks.add(overlappingReplaceLock); + + //Lower priority + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024/2025"), 25); + + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-01-01/2024-02-01"), 75); + + // Non-overlapping interval + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-12-01/2025-01-01"), 75); + + final TaskLock overlappingExclusiveLock = + validator.expectLockCreated(TaskLockType.EXCLUSIVE, Intervals.of("2020/2021"), 50); + expectedLocks.add(overlappingExclusiveLock); + + LockFilterPolicy policy = new LockFilterPolicy( + "none", + 50, + ImmutableList.of(Intervals.of("2020/2021"), Intervals.of("2024-01-01/2024-07-01")), + ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true, Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name()) + ); + + LockFilterPolicy policyForNonExistentDatasource = new LockFilterPolicy( + "nonExistent", + 0, + null, + null + ); + + Map> activeLocks = + lockbox.getActiveLocks(ImmutableList.of(policy, policyForNonExistentDatasource)); + Assert.assertEquals(1, activeLocks.size()); + Assert.assertEquals(expectedLocks, new HashSet<>(activeLocks.get("none"))); + } + + @Test + public void testGetActiveLocksWithoutConcurrentLocksConsidersAppendLocks() + { + final Set expectedLocks = new HashSet<>(); + final TaskLock overlappingReplaceLock = + validator.expectLockCreated(TaskLockType.REPLACE, Intervals.of("2024/2025"), 50); + + expectedLocks.add(overlappingReplaceLock); + + //Lower priority + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024/2025"), 25); + + final TaskLock overlappingAppendLock = + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-01-01/2024-02-01"), 75); + expectedLocks.add(overlappingAppendLock); + + // Non-overlapping interval + validator.expectLockCreated(TaskLockType.APPEND, Intervals.of("2024-12-01/2025-01-01"), 75); + + final TaskLock overlappingExclusiveLock = + validator.expectLockCreated(TaskLockType.EXCLUSIVE, Intervals.of("2020/2021"), 50); + expectedLocks.add(overlappingExclusiveLock); + + LockFilterPolicy policy = new LockFilterPolicy( + "none", + 50, + ImmutableList.of(Intervals.of("2020/2021"), Intervals.of("2024-01-01/2024-07-01")), + ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, false, Tasks.TASK_LOCK_TYPE, TaskLockType.APPEND.name()) + ); + + LockFilterPolicy policyForNonExistentDatasource = new LockFilterPolicy( + "nonExistent", + 0, + null, + null + ); + + Map> activeLocks = + lockbox.getActiveLocks(ImmutableList.of(policy, policyForNonExistentDatasource)); + Assert.assertEquals(1, activeLocks.size()); + Assert.assertEquals(expectedLocks, new HashSet<>(activeLocks.get("none"))); + } + @Test public void testExclusiveLockCompatibility() { @@ -1770,50 +1845,6 @@ public class TaskLockboxTest validator.expectLockNotGranted(TaskLockType.APPEND, otherGroupTask, Intervals.of("2024/2025")); } - @Test - public void testGetLockedIntervalsForRevokedLocks() - { - // Acquire lock for a low priority task - final Task lowPriorityTask = NoopTask.ofPriority(5); - lockbox.add(lowPriorityTask); - taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - lowPriorityTask, - Intervals.of("2017/2018") - ); - - final Map minTaskPriority = new HashMap<>(); - minTaskPriority.put(lowPriorityTask.getDataSource(), 1); - - Map> lockedIntervals = lockbox.getLockedIntervals(minTaskPriority); - Assert.assertEquals(1, lockedIntervals.size()); - Assert.assertEquals( - Collections.singletonList( - Intervals.of("2017/2018")), - lockedIntervals.get(lowPriorityTask.getDataSource()) - ); - - // Revoke the lowPriorityTask - final Task highPriorityTask = NoopTask.ofPriority(10); - lockbox.add(highPriorityTask); - tryTimeChunkLock( - TaskLockType.EXCLUSIVE, - highPriorityTask, - Intervals.of("2017-05-01/2017-06-01") - ); - - // Verify the locked intervals - minTaskPriority.put(highPriorityTask.getDataSource(), 1); - lockedIntervals = lockbox.getLockedIntervals(minTaskPriority); - Assert.assertEquals(1, lockedIntervals.size()); - Assert.assertEquals( - Collections.singletonList( - Intervals.of("2017-05-01/2017-06-01")), - lockedIntervals.get(highPriorityTask.getDataSource()) - ); - } - @Test public void testFailedToReacquireTaskLock() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java index 8984e73b6b9..ee90a3335a1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java @@ -558,7 +558,6 @@ public class TaskQueueTest extends IngestionTestBase null ); final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new HttpInputSource(Collections.singletonList(URI.create("http://host.org")), "user", new DefaultPasswordProvider(password), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java index edebc0fe8dc..19e4bd33fd8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java @@ -57,8 +57,8 @@ import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.segment.loading.DataSegmentMover; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordRedirectInfoTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordRedirectInfoTest.java index 46e03206f49..f9a0105c067 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordRedirectInfoTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordRedirectInfoTest.java @@ -20,7 +20,7 @@ package org.apache.druid.indexing.overlord.http; import com.google.common.base.Optional; -import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.DruidOverlord; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Before; @@ -32,48 +32,48 @@ import java.net.URLEncoder; public class OverlordRedirectInfoTest { - private TaskMaster taskMaster; + private DruidOverlord overlord; private OverlordRedirectInfo redirectInfo; @Before public void setUp() { - taskMaster = EasyMock.createMock(TaskMaster.class); - redirectInfo = new OverlordRedirectInfo(taskMaster); + overlord = EasyMock.createMock(DruidOverlord.class); + redirectInfo = new OverlordRedirectInfo(overlord); } @Test public void testDoLocalWhenLeading() { - EasyMock.expect(taskMaster.isLeader()).andReturn(true).anyTimes(); - EasyMock.replay(taskMaster); + EasyMock.expect(overlord.isLeader()).andReturn(true).anyTimes(); + EasyMock.replay(overlord); Assert.assertTrue(redirectInfo.doLocal(null)); Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/leader")); Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/isLeader")); Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/other/path")); - EasyMock.verify(taskMaster); + EasyMock.verify(overlord); } @Test public void testDoLocalWhenNotLeading() { - EasyMock.expect(taskMaster.isLeader()).andReturn(false).anyTimes(); - EasyMock.replay(taskMaster); + EasyMock.expect(overlord.isLeader()).andReturn(false).anyTimes(); + EasyMock.replay(overlord); Assert.assertFalse(redirectInfo.doLocal(null)); Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/leader")); Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/isLeader")); Assert.assertFalse(redirectInfo.doLocal("/druid/indexer/v1/other/path")); - EasyMock.verify(taskMaster); + EasyMock.verify(overlord); } @Test public void testGetRedirectURLWithEmptyLocation() { - EasyMock.expect(taskMaster.getRedirectLocation()).andReturn(Optional.absent()).anyTimes(); - EasyMock.replay(taskMaster); + EasyMock.expect(overlord.getRedirectLocation()).andReturn(Optional.absent()).anyTimes(); + EasyMock.replay(overlord); URL url = redirectInfo.getRedirectURL("query", "/request"); Assert.assertNull(url); - EasyMock.verify(taskMaster); + EasyMock.verify(overlord); } @Test @@ -82,11 +82,11 @@ public class OverlordRedirectInfoTest String host = "http://localhost"; String query = "foo=bar&x=y"; String request = "/request"; - EasyMock.expect(taskMaster.getRedirectLocation()).andReturn(Optional.of(host)).anyTimes(); - EasyMock.replay(taskMaster); + EasyMock.expect(overlord.getRedirectLocation()).andReturn(Optional.of(host)).anyTimes(); + EasyMock.replay(overlord); URL url = redirectInfo.getRedirectURL(query, request); Assert.assertEquals("http://localhost/request?foo=bar&x=y", url.toString()); - EasyMock.verify(taskMaster); + EasyMock.verify(overlord); } @Test @@ -98,14 +98,14 @@ public class OverlordRedirectInfoTest "UTF-8" ) + "/status"; - EasyMock.expect(taskMaster.getRedirectLocation()).andReturn(Optional.of(host)).anyTimes(); - EasyMock.replay(taskMaster); + EasyMock.expect(overlord.getRedirectLocation()).andReturn(Optional.of(host)).anyTimes(); + EasyMock.replay(overlord); URL url = redirectInfo.getRedirectURL(null, request); Assert.assertEquals( "http://localhost/druid/indexer/v1/task/index_hadoop_datasource_2017-07-12T07%3A43%3A01.495Z/status", url.toString() ); - EasyMock.verify(taskMaster); + EasyMock.verify(overlord); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java index 4f2c3a38794..93f80ac9709 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java @@ -36,16 +36,22 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.task.KillUnusedSegmentsTask; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.ImmutableWorkerInfo; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; +import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskQueue; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; +import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.WorkerTaskRunner; import org.apache.druid.indexing.overlord.WorkerTaskRunnerQueryAdapter; import org.apache.druid.indexing.overlord.autoscaling.AutoScaler; @@ -58,6 +64,8 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.UOE; +import org.apache.druid.metadata.LockFilterPolicy; +import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup; import org.apache.druid.metadata.TaskLookup.TaskLookupType; @@ -102,7 +110,10 @@ import java.util.concurrent.atomic.AtomicReference; public class OverlordResourceTest { private OverlordResource overlordResource; + private DruidOverlord overlord; private TaskMaster taskMaster; + private TaskStorage taskStorage; + private TaskLockbox taskLockbox; private JacksonConfigManager configManager; private ProvisioningStrategy provisioningStrategy; private AuthConfig authConfig; @@ -121,12 +132,21 @@ public class OverlordResourceTest public void setUp() { taskRunner = EasyMock.createMock(TaskRunner.class); - taskQueue = EasyMock.createMock(TaskQueue.class); + taskQueue = EasyMock.createStrictMock(TaskQueue.class); configManager = EasyMock.createMock(JacksonConfigManager.class); provisioningStrategy = EasyMock.createMock(ProvisioningStrategy.class); authConfig = EasyMock.createMock(AuthConfig.class); + overlord = EasyMock.createStrictMock(DruidOverlord.class); taskMaster = EasyMock.createStrictMock(TaskMaster.class); - taskQueryTool = EasyMock.createStrictMock(TaskQueryTool.class); + taskStorage = EasyMock.createStrictMock(TaskStorage.class); + taskLockbox = EasyMock.createStrictMock(TaskLockbox.class); + taskQueryTool = new TaskQueryTool( + taskStorage, + taskLockbox, + taskMaster, + provisioningStrategy, + configManager + ); indexerMetadataStorageAdapter = EasyMock.createStrictMock(IndexerMetadataStorageAdapter.class); req = EasyMock.createStrictMock(HttpServletRequest.class); workerTaskRunnerQueryAdapter = EasyMock.createStrictMock(WorkerTaskRunnerQueryAdapter.class); @@ -170,6 +190,7 @@ public class OverlordResourceTest }; overlordResource = new OverlordResource( + overlord, taskMaster, taskQueryTool, indexerMetadataStorageAdapter, @@ -178,7 +199,6 @@ public class OverlordResourceTest auditManager, authMapper, workerTaskRunnerQueryAdapter, - provisioningStrategy, authConfig ); } @@ -189,7 +209,8 @@ public class OverlordResourceTest EasyMock.verify( taskRunner, taskMaster, - taskQueryTool, + taskStorage, + taskLockbox, indexerMetadataStorageAdapter, req, workerTaskRunnerQueryAdapter, @@ -200,9 +221,12 @@ public class OverlordResourceTest private void replayAll() { EasyMock.replay( + overlord, taskRunner, + taskQueue, taskMaster, - taskQueryTool, + taskStorage, + taskLockbox, indexerMetadataStorageAdapter, req, workerTaskRunnerQueryAdapter, @@ -216,7 +240,7 @@ public class OverlordResourceTest @Test public void testLeader() { - EasyMock.expect(taskMaster.getCurrentLeader()).andReturn("boz").once(); + EasyMock.expect(overlord.getCurrentLeader()).andReturn("boz").once(); replayAll(); final Response response = overlordResource.getLeader(); @@ -227,8 +251,8 @@ public class OverlordResourceTest @Test public void testIsLeader() { - EasyMock.expect(taskMaster.isLeader()).andReturn(true).once(); - EasyMock.expect(taskMaster.isLeader()).andReturn(false).once(); + EasyMock.expect(overlord.isLeader()).andReturn(true).once(); + EasyMock.expect(overlord.isLeader()).andReturn(false).once(); replayAll(); // true @@ -247,7 +271,7 @@ public class OverlordResourceTest { expectAuthorizationTokenCheck(); EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -282,7 +306,7 @@ public class OverlordResourceTest List tasksIds = ImmutableList.of("id_1", "id_2", "id_3"); EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, (Duration) null)), null) ).andStubReturn( ImmutableList.of( @@ -313,7 +337,7 @@ public class OverlordResourceTest ) ); EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -340,7 +364,7 @@ public class OverlordResourceTest { expectAuthorizationTokenCheck(); EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance(), @@ -381,7 +405,7 @@ public class OverlordResourceTest expectAuthorizationTokenCheck(); //completed tasks EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -423,7 +447,7 @@ public class OverlordResourceTest expectAuthorizationTokenCheck(); //active tasks EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance() @@ -466,7 +490,7 @@ public class OverlordResourceTest { expectAuthorizationTokenCheck(); EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance() @@ -516,7 +540,7 @@ public class OverlordResourceTest ) ); EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -550,7 +574,7 @@ public class OverlordResourceTest { expectAuthorizationTokenCheck(); EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, (Duration) null)), null ) @@ -577,7 +601,7 @@ public class OverlordResourceTest expectAuthorizationTokenCheck(); Duration duration = new Period("PT86400S").toStandardDuration(); EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( EasyMock.anyObject(), EasyMock.anyObject() ) @@ -609,7 +633,7 @@ public class OverlordResourceTest // Setup mocks to return completed, active, known, pending and running tasks EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -658,7 +682,7 @@ public class OverlordResourceTest // Setup mocks to return completed, active, known, pending and running tasks EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -707,8 +731,10 @@ public class OverlordResourceTest replayAll(); // Verify that only the tasks of read access datasource are returned - expectedException.expect(WebApplicationException.class); - overlordResource.getTasks(null, Datasources.BUZZFEED, null, null, null, req); + Assert.assertThrows( + WebApplicationException.class, + () -> overlordResource.getTasks(null, Datasources.BUZZFEED, null, null, null, req) + ); } @Test @@ -716,7 +742,7 @@ public class OverlordResourceTest { expectAuthorizationTokenCheck(); EasyMock.expect( - taskQueryTool.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null) @@ -749,7 +775,7 @@ public class OverlordResourceTest .getTasks("blah", "ds_test", null, null, null, req) .getEntity(); Assert.assertEquals( - "Invalid state : blah, valid values are: [pending, waiting, running, complete]", + "Invalid task state[blah]. Must be one of [pending, waiting, running, complete].", responseObject.toString() ); } @@ -824,7 +850,7 @@ public class OverlordResourceTest { expectAuthorizationTokenCheck(); - EasyMock.expect(taskMaster.isLeader()).andReturn(true); + EasyMock.expect(overlord.isLeader()).andReturn(true); EasyMock .expect( indexerMetadataStorageAdapter.deletePendingSegments( @@ -847,7 +873,7 @@ public class OverlordResourceTest { expectAuthorizationTokenCheck(); - EasyMock.expect(taskMaster.isLeader()).andReturn(true); + EasyMock.expect(overlord.isLeader()).andReturn(true); final String exceptionMsg = "Some exception msg"; EasyMock .expect( @@ -873,7 +899,7 @@ public class OverlordResourceTest { expectAuthorizationTokenCheck(); - EasyMock.expect(taskMaster.isLeader()).andReturn(true); + EasyMock.expect(overlord.isLeader()).andReturn(true); final String exceptionMsg = "An internal defensive exception"; EasyMock .expect( @@ -899,7 +925,7 @@ public class OverlordResourceTest { expectAuthorizationTokenCheck(); - EasyMock.expect(taskMaster.isLeader()).andReturn(true); + EasyMock.expect(overlord.isLeader()).andReturn(true); final String exceptionMsg = "An unexpected illegal state exception"; EasyMock .expect( @@ -925,7 +951,7 @@ public class OverlordResourceTest { expectAuthorizationTokenCheck(); - EasyMock.expect(taskMaster.isLeader()).andReturn(false); + EasyMock.expect(overlord.isLeader()).andReturn(false); replayAll(); @@ -943,11 +969,13 @@ public class OverlordResourceTest // set authorization token properly, but isn't called in this test. // This should be fixed in https://github.com/apache/druid/issues/6685. // expectAuthorizationTokenCheck(); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.absent()).anyTimes(); + final NoopTask task = NoopTask.create(); - EasyMock.expect(taskQueryTool.getTask("mytask")) + EasyMock.expect(taskStorage.getTask("mytask")) .andReturn(Optional.of(task)); - EasyMock.expect(taskQueryTool.getTask("othertask")) + EasyMock.expect(taskStorage.getTask("othertask")) .andReturn(Optional.absent()); replayAll(); @@ -1033,31 +1061,33 @@ public class OverlordResourceTest @Test public void testGetLockedIntervals() throws Exception { - final Map minTaskPriority = Collections.singletonMap("ds1", 0); - final Map> expectedLockedIntervals = Collections.singletonMap( + final List lockFilterPolicies = ImmutableList.of( + new LockFilterPolicy("ds1", 25, null, null) + ); + final Map> expectedIntervals = Collections.singletonMap( "ds1", Arrays.asList( Intervals.of("2012-01-01/2012-01-02"), - Intervals.of("2012-01-02/2012-01-03") + Intervals.of("2012-01-01/2012-01-02") ) ); - EasyMock.expect(taskQueryTool.getLockedIntervals(minTaskPriority)) - .andReturn(expectedLockedIntervals); + EasyMock.expect(taskLockbox.getLockedIntervals(lockFilterPolicies)) + .andReturn(expectedIntervals); replayAll(); - final Response response = overlordResource.getDatasourceLockedIntervals(minTaskPriority); + final Response response = overlordResource.getDatasourceLockedIntervals(lockFilterPolicies); Assert.assertEquals(200, response.getStatus()); final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); - Map> observedLockedIntervals = jsonMapper.readValue( + Map> observedIntervals = jsonMapper.readValue( jsonMapper.writeValueAsString(response.getEntity()), new TypeReference>>() { } ); - Assert.assertEquals(expectedLockedIntervals, observedLockedIntervals); + Assert.assertEquals(expectedIntervals, observedIntervals); } @Test @@ -1068,7 +1098,65 @@ public class OverlordResourceTest Response response = overlordResource.getDatasourceLockedIntervals(null); Assert.assertEquals(400, response.getStatus()); - response = overlordResource.getDatasourceLockedIntervals(Collections.emptyMap()); + response = overlordResource.getDatasourceLockedIntervals(Collections.emptyList()); + Assert.assertEquals(400, response.getStatus()); + } + + @Test + public void testGetActiveLocks() throws Exception + { + final List lockFilterPolicies = ImmutableList.of( + new LockFilterPolicy("ds1", 25, null, null) + ); + final Map> expectedLocks = Collections.singletonMap( + "ds1", + Arrays.asList( + new TimeChunkLock( + TaskLockType.REPLACE, + "groupId", + "datasource", + Intervals.of("2012-01-01/2012-01-02"), + "version", + 25 + ), + new TimeChunkLock( + TaskLockType.EXCLUSIVE, + "groupId", + "datasource", + Intervals.of("2012-01-02/2012-01-03"), + "version", + 75 + ) + ) + ); + + EasyMock.expect(taskLockbox.getActiveLocks(lockFilterPolicies)) + .andReturn(expectedLocks); + replayAll(); + + final Response response = overlordResource.getActiveLocks(lockFilterPolicies); + Assert.assertEquals(200, response.getStatus()); + + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + Map> observedLocks = jsonMapper.readValue( + jsonMapper.writeValueAsString(response.getEntity()), + new TypeReference() + { + } + ).getDatasourceToLocks(); + + Assert.assertEquals(expectedLocks, observedLocks); + } + + @Test + public void testGetActiveLocksWithEmptyBody() + { + replayAll(); + + Response response = overlordResource.getActiveLocks(null); + Assert.assertEquals(400, response.getStatus()); + + response = overlordResource.getActiveLocks(Collections.emptyList()); Assert.assertEquals(400, response.getStatus()); } @@ -1079,13 +1167,9 @@ public class OverlordResourceTest // This should be fixed in https://github.com/apache/druid/issues/6685. // expectAuthorizationTokenCheck(); TaskQueue mockQueue = EasyMock.createMock(TaskQueue.class); - EasyMock.expect(taskMaster.isLeader()).andReturn(true).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn( - Optional.of(taskRunner) - ).anyTimes(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn( Optional.of(mockQueue) - ).anyTimes(); + ).once(); mockQueue.shutdown("id_1", "Shutdown request from user"); EasyMock.expectLastCall(); @@ -1105,14 +1189,12 @@ public class OverlordResourceTest // This should be fixed in https://github.com/apache/druid/issues/6685. // expectAuthorizationTokenCheck(); TaskQueue mockQueue = EasyMock.createMock(TaskQueue.class); - EasyMock.expect(taskMaster.isLeader()).andReturn(true).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn( - Optional.of(taskRunner) - ).anyTimes(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn( Optional.of(mockQueue) ).anyTimes(); - EasyMock.expect(taskQueryTool.getActiveTaskInfo("datasource")).andStubReturn(ImmutableList.of( + EasyMock.expect( + taskStorage.getTaskInfos(TaskLookup.activeTasksOnly(), "datasource") + ).andStubReturn(ImmutableList.of( new TaskInfo<>( "id_1", DateTime.now(ISOChronology.getInstanceUTC()), @@ -1146,9 +1228,10 @@ public class OverlordResourceTest public void testShutdownAllTasksForNonExistingDataSource() { final TaskQueue taskQueue = EasyMock.createMock(TaskQueue.class); - EasyMock.expect(taskMaster.isLeader()).andReturn(true).anyTimes(); + EasyMock.expect(overlord.isLeader()).andReturn(true).anyTimes(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskQueryTool.getActiveTaskInfo(EasyMock.anyString())).andReturn(Collections.emptyList()); + EasyMock.expect(taskStorage.getTaskInfos(EasyMock.anyObject(TaskLookup.class), EasyMock.anyString())) + .andReturn(Collections.emptyList()); replayAll(); final Response response = overlordResource.shutdownTasksForDataSource("notExisting"); @@ -1222,10 +1305,7 @@ public class OverlordResourceTest @Test public void testGetTotalWorkerCapacityNotLeader() { - EasyMock.reset(taskMaster); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn( - Optional.absent() - ).anyTimes(); + EasyMock.expect(overlord.isLeader()).andReturn(false); replayAll(); final Response response = overlordResource.getTotalWorkerCapacity(); Assert.assertEquals(HttpResponseStatus.SERVICE_UNAVAILABLE.getCode(), response.getStatus()); @@ -1235,10 +1315,13 @@ public class OverlordResourceTest public void testGetTotalWorkerCapacityWithUnknown() { WorkerBehaviorConfig workerBehaviorConfig = EasyMock.createMock(WorkerBehaviorConfig.class); - AtomicReference workerBehaviorConfigAtomicReference = new AtomicReference<>(workerBehaviorConfig); - EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference); + AtomicReference workerBehaviorConfigAtomicReference + = new AtomicReference<>(workerBehaviorConfig); + EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)) + .andReturn(workerBehaviorConfigAtomicReference); EasyMock.expect(taskRunner.getTotalCapacity()).andReturn(-1); EasyMock.expect(taskRunner.getUsedCapacity()).andReturn(-1); + EasyMock.expect(overlord.isLeader()).andReturn(true); replayAll(); final Response response = overlordResource.getTotalWorkerCapacity(); @@ -1255,6 +1338,7 @@ public class OverlordResourceTest EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference); EasyMock.expect(taskRunner.getTotalCapacity()).andReturn(-1); EasyMock.expect(taskRunner.getUsedCapacity()).andReturn(-1); + EasyMock.expect(overlord.isLeader()).andReturn(true); replayAll(); final Response response = overlordResource.getTotalWorkerCapacity(); @@ -1272,6 +1356,7 @@ public class OverlordResourceTest EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference); EasyMock.expect(taskRunner.getTotalCapacity()).andReturn(-1); EasyMock.expect(taskRunner.getUsedCapacity()).andReturn(-1); + EasyMock.expect(overlord.isLeader()).andReturn(true); replayAll(); final Response response = overlordResource.getTotalWorkerCapacity(); @@ -1317,6 +1402,7 @@ public class OverlordResourceTest workerTaskRunner, autoScaler ); + EasyMock.expect(overlord.isLeader()).andReturn(true); replayAll(); final Response response = overlordResource.getTotalWorkerCapacity(); Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus()); @@ -1362,6 +1448,7 @@ public class OverlordResourceTest workerTaskRunner, autoScaler ); + EasyMock.expect(overlord.isLeader()).andReturn(true); replayAll(); final Response response = overlordResource.getTotalWorkerCapacity(); Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus()); @@ -1400,7 +1487,7 @@ public class OverlordResourceTest } @Test - public void testResourceActionsForTaskWithFirehoseAndInputSecurityEnabled() + public void testResourceActionsForTaskWithInvalidSecurityAndInputSecurityEnabled() { final String dataSource = "dataSourceTest"; final UOE expectedException = new UOE("unsupported"); @@ -1452,26 +1539,12 @@ public class OverlordResourceTest @Test public void testGetMultipleTaskStatuses_presentTaskQueue() { - replayAll(); - - TaskQueue taskQueue = EasyMock.createMock(TaskQueue.class); + EasyMock.expect(taskMaster.getTaskQueue()) + .andReturn(Optional.of(taskQueue)); EasyMock.expect(taskQueue.getTaskStatus("task")) .andReturn(Optional.of(TaskStatus.running("task"))); - TaskMaster taskMaster = EasyMock.createMock(TaskMaster.class); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)); - EasyMock.replay(taskMaster, taskQueue); - OverlordResource overlordResource = new OverlordResource( - taskMaster, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + replayAll(); + final Object response = overlordResource.getMultipleTaskStatuses(ImmutableSet.of("task")) .getEntity(); Assert.assertEquals(ImmutableMap.of("task", TaskStatus.running("task")), response); @@ -1480,27 +1553,11 @@ public class OverlordResourceTest @Test public void testGetMultipleTaskStatuses_absentTaskQueue() { + EasyMock.expect(taskStorage.getStatus("task")) + .andReturn(Optional.of(TaskStatus.running("task"))); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.absent()); replayAll(); - TaskQueryTool taskQueryTool = EasyMock.createMock(TaskQueryTool.class); - EasyMock.expect(taskQueryTool.getStatus("task")) - .andReturn(Optional.of(TaskStatus.running("task"))); - - TaskMaster taskMaster = EasyMock.createMock(TaskMaster.class); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.absent()); - EasyMock.replay(taskMaster, taskQueryTool); - OverlordResource overlordResource = new OverlordResource( - taskMaster, - taskQueryTool, - null, - null, - null, - null, - null, - null, - null, - null - ); final Object response = overlordResource.getMultipleTaskStatuses(ImmutableSet.of("task")) .getEntity(); Assert.assertEquals(ImmutableMap.of("task", TaskStatus.running("task")), response); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java index 9bbf2e9fc8a..d1c2167b929 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java @@ -47,6 +47,7 @@ import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.NoopTaskContextEnricher; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -106,6 +107,7 @@ public class OverlordTest private TestingServer server; private Timing timing; private CuratorFramework curator; + private DruidOverlord overlord; private TaskMaster taskMaster; private TaskLockbox taskLockbox; private TaskStorage taskStorage; @@ -235,6 +237,11 @@ public class OverlordTest taskRunnerFactory.build().run(goodTask); taskMaster = new TaskMaster( + taskActionClientFactory, + supervisorManager + ); + overlord = new DruidOverlord( + taskMaster, new TaskLockConfig(), new TaskQueueConfig(null, new Period(1), null, new Period(10), null, null), new DefaultTaskConfig(), @@ -260,20 +267,23 @@ public class OverlordTest public void testOverlordRun() throws Exception { // basic task master lifecycle test - taskMaster.start(); + overlord.start(); announcementLatch.await(); - while (!taskMaster.isLeader()) { + while (!overlord.isLeader()) { // I believe the control will never reach here and thread will never sleep but just to be on safe side Thread.sleep(10); } - Assert.assertEquals(taskMaster.getCurrentLeader(), druidNode.getHostAndPort()); - Assert.assertEquals(Optional.absent(), taskMaster.getRedirectLocation()); + Assert.assertEquals(overlord.getCurrentLeader(), druidNode.getHostAndPort()); + Assert.assertEquals(Optional.absent(), overlord.getRedirectLocation()); - final TaskQueryTool taskQueryTool = new TaskQueryTool(taskStorage, taskLockbox, taskMaster); - final WorkerTaskRunnerQueryAdapter workerTaskRunnerQueryAdapter = new WorkerTaskRunnerQueryAdapter(taskMaster, null); + final TaskQueryTool taskQueryTool + = new TaskQueryTool(taskStorage, taskLockbox, taskMaster, null, null); + final WorkerTaskRunnerQueryAdapter workerTaskRunnerQueryAdapter + = new WorkerTaskRunnerQueryAdapter(taskMaster, null); // Test Overlord resource stuff AuditManager auditManager = EasyMock.createNiceMock(AuditManager.class); overlordResource = new OverlordResource( + overlord, taskMaster, taskQueryTool, new IndexerMetadataStorageAdapter(taskStorage, null), @@ -282,7 +292,6 @@ public class OverlordTest auditManager, AuthTestUtils.TEST_AUTHORIZER_MAPPER, workerTaskRunnerQueryAdapter, - null, new AuthConfig() ); Response response = overlordResource.getLeader(); @@ -351,8 +360,8 @@ public class OverlordTest Assert.assertEquals(1, (((List) response.getEntity()).size())); Assert.assertEquals(1, taskMaster.getStats().rowCount()); - taskMaster.stop(); - Assert.assertFalse(taskMaster.isLeader()); + overlord.stop(); + Assert.assertFalse(overlord.isLeader()); Assert.assertEquals(0, taskMaster.getStats().rowCount()); EasyMock.verify(taskActionClientFactory); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java index 6f92aeef646..36dfd966e8d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.overlord.sampler; import com.fasterxml.jackson.databind.InjectableValues; -import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import org.apache.druid.client.indexing.SamplerResponse; @@ -28,7 +27,6 @@ import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.indexing.DataSchema; @@ -64,8 +62,7 @@ public class IndexTaskSamplerSpecTest extends EasyMockSupport .addValue(InputSourceSampler.class, inputSourceSampler) .addValue(ObjectMapper.class, MAPPER) ); - MAPPER.registerModules((Iterable) new SamplerModule().getJacksonModules()); - MAPPER.registerModules((Iterable) new FirehoseModule().getJacksonModules()); + MAPPER.registerModules(new SamplerModule().getJacksonModules()); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index 3b5775c22b8..258ebff7b50 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -117,8 +117,8 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; @@ -573,7 +573,6 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport .setBaseTaskDir(new File(directory, "baseTaskDir").getPath()) .setDefaultRowFlushBoundary(50000) .setRestoreTasksOnRestart(true) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); final TestDerbyConnector derbyConnector = derby.getConnector(); derbyConnector.createDataSourceTable(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java index ca3cd6edf8b..88249509ef9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java @@ -46,7 +46,7 @@ public class TaskAnnouncementTest @Test public void testBackwardsCompatibleSerde() throws Exception { - final IndexTask.IndexIOConfig ioConfig = new IndexTask.IndexIOConfig(null, new NoopInputSource(), null, null, null); + final IndexTask.IndexIOConfig ioConfig = new IndexTask.IndexIOConfig(new NoopInputSource(), null, null, null); final Task task = new IndexTask( "theid", new TaskResource("rofl", 2), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index 8217a12eb83..6b08be3a3c6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -52,7 +52,7 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; import org.apache.druid.server.security.AuthTestUtils; @@ -116,7 +116,6 @@ public class WorkerTaskManagerTest .setBaseDir(FileUtils.createTempDir().toString()) .setDefaultRowFlushBoundary(0) .setRestoreTasksOnRestart(restoreTasksOnRestart) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); TaskActionClientFactory taskActionClientFactory = EasyMock.createNiceMock(TaskActionClientFactory.class); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index c3191cc068d..e1822505b97 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -55,7 +55,7 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.IndexerZkConfig; import org.apache.druid.server.initialization.ServerConfig; @@ -160,7 +160,6 @@ public class WorkerTaskMonitorTest final TaskConfig taskConfig = new TaskConfigBuilder() .setBaseDir(FileUtils.createTempDir().toString()) .setDefaultRowFlushBoundary(0) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); TaskActionClientFactory taskActionClientFactory = EasyMock.createNiceMock(TaskActionClientFactory.class); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java index f5b0ca5644e..5505160d989 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java @@ -67,7 +67,6 @@ public class LocalIntermediaryDataManagerAutoCleanupTest { this.taskConfig = new TaskConfigBuilder() .setShuffleDataLocations(ImmutableList.of(new StorageLocationConfig(tempDir.newFolder(), null, null))) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); this.overlordClient = new NoopOverlordClient() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java index fabb1cfb196..cc14d4505e9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java @@ -70,7 +70,6 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest siblingLocation = tempDir.newFolder(); final TaskConfig taskConfig = new TaskConfigBuilder() .setShuffleDataLocations(ImmutableList.of(new StorageLocationConfig(intermediarySegmentsLocation, 1200L, null))) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); final OverlordClient overlordClient = new NoopOverlordClient(); intermediaryDataManager = new LocalIntermediaryDataManager(workerConfig, taskConfig, overlordClient); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java index ceb74f3631a..c8c9b7fe066 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java @@ -101,7 +101,6 @@ public class ShuffleDataSegmentPusherTest final WorkerConfig workerConfig = new WorkerConfig(); final TaskConfig taskConfig = new TaskConfigBuilder() .setShuffleDataLocations(ImmutableList.of(new StorageLocationConfig(temporaryFolder.newFolder(), null, null))) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); final OverlordClient overlordClient = new NoopOverlordClient(); if (LOCAL.equals(intermediateDataStore)) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java index 40e8ad3e276..69d48417dee 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java @@ -93,7 +93,6 @@ public class ShuffleResourceTest }; final TaskConfig taskConfig = new TaskConfigBuilder() .setShuffleDataLocations(ImmutableList.of(new StorageLocationConfig(tempDir.newFolder(), null, null))) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); final OverlordClient overlordClient = new NoopOverlordClient() { diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java index 96e8d500fa6..20bdd9767a7 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java @@ -130,7 +130,7 @@ public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTes new Pair<>(false, false) ); - // with DruidInputSource instead of IngestSegmentFirehose + // with DruidInputSource doReindexTest( INDEX_DATASOURCE, INDEX_DRUID_INPUT_SOURCE_DATASOURCE, diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java index 65b8dc0b1ac..84f2dff1d79 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java @@ -24,6 +24,7 @@ import com.google.inject.Inject; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.testing.clients.CoordinatorResourceTestClient; import org.apache.druid.testing.utils.ITRetryUtil; @@ -343,12 +344,12 @@ public class ITIndexerTest extends AbstractITBatchIndexTest submitIndexTask(INDEX_TASK, datasourceName); // Wait until it acquires a lock - final Map minTaskPriority = Collections.singletonMap(datasourceName, 0); + final List lockFilterPolicies = Collections.singletonList(new LockFilterPolicy(datasourceName, 0, null, null)); final Map> lockedIntervals = new HashMap<>(); ITRetryUtil.retryUntilFalse( () -> { lockedIntervals.clear(); - lockedIntervals.putAll(indexer.getLockedIntervals(minTaskPriority)); + lockedIntervals.putAll(indexer.getLockedIntervals(lockFilterPolicies)); return lockedIntervals.isEmpty(); }, "Verify Intervals are Locked" diff --git a/integration-tests-ex/cases/src/test/resources/indexer/export_task.json b/integration-tests-ex/cases/src/test/resources/indexer/export_task.json index e5bfdac4af7..45e59704a85 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/export_task.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/export_task.json @@ -129,7 +129,6 @@ "delta", "page" ], - "legacy": false, "context": { "__exportFileFormat": "CSV", "__resultFormat": "array", diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java index 8e5ee737f03..74d56065ef4 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java @@ -28,8 +28,8 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.StatusResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.guice.TestClient; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -86,7 +86,7 @@ public class CompactionResourceTestClient } } - public void deleteCompactionConfig(final String dataSource) throws Exception + public void deleteDataSourceCompactionConfig(final String dataSource) throws Exception { String url = StringUtils.format("%sconfig/compaction/%s", getCoordinatorURL(), StringUtils.urlEncode(dataSource)); StatusResponseHolder response = httpClient.go(new Request(HttpMethod.DELETE, new URL(url)), responseHandler).get(); @@ -100,7 +100,7 @@ public class CompactionResourceTestClient } } - public CoordinatorCompactionConfig getCoordinatorCompactionConfigs() throws Exception + public DruidCompactionConfig getCompactionConfig() throws Exception { String url = StringUtils.format("%sconfig/compaction", getCoordinatorURL()); StatusResponseHolder response = httpClient.go( @@ -113,7 +113,7 @@ public class CompactionResourceTestClient response.getContent() ); } - return jsonMapper.readValue(response.getContent(), new TypeReference() {}); + return jsonMapper.readValue(response.getContent(), new TypeReference() {}); } public DataSourceCompactionConfig getDataSourceCompactionConfig(String dataSource) throws Exception diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java deleted file mode 100644 index 761d9f55dba..00000000000 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java +++ /dev/null @@ -1,181 +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.testing.clients; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.java.util.http.client.response.StatusResponseHandler; -import org.apache.druid.java.util.http.client.response.StatusResponseHolder; -import org.apache.druid.testing.guice.TestClient; -import org.jboss.netty.handler.codec.http.HttpMethod; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; - -import javax.ws.rs.core.MediaType; -import java.io.BufferedReader; -import java.io.InputStreamReader; -import java.net.URL; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Map; -import java.util.concurrent.ExecutionException; - -public class EventReceiverFirehoseTestClient -{ - private static final Logger LOG = new Logger(EventReceiverFirehoseTestClient.class); - - static final int NUM_RETRIES = 30; - static final long DELAY_FOR_RETRIES_MS = 10000; - - private final String host; - private final ObjectMapper jsonMapper; - private final HttpClient httpClient; - private final String chatID; - private final ObjectMapper smileMapper; - - public EventReceiverFirehoseTestClient( - String host, - String chatID, - ObjectMapper jsonMapper, - @TestClient HttpClient httpClient, - ObjectMapper smileMapper - ) - { - this.host = host; - this.jsonMapper = jsonMapper; - this.httpClient = httpClient; - this.chatID = chatID; - this.smileMapper = smileMapper; - } - - private String getURL() - { - return StringUtils.format( - "https://%s/druid/worker/v1/chat/%s/push-events/", - host, - chatID - ); - } - - /** - * post events from the collection and return the count of events accepted - * - * @param events Collection of events to be posted - * - * @return - */ - public int postEvents(Collection> events, ObjectMapper objectMapper, String mediaType) - throws InterruptedException - { - int retryCount = 0; - while (true) { - try { - StatusResponseHolder response = httpClient.go( - new Request(HttpMethod.POST, new URL(getURL())) - .setContent(mediaType, objectMapper.writeValueAsBytes(events)), - StatusResponseHandler.getInstance() - ).get(); - - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE( - "Error while posting events to url[%s] status[%s] content[%s]", - getURL(), - response.getStatus(), - response.getContent() - ); - } - Map responseData = objectMapper.readValue( - response.getContent(), new TypeReference>() - { - } - ); - return responseData.get("eventCount"); - } - // adding retries to flaky tests using channels - catch (ExecutionException e) { - if (retryCount > NUM_RETRIES) { - throw new RuntimeException(e); //giving up now - } else { - LOG.info(e, "received exception, sleeping and retrying"); - retryCount++; - Thread.sleep(DELAY_FOR_RETRIES_MS); - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - } - - /** - * Reads each events from file and post them to the indexing service. - * Uses both smileMapper and jsonMapper to send events alternately. - * - * @param file location of file to post events from - * - * @return number of events sent to the indexing service - */ - public int postEventsFromFile(String file) - { - try ( - BufferedReader reader = new BufferedReader( - new InputStreamReader( - EventReceiverFirehoseTestClient.class.getResourceAsStream(file), - StandardCharsets.UTF_8 - ) - ) - ) { - - String s; - Collection> events = new ArrayList>(); - // Test sending events using both jsonMapper and smileMapper. - // sends events one by one using both jsonMapper and smileMapper. - int totalEventsPosted = 0; - int expectedEventsPosted = 0; - while ((s = reader.readLine()) != null) { - events.add(this.jsonMapper.readValue(s, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT)); - ObjectMapper mapper = (totalEventsPosted % 2 == 0) ? jsonMapper : smileMapper; - String mediaType = (totalEventsPosted % 2 == 0) - ? MediaType.APPLICATION_JSON - : SmileMediaTypes.APPLICATION_JACKSON_SMILE; - totalEventsPosted += postEvents(events, mapper, mediaType); - - expectedEventsPosted += events.size(); - events = new ArrayList<>(); - } - - if (totalEventsPosted != expectedEventsPosted) { - throw new ISE("All events not posted, expected : %d actual : %d", events.size(), totalEventsPosted); - } - return totalEventsPosted; - } - catch (Exception e) { - throw new RuntimeException(e); - } - - } -} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java index 8167b9b64e1..f75dc6043f9 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java @@ -39,6 +39,7 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.StatusResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.guice.TestClient; @@ -334,13 +335,13 @@ public class OverlordResourceTestClient } } - public Map> getLockedIntervals(Map minTaskPriority) + public Map> getLockedIntervals(List lockFilterPolicies) { try { - String jsonBody = jsonMapper.writeValueAsString(minTaskPriority); + String jsonBody = jsonMapper.writeValueAsString(lockFilterPolicies); StatusResponseHolder response = httpClient.go( - new Request(HttpMethod.POST, new URL(getIndexerURL() + "lockedIntervals")) + new Request(HttpMethod.POST, new URL(getIndexerURL() + "lockedIntervals/v2")) .setContent( "application/json", StringUtils.toUtf8(jsonBody) diff --git a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java index d08712dea35..1195795cd97 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java @@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Injector; import com.google.inject.Module; import org.apache.druid.guice.GuiceInjectors; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceTuningConfigModule; import org.apache.druid.initialization.Initialization; @@ -50,7 +49,6 @@ public class DruidTestModuleFactory implements IModuleFactory { return ImmutableList.of( new DruidTestModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTuningConfigModule() ); diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java deleted file mode 100644 index 63316e76687..00000000000 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/CompactionUtil.java +++ /dev/null @@ -1,83 +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.testing.utils; - -import org.apache.druid.data.input.MaxSizeSplitHintSpec; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; -import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; -import org.joda.time.Period; - -/** - * Contains utility methods for Compaction. - */ -public class CompactionUtil -{ - - private CompactionUtil() - { - // no instantiation - } - - public static DataSourceCompactionConfig createCompactionConfig( - String fullDatasourceName, - Integer maxRowsPerSegment, - Period skipOffsetFromLatest - ) - { - return new DataSourceCompactionConfig( - fullDatasourceName, - null, - null, - null, - skipOffsetFromLatest, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - new MaxSizeSplitHintSpec(null, 1), - new DynamicPartitionsSpec(maxRowsPerSegment, null), - null, - null, - null, - null, - null, - 1, - null, - null, - null, - null, - null, - 1, - null - ), - null, - null, - null, - null, - new UserCompactionTaskIOConfig(true), - null, - null - ); - } - -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java index 8d980d76f12..0a1071c38e7 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java @@ -19,18 +19,19 @@ package org.apache.druid.tests.coordinator.duty; +import com.google.common.collect.ImmutableList; import com.google.inject.Inject; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.testing.clients.CompactionResourceTestClient; import org.apache.druid.testing.clients.TaskResponseObject; import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.CompactionUtil; import org.apache.druid.testing.utils.EventSerializer; import org.apache.druid.testing.utils.ITRetryUtil; import org.apache.druid.testing.utils.KafkaUtil; @@ -53,7 +54,6 @@ import org.testng.annotations.Test; import java.io.Closeable; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -265,13 +265,13 @@ public class ITAutoCompactionLockContentionTest extends AbstractKafkaIndexingSer */ private void ensureLockedIntervals(Interval... intervals) { - final Map minTaskPriority = Collections.singletonMap(fullDatasourceName, 0); + final LockFilterPolicy lockFilterPolicy = new LockFilterPolicy(fullDatasourceName, 0, null, null); final List lockedIntervals = new ArrayList<>(); ITRetryUtil.retryUntilTrue( () -> { lockedIntervals.clear(); - Map> allIntervals = indexer.getLockedIntervals(minTaskPriority); + Map> allIntervals = indexer.getLockedIntervals(ImmutableList.of(lockFilterPolicy)); if (allIntervals.containsKey(fullDatasourceName)) { lockedIntervals.addAll(allIntervals.get(fullDatasourceName)); } @@ -304,26 +304,26 @@ public class ITAutoCompactionLockContentionTest extends AbstractKafkaIndexingSer */ private void submitAndVerifyCompactionConfig() throws Exception { - final DataSourceCompactionConfig compactionConfig = CompactionUtil - .createCompactionConfig(fullDatasourceName, Specs.MAX_ROWS_PER_SEGMENT, Period.ZERO); + final DataSourceCompactionConfig dataSourceCompactionConfig = DataSourceCompactionConfig + .builder() + .forDataSource(fullDatasourceName) + .withSkipOffsetFromLatest(Period.ZERO) + .withMaxRowsPerSegment(Specs.MAX_ROWS_PER_SEGMENT) + .build(); compactionResource.updateCompactionTaskSlot(0.5, 10, null); - compactionResource.submitCompactionConfig(compactionConfig); + compactionResource.submitCompactionConfig(dataSourceCompactionConfig); // Wait for compaction config to persist Thread.sleep(2000); // Verify that the compaction config is updated correctly. - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - DataSourceCompactionConfig observedCompactionConfig = null; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSourceCompactionConfig.getDataSource().equals(fullDatasourceName)) { - observedCompactionConfig = dataSourceCompactionConfig; - } - } - Assert.assertEquals(observedCompactionConfig, compactionConfig); + DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); + DataSourceCompactionConfig observedCompactionConfig + = compactionConfig.findConfigForDatasource(fullDatasourceName).orNull(); + Assert.assertEquals(observedCompactionConfig, dataSourceCompactionConfig); observedCompactionConfig = compactionResource.getDataSourceCompactionConfig(fullDatasourceName); - Assert.assertEquals(observedCompactionConfig, compactionConfig); + Assert.assertEquals(observedCompactionConfig, dataSourceCompactionConfig); } /** diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index 230a19236c1..d09bced3313 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -28,10 +28,10 @@ import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.task.CompactionIntervalSpec; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.overlord.http.TaskPayloadResponse; @@ -54,8 +54,8 @@ import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregat import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; @@ -328,8 +328,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics(CompactionEngine engine) throws Exception { // added = 31, count = null, sum_added = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); @@ -356,7 +356,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("sum_added", "added")}, - false + false, + engine ); // should now only have 1 row after compaction // added = null, count = 2, sum_added = 62 @@ -559,8 +560,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionDutyCanUpdateCompactionConfig() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyCanUpdateCompactionConfig(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -571,9 +572,9 @@ public class ITAutoCompactionTest extends AbstractIndexerTest verifyQuery(INDEX_QUERIES_RESOURCE); // Dummy compaction config which will be overwritten - submitCompactionConfig(10000, NO_SKIP_OFFSET); + submitCompactionConfig(10000, NO_SKIP_OFFSET, engine); // New compaction config should overwrites the existing compaction config - submitCompactionConfig(1, NO_SKIP_OFFSET); + submitCompactionConfig(1, NO_SKIP_OFFSET, engine); LOG.info("Auto compaction test with dynamic partitioning"); @@ -584,25 +585,28 @@ public class ITAutoCompactionTest extends AbstractIndexerTest verifySegmentsCompacted(10, 1); checkCompactionIntervals(intervalsBeforeCompaction); - LOG.info("Auto compaction test with hash partitioning"); + if (engine == CompactionEngine.NATIVE) { + // HashedPartitionsSpec not supported by MSQ. + LOG.info("Auto compaction test with hash partitioning"); - final HashedPartitionsSpec hashedPartitionsSpec = new HashedPartitionsSpec(null, 3, null); - submitCompactionConfig(hashedPartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false); - // 2 segments published per day after compaction. - forceTriggerAutoCompaction(4); - verifyQuery(INDEX_QUERIES_RESOURCE); - verifySegmentsCompacted(hashedPartitionsSpec, 4); - checkCompactionIntervals(intervalsBeforeCompaction); + final HashedPartitionsSpec hashedPartitionsSpec = new HashedPartitionsSpec(null, 3, null); + submitCompactionConfig(hashedPartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false, engine); + // 2 segments published per day after compaction. + forceTriggerAutoCompaction(4); + verifyQuery(INDEX_QUERIES_RESOURCE); + verifySegmentsCompacted(hashedPartitionsSpec, 4); + checkCompactionIntervals(intervalsBeforeCompaction); + } LOG.info("Auto compaction test with range partitioning"); - final SingleDimensionPartitionsSpec rangePartitionsSpec = new SingleDimensionPartitionsSpec( + final DimensionRangePartitionsSpec rangePartitionsSpec = new DimensionRangePartitionsSpec( 5, null, - "city", + ImmutableList.of("city"), false ); - submitCompactionConfig(rangePartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false); + submitCompactionConfig(rangePartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false, engine); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(rangePartitionsSpec, 2); @@ -695,8 +699,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue(CompactionEngine engine) throws Exception { // Interval is "2013-08-31/2013-09-02", segment gran is DAY, // "maxRowsPerSegment": 3 @@ -732,7 +736,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest Granularity newGranularity = Granularities.YEAR; // Set dropExisting to true // "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z", - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + true, + engine + ); List expectedIntervalAfterCompaction = new ArrayList<>(); for (String interval : intervalsBeforeCompaction) { @@ -750,7 +760,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest // "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z", newGranularity = Granularities.MONTH; // Set dropExisting to true - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + true, + engine + ); // Since dropExisting is set to true... // Again data is only in two days @@ -778,7 +794,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest // compact only tombstones, so it should be a tombstone itself. newGranularity = new PeriodGranularity(new Period("P6M"), null, DateTimeZone.UTC); // Set dropExisting to true - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + true, + engine + ); // Since dropExisting is set to true... // The earlier 12 segments with MONTH granularity will be completely covered, overshadowed, by the @@ -804,8 +826,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueThenFalse() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrueThenFalse(CompactionEngine engine) throws Exception { // Interval is "2013-08-31/2013-09-02", segment gran is DAY, // "maxRowsPerSegment": 3 @@ -841,7 +863,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest Granularity newGranularity = Granularities.YEAR; // Set dropExisting to true // "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z", - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); List expectedIntervalAfterCompaction = new ArrayList<>(); for (String interval : intervalsBeforeCompaction) { @@ -859,7 +881,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest // "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z", newGranularity = Granularities.MONTH; // Set dropExisting to true - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); // Since dropExisting is set to true... // Again data is only in two days @@ -885,9 +907,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest LOG.info("Auto compaction test with SEMESTER segment granularity, dropExisting is false, over tombstones"); newGranularity = new PeriodGranularity(new Period("P6M"), null, DateTimeZone.UTC); // Set dropExisting to false - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, - null, null - ), false); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + false, + engine + ); // Since dropExisting is set to false the first semester will be forced to dropExisting true // Hence, we will have two, one tombstone for the first semester and one data segment for the second. @@ -963,8 +989,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndMixedVersion(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -974,14 +1000,14 @@ public class ITAutoCompactionTest extends AbstractIndexerTest verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1)); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1), engine); //...compacted into 1 new segment for 1 day. 1 day compacted and 1 day skipped/remains uncompacted. (3 total) forceTriggerAutoCompaction(3); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); Granularity newGranularity = Granularities.YEAR; - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), engine); LOG.info("Auto compaction test with YEAR segment granularity"); @@ -1001,8 +1027,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegmentsHaveSameSegmentGranularity() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegmentsHaveSameSegmentGranularity(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -1013,7 +1039,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest verifyQuery(INDEX_QUERIES_RESOURCE); // Compacted without SegmentGranularity in auto compaction config - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, engine); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1023,7 +1049,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest // Segments were compacted and already has DAY granularity since it was initially ingested with DAY granularity. // Now set auto compaction with DAY granularity in the granularitySpec Granularity newGranularity = Granularities.DAY; - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), engine); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1033,8 +1059,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegmentsHaveDifferentSegmentGranularity() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndExistingCompactedSegmentsHaveDifferentSegmentGranularity(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -1045,7 +1071,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest verifyQuery(INDEX_QUERIES_RESOURCE); // Compacted without SegmentGranularity in auto compaction config - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, engine); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1055,7 +1081,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest // Segments were compacted and already has DAY granularity since it was initially ingested with DAY granularity. // Now set auto compaction with DAY granularity in the granularitySpec Granularity newGranularity = Granularities.YEAR; - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null)); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), engine); forceTriggerAutoCompaction(1); verifyQuery(INDEX_QUERIES_RESOURCE); verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED); @@ -1066,8 +1092,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranularityCoveringMultipleSegmentsInTimelineAndDropExistingTrue() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityAndSmallerSegmentGranularityCoveringMultipleSegmentsInTimelineAndDropExistingTrue(CompactionEngine engine) throws Exception { loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -1079,7 +1105,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest Granularity newGranularity = Granularities.YEAR; // Set dropExisting to true - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); List expectedIntervalAfterCompaction = new ArrayList<>(); // We will still have one visible segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR) @@ -1109,7 +1135,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest coordinator.getSegmentIntervals(fullDatasourceName); // Since dropExisting is set to true... // This will submit a single compaction task for interval of 2013-01-01/2014-01-01 with MONTH granularity - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); // verify: expectedIntervalAfterCompaction = new ArrayList<>(); // The previous segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR) will be @@ -1245,8 +1271,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSegment() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithSegmentGranularityCoarserAndNotAlignWithSegment(CompactionEngine engine) throws Exception { updateCompactionTaskSlot(1, 1, null); final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); @@ -1263,7 +1289,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null), - false + false, + engine ); // Before compaction, we have segments with the interval 2013-08-26T00:00:00.000Z/2013-09-02T00:00:00.000Z // We will compact the latest segment to MONTH. @@ -1292,8 +1319,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionDutyWithRollup() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithRollup(CompactionEngine engine) throws Exception { final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); Map specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.DAY, Granularities.DAY, false, ImmutableList.of(new Interval("2013-08-31/2013-09-02", chrono)))); @@ -1309,7 +1336,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(null, null, true), - false + false, + engine ); forceTriggerAutoCompaction(2); queryAndResultFields = ImmutableMap.of( @@ -1328,8 +1356,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionDutyWithQueryGranularity() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithQueryGranularity(CompactionEngine engine) throws Exception { final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); Map specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, true, ImmutableList.of(new Interval("2013-08-31/2013-09-02", chrono)))); @@ -1345,7 +1373,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(null, Granularities.DAY, null), - false + false, + engine ); forceTriggerAutoCompaction(2); queryAndResultFields = ImmutableMap.of( @@ -1364,8 +1393,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test - public void testAutoCompactionDutyWithDimensionsSpec() throws Exception + @Test(dataProvider = "engine") + public void testAutoCompactionDutyWithDimensionsSpec(CompactionEngine engine) throws Exception { // Index data with dimensions "page", "language", "user", "unpatrolled", "newPage", "robot", "anonymous", // "namespace", "continent", "country", "region", "city" @@ -1392,7 +1421,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, null, - false + false, + engine ); forceTriggerAutoCompaction(2); @@ -1575,17 +1605,17 @@ public class ITAutoCompactionTest extends AbstractIndexerTest { // First try update without useAutoScaleSlots updateCompactionTaskSlot(3, 5, null); - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); + DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); // Should be default value which is false - Assert.assertFalse(coordinatorCompactionConfig.isUseAutoScaleSlots()); + Assert.assertFalse(compactionConfig.isUseAutoScaleSlots()); // Now try update from default value to useAutoScaleSlots=true updateCompactionTaskSlot(3, 5, true); - coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - Assert.assertTrue(coordinatorCompactionConfig.isUseAutoScaleSlots()); + compactionConfig = compactionResource.getCompactionConfig(); + Assert.assertTrue(compactionConfig.isUseAutoScaleSlots()); // Now try update from useAutoScaleSlots=true to useAutoScaleSlots=false updateCompactionTaskSlot(3, 5, false); - coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - Assert.assertFalse(coordinatorCompactionConfig.isUseAutoScaleSlots()); + compactionConfig = compactionResource.getCompactionConfig(); + Assert.assertFalse(compactionConfig.isUseAutoScaleSlots()); } private void loadData(String indexTask) throws Exception @@ -1760,30 +1790,6 @@ public class ITAutoCompactionTest extends AbstractIndexerTest ); } - private void submitCompactionConfig( - PartitionsSpec partitionsSpec, - Period skipOffsetFromLatest, - int maxNumConcurrentSubTasks, - UserCompactionTaskGranularityConfig granularitySpec, - UserCompactionTaskDimensionsConfig dimensionsSpec, - UserCompactionTaskTransformConfig transformSpec, - AggregatorFactory[] metricsSpec, - boolean dropExisting - ) throws Exception - { - submitCompactionConfig( - partitionsSpec, - skipOffsetFromLatest, - maxNumConcurrentSubTasks, - granularitySpec, - dimensionsSpec, - transformSpec, - metricsSpec, - dropExisting, - null - ); - } - private void submitCompactionConfig( PartitionsSpec partitionsSpec, Period skipOffsetFromLatest, @@ -1796,7 +1802,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest @Nullable CompactionEngine engine ) throws Exception { - DataSourceCompactionConfig compactionConfig = new DataSourceCompactionConfig( + DataSourceCompactionConfig dataSourceCompactionConfig = new DataSourceCompactionConfig( fullDatasourceName, null, null, @@ -1829,21 +1835,17 @@ public class ITAutoCompactionTest extends AbstractIndexerTest transformSpec, !dropExisting ? null : new UserCompactionTaskIOConfig(true), engine, - null + ImmutableMap.of("maxNumTasks", 2) ); - compactionResource.submitCompactionConfig(compactionConfig); + compactionResource.submitCompactionConfig(dataSourceCompactionConfig); // Wait for compaction config to persist Thread.sleep(2000); // Verify that the compaction config is updated correctly. - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - DataSourceCompactionConfig foundDataSourceCompactionConfig = null; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSourceCompactionConfig.getDataSource().equals(fullDatasourceName)) { - foundDataSourceCompactionConfig = dataSourceCompactionConfig; - } - } + DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); + DataSourceCompactionConfig foundDataSourceCompactionConfig + = compactionConfig.findConfigForDatasource(fullDatasourceName).orNull(); Assert.assertNotNull(foundDataSourceCompactionConfig); Assert.assertNotNull(foundDataSourceCompactionConfig.getTuningConfig()); Assert.assertEquals(foundDataSourceCompactionConfig.getTuningConfig().getPartitionsSpec(), partitionsSpec); @@ -1858,16 +1860,12 @@ public class ITAutoCompactionTest extends AbstractIndexerTest private void deleteCompactionConfig() throws Exception { - compactionResource.deleteCompactionConfig(fullDatasourceName); + compactionResource.deleteDataSourceCompactionConfig(fullDatasourceName); // Verify that the compaction config is updated correctly. - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - DataSourceCompactionConfig foundDataSourceCompactionConfig = null; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSourceCompactionConfig.getDataSource().equals(fullDatasourceName)) { - foundDataSourceCompactionConfig = dataSourceCompactionConfig; - } - } + DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); + DataSourceCompactionConfig foundDataSourceCompactionConfig + = compactionConfig.findConfigForDatasource(fullDatasourceName).orNull(); Assert.assertNull(foundDataSourceCompactionConfig); } @@ -1949,11 +1947,11 @@ public class ITAutoCompactionTest extends AbstractIndexerTest { compactionResource.updateCompactionTaskSlot(compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots); // Verify that the compaction config is updated correctly. - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - Assert.assertEquals(coordinatorCompactionConfig.getCompactionTaskSlotRatio(), compactionTaskSlotRatio); - Assert.assertEquals(coordinatorCompactionConfig.getMaxCompactionTaskSlots(), maxCompactionTaskSlots); + DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); + Assert.assertEquals(compactionConfig.getCompactionTaskSlotRatio(), compactionTaskSlotRatio); + Assert.assertEquals(compactionConfig.getMaxCompactionTaskSlots(), maxCompactionTaskSlots); if (useAutoScaleSlots != null) { - Assert.assertEquals(coordinatorCompactionConfig.isUseAutoScaleSlots(), useAutoScaleSlots.booleanValue()); + Assert.assertEquals(compactionConfig.isUseAutoScaleSlots(), useAutoScaleSlots.booleanValue()); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java index 9e1b54143bb..b2ca693a442 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java @@ -24,9 +24,8 @@ import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; @@ -44,7 +43,6 @@ import org.testng.annotations.Test; @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITAutoCompactionUpgradeTest extends AbstractIndexerTest { - private static final Logger LOG = new Logger(ITAutoCompactionUpgradeTest.class); private static final String UPGRADE_DATASOURCE_NAME = "upgradeTest"; @Inject @@ -58,67 +56,56 @@ public class ITAutoCompactionUpgradeTest extends AbstractIndexerTest { // Verify that compaction config already exist. This config was inserted manually into the database using SQL script. // This auto compaction configuration payload is from Druid 0.21.0 - CoordinatorCompactionConfig coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - DataSourceCompactionConfig foundDataSourceCompactionConfig = null; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSourceCompactionConfig.getDataSource().equals(UPGRADE_DATASOURCE_NAME)) { - foundDataSourceCompactionConfig = dataSourceCompactionConfig; - } - } + DruidCompactionConfig coordinatorCompactionConfig = compactionResource.getCompactionConfig(); + DataSourceCompactionConfig foundDataSourceCompactionConfig + = coordinatorCompactionConfig.findConfigForDatasource(UPGRADE_DATASOURCE_NAME).orNull(); Assert.assertNotNull(foundDataSourceCompactionConfig); // Now submit a new auto compaction configuration PartitionsSpec newPartitionsSpec = new DynamicPartitionsSpec(4000, null); Period newSkipOffset = Period.seconds(0); - DataSourceCompactionConfig compactionConfig = new DataSourceCompactionConfig( - UPGRADE_DATASOURCE_NAME, - null, - null, - null, - newSkipOffset, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - new MaxSizeSplitHintSpec(null, 1), - newPartitionsSpec, - null, - null, - null, - null, - null, - 1, - null, - null, - null, - null, - null, - 1, - null - ), - new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null), - null, - null, - null, - new UserCompactionTaskIOConfig(true), - null, - null - ); + DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig + .builder() + .forDataSource(UPGRADE_DATASOURCE_NAME) + .withSkipOffsetFromLatest(newSkipOffset) + .withTuningConfig( + new UserCompactionTaskQueryTuningConfig( + null, + null, + null, + null, + new MaxSizeSplitHintSpec(null, 1), + newPartitionsSpec, + null, + null, + null, + null, + null, + 1, + null, + null, + null, + null, + null, + 1, + null + ) + ) + .withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) + ) + .withIoConfig(new UserCompactionTaskIOConfig(true)) + .build(); compactionResource.submitCompactionConfig(compactionConfig); // Wait for compaction config to persist Thread.sleep(2000); // Verify that compaction was successfully updated - coordinatorCompactionConfig = compactionResource.getCoordinatorCompactionConfigs(); - foundDataSourceCompactionConfig = null; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSourceCompactionConfig.getDataSource().equals(UPGRADE_DATASOURCE_NAME)) { - foundDataSourceCompactionConfig = dataSourceCompactionConfig; - } - } + coordinatorCompactionConfig = compactionResource.getCompactionConfig(); + foundDataSourceCompactionConfig + = coordinatorCompactionConfig.findConfigForDatasource(UPGRADE_DATASOURCE_NAME).orNull(); Assert.assertNotNull(foundDataSourceCompactionConfig); Assert.assertNotNull(foundDataSourceCompactionConfig.getTuningConfig()); Assert.assertEquals(foundDataSourceCompactionConfig.getTuningConfig().getPartitionsSpec(), newPartitionsSpec); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java index be2ab61a5f5..1312685fe5a 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java @@ -137,7 +137,7 @@ public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTes new Pair<>(false, false) ); - // with DruidInputSource instead of IngestSegmentFirehose + // with DruidInputSource doReindexTest( INDEX_DATASOURCE, INDEX_DRUID_INPUT_SOURCE_DATASOURCE, diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java index f527135c80d..dfe308e2c1b 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITIndexerTest.java @@ -24,6 +24,7 @@ import com.google.inject.Inject; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.testing.clients.CoordinatorResourceTestClient; import org.apache.druid.testing.guice.DruidTestModuleFactory; @@ -342,12 +343,12 @@ public class ITIndexerTest extends AbstractITBatchIndexTest submitIndexTask(INDEX_TASK, datasourceName); // Wait until it acquires a lock - final Map minTaskPriority = Collections.singletonMap(datasourceName, 0); + final List lockFilterPolicies = Collections.singletonList(new LockFilterPolicy(datasourceName, 0, null, null)); final Map> lockedIntervals = new HashMap<>(); ITRetryUtil.retryUntilFalse( () -> { lockedIntervals.clear(); - lockedIntervals.putAll(indexer.getLockedIntervals(minTaskPriority)); + lockedIntervals.putAll(indexer.getLockedIntervals(lockFilterPolicies)); return lockedIntervals.isEmpty(); }, "Verify Intervals are Locked" diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java index 0e8fc904949..5695156fa9b 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java @@ -115,7 +115,7 @@ public class ITPerfectRollupParallelIndexTest extends AbstractITBatchIndexTest ); - // with DruidInputSource instead of IngestSegmentFirehose + // with DruidInputSource doReindexTest( INDEX_DATASOURCE, INDEX_DRUID_INPUT_SOURCE_DATASOURCE, diff --git a/licenses.yaml b/licenses.yaml index abfc8924898..dcdac7bd187 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -5031,7 +5031,7 @@ license_category: binary module: web-console license_name: Apache License version 2.0 copyright: Palantir Technologies -version: 4.2.1 +version: 5.1.1 --- @@ -5040,7 +5040,7 @@ license_category: binary module: web-console license_name: Apache License version 2.0 copyright: Palantir Technologies -version: 4.20.2 +version: 5.10.5 --- @@ -5049,7 +5049,7 @@ license_category: binary module: web-console license_name: Apache License version 2.0 copyright: Palantir Technologies -version: 0.9.35 +version: 2.3.7 --- @@ -5058,7 +5058,7 @@ license_category: binary module: web-console license_name: Apache License version 2.0 copyright: Palantir Technologies -version: 4.4.36 +version: 5.3.7 --- @@ -5067,16 +5067,7 @@ license_category: binary module: web-console license_name: Apache License version 2.0 copyright: Palantir Technologies -version: 4.16.0 - ---- - -name: "@blueprintjs/popover2" -license_category: binary -module: web-console -license_name: Apache License version 2.0 -copyright: Palantir Technologies -version: 1.14.11 +version: 5.10.0 --- @@ -5085,7 +5076,7 @@ license_category: binary module: web-console license_name: Apache License version 2.0 copyright: Palantir Technologies -version: 4.9.24 +version: 5.2.1 --- @@ -5206,13 +5197,13 @@ license_file_path: licenses/bin/@emotion-weak-memoize.MIT --- -name: "@hypnosphi/create-react-context" +name: "@fontsource/open-sans" license_category: binary module: web-console -license_name: MIT License -copyright: James Kyle -version: 0.3.1 -license_file_path: licenses/bin/@hypnosphi-create-react-context.MIT +license_name: SIL Open Font License 1.1 +copyright: Google Inc. +version: 5.0.28 +license_file_path: licenses/bin/@fontsource-open-sans.OFL --- @@ -5235,16 +5226,6 @@ license_file_path: licenses/bin/@popperjs-core.MIT --- -name: "@types/dom4" -license_category: binary -module: web-console -license_name: MIT License -copyright: Microsoft Corporation. -version: 2.0.2 -license_file_path: licenses/bin/@types-dom4.MIT - ---- - name: "@types/parse-json" license_category: binary module: web-console @@ -5355,16 +5336,6 @@ license_file_path: licenses/bin/babel-plugin-syntax-jsx.MIT --- -name: "call-bind" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.0.2 -license_file_path: licenses/bin/call-bind.MIT - ---- - name: "callsites" license_category: binary module: web-console @@ -5649,7 +5620,7 @@ license_category: binary module: web-console license_name: MIT License copyright: Marnus Weststrate -version: 1.3.8 +version: 2.0.1 license_file_path: licenses/bin/date-fns-tz.MIT --- @@ -5664,26 +5635,6 @@ license_file_path: licenses/bin/date-fns.MIT --- -name: "deep-equal" -license_category: binary -module: web-console -license_name: MIT License -copyright: James Halliday -version: 1.1.1 -license_file_path: licenses/bin/deep-equal.MIT - ---- - -name: "define-properties" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.2.0 -license_file_path: licenses/bin/define-properties.MIT - ---- - name: "delayed-stream" license_category: binary module: web-console @@ -5723,16 +5674,6 @@ license_file_path: licenses/bin/dom-helpers.MIT --- -name: "dom4" -license_category: binary -module: web-console -license_name: MIT License -copyright: Andrea Giammarchi -version: 2.1.6 -license_file_path: licenses/bin/dom4.MIT - ---- - name: "dot-case" license_category: binary module: web-console @@ -5812,16 +5753,6 @@ license_file_path: licenses/bin/follow-redirects.MIT --- -name: "fontsource-open-sans" -license_category: binary -module: web-console -license_name: MIT License -copyright: Lotus -version: 3.0.9 -license_file_path: licenses/bin/fontsource-open-sans.MIT - ---- - name: "form-data" license_category: binary module: web-console @@ -5842,36 +5773,6 @@ license_file_path: licenses/bin/function-bind.MIT --- -name: "functions-have-names" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.2.3 -license_file_path: licenses/bin/functions-have-names.MIT - ---- - -name: "get-intrinsic" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.2.0 -license_file_path: licenses/bin/get-intrinsic.MIT - ---- - -name: "gud" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jamie Kyle -version: 1.0.0 -license_file_path: licenses/bin/gud.MIT - ---- - name: "has-flag" license_category: binary module: web-console @@ -5892,36 +5793,6 @@ license_file_path: licenses/bin/has-own-prop.MIT --- -name: "has-property-descriptors" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.0.0 -license_file_path: licenses/bin/has-property-descriptors.MIT - ---- - -name: "has-symbols" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.0.3 -license_file_path: licenses/bin/has-symbols.MIT - ---- - -name: "has-tostringtag" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.0.0 -license_file_path: licenses/bin/has-tostringtag.MIT - ---- - name: "has" license_category: binary module: web-console @@ -6011,16 +5882,6 @@ license_file_path: licenses/bin/internmap.ISC --- -name: "is-arguments" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.1.1 -license_file_path: licenses/bin/is-arguments.MIT - ---- - name: "is-arrayish" license_category: binary module: web-console @@ -6041,26 +5902,6 @@ license_file_path: licenses/bin/is-core-module.MIT --- -name: "is-date-object" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.0.5 -license_file_path: licenses/bin/is-date-object.MIT - ---- - -name: "is-regex" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.1.4 -license_file_path: licenses/bin/is-regex.MIT - ---- - name: "isarray" license_category: binary module: web-console @@ -6151,16 +5992,6 @@ license_file_path: licenses/bin/lodash.isequal.MIT --- -name: "lodash" -license_category: binary -module: web-console -license_name: MIT License -copyright: John-David Dalton -version: 4.17.21 -license_file_path: licenses/bin/lodash.MIT - ---- - name: "loose-envify" license_category: binary module: web-console @@ -6271,26 +6102,6 @@ license_file_path: licenses/bin/object-assign.MIT --- -name: "object-is" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.1.5 -license_file_path: licenses/bin/object-is.MIT - ---- - -name: "object-keys" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.1.1 -license_file_path: licenses/bin/object-keys.MIT - ---- - name: "opensans" license_category: binary module: web-console @@ -6380,16 +6191,6 @@ license_file_path: licenses/bin/path-type.MIT --- -name: "popper.js" -license_category: binary -module: web-console -license_name: MIT License -copyright: Federico Zivolo -version: 1.16.1 -license_file_path: licenses/bin/popper.js.MIT - ---- - name: "prop-types" license_category: binary module: web-console @@ -6460,6 +6261,16 @@ license_file_path: licenses/bin/react-fast-compare.MIT --- +name: "react-innertext" +license_category: binary +module: web-console +license_name: MIT License +copyright: Charles Stover +version: 1.1.5 +license_file_path: licenses/bin/react-innertext.MIT + +--- + name: "react-is" license_category: binary module: web-console @@ -6475,7 +6286,7 @@ license_category: binary module: web-console license_name: MIT License copyright: Travis Arnold -version: 1.3.11 +version: 2.3.0 license_file_path: licenses/bin/react-popper.MIT --- @@ -6530,6 +6341,16 @@ license_file_path: licenses/bin/react-transition-group.BSD3 --- +name: "react-uid" +license_category: binary +module: web-console +license_name: MIT License +copyright: Marina Korzunova +version: 2.3.3 +license_file_path: licenses/bin/react-uid.MIT + +--- + name: "react" license_category: binary module: web-console @@ -6550,16 +6371,6 @@ license_file_path: licenses/bin/regenerator-runtime.MIT --- -name: "regexp.prototype.flags" -license_category: binary -module: web-console -license_name: MIT License -copyright: Jordan Harband -version: 1.5.0 -license_file_path: licenses/bin/regexp.prototype.flags.MIT - ---- - name: "resolve-from" license_category: binary module: web-console @@ -6730,16 +6541,6 @@ license_file_path: licenses/bin/tslib.0BSD --- -name: "typed-styles" -license_category: binary -module: web-console -license_name: MIT License -copyright: lttb -version: 0.0.7 -license_file_path: licenses/bin/typed-styles.MIT - ---- - name: "upper-case-first" license_category: binary module: web-console diff --git a/licenses/bin/@fontsource-open-sans.OFL b/licenses/bin/@fontsource-open-sans.OFL new file mode 100644 index 00000000000..d5e5ab592f4 --- /dev/null +++ b/licenses/bin/@fontsource-open-sans.OFL @@ -0,0 +1,93 @@ +Copyright 2020 The Open Sans Project Authors (https://github.com/googlefonts/opensans) + +This Font Software is licensed under the SIL Open Font License, Version 1.1. +This license is copied below, and is also available with a FAQ at: +http://scripts.sil.org/OFL + + +----------------------------------------------------------- +SIL OPEN FONT LICENSE Version 1.1 - 26 February 2007 +----------------------------------------------------------- + +PREAMBLE +The goals of the Open Font License (OFL) are to stimulate worldwide +development of collaborative font projects, to support the font creation +efforts of academic and linguistic communities, and to provide a free and +open framework in which fonts may be shared and improved in partnership +with others. + +The OFL allows the licensed fonts to be used, studied, modified and +redistributed freely as long as they are not sold by themselves. The +fonts, including any derivative works, can be bundled, embedded, +redistributed and/or sold with any software provided that any reserved +names are not used by derivative works. The fonts and derivatives, +however, cannot be released under any other type of license. The +requirement for fonts to remain under this license does not apply +to any document created using the fonts or their derivatives. + +DEFINITIONS +"Font Software" refers to the set of files released by the Copyright +Holder(s) under this license and clearly marked as such. This may +include source files, build scripts and documentation. + +"Reserved Font Name" refers to any names specified as such after the +copyright statement(s). + +"Original Version" refers to the collection of Font Software components as +distributed by the Copyright Holder(s). + +"Modified Version" refers to any derivative made by adding to, deleting, +or substituting -- in part or in whole -- any of the components of the +Original Version, by changing formats or by porting the Font Software to a +new environment. + +"Author" refers to any designer, engineer, programmer, technical +writer or other person who contributed to the Font Software. + +PERMISSION & CONDITIONS +Permission is hereby granted, free of charge, to any person obtaining +a copy of the Font Software, to use, study, copy, merge, embed, modify, +redistribute, and sell modified and unmodified copies of the Font +Software, subject to the following conditions: + +1) Neither the Font Software nor any of its individual components, +in Original or Modified Versions, may be sold by itself. + +2) Original or Modified Versions of the Font Software may be bundled, +redistributed and/or sold with any software, provided that each copy +contains the above copyright notice and this license. These can be +included either as stand-alone text files, human-readable headers or +in the appropriate machine-readable metadata fields within text or +binary files as long as those fields can be easily viewed by the user. + +3) No Modified Version of the Font Software may use the Reserved Font +Name(s) unless explicit written permission is granted by the corresponding +Copyright Holder. This restriction only applies to the primary font name as +presented to the users. + +4) The name(s) of the Copyright Holder(s) or the Author(s) of the Font +Software shall not be used to promote, endorse or advertise any +Modified Version, except to acknowledge the contribution(s) of the +Copyright Holder(s) and the Author(s) or with their explicit written +permission. + +5) The Font Software, modified or unmodified, in part or in whole, +must be distributed entirely under this license, and must not be +distributed under any other license. The requirement for fonts to +remain under this license does not apply to any document created +using the Font Software. + +TERMINATION +This license becomes null and void if any of the above conditions are +not met. + +DISCLAIMER +THE FONT SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO ANY WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT +OF COPYRIGHT, PATENT, TRADEMARK, OR OTHER RIGHT. IN NO EVENT SHALL THE +COPYRIGHT HOLDER BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +INCLUDING ANY GENERAL, SPECIAL, INDIRECT, INCIDENTAL, OR CONSEQUENTIAL +DAMAGES, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF THE USE OR INABILITY TO USE THE FONT SOFTWARE OR FROM +OTHER DEALINGS IN THE FONT SOFTWARE. diff --git a/licenses/bin/react-innertext.MIT b/licenses/bin/react-innertext.MIT new file mode 100644 index 00000000000..8109ef9ff8a --- /dev/null +++ b/licenses/bin/react-innertext.MIT @@ -0,0 +1,21 @@ +MIT License + +Copyright (c) 2017 Charles Stover + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/licenses/bin/react-uid.MIT b/licenses/bin/react-uid.MIT new file mode 100644 index 00000000000..6c48e928b3c --- /dev/null +++ b/licenses/bin/react-uid.MIT @@ -0,0 +1,21 @@ +MIT License + +Copyright (c) 2016 Anton + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/pom.xml b/pom.xml index 09d237a3005..11738830512 100644 --- a/pom.xml +++ b/pom.xml @@ -609,7 +609,7 @@ pom - org.hibernate + org.hibernate.validator hibernate-validator ${hibernate-validator.version} diff --git a/processing/pom.xml b/processing/pom.xml index df0ca6fd464..bfb39cf2f77 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -203,7 +203,7 @@ airline - org.hibernate + org.hibernate.validator hibernate-validator runtime diff --git a/processing/src/main/java/org/apache/druid/collections/CircularList.java b/processing/src/main/java/org/apache/druid/collections/CircularList.java new file mode 100644 index 00000000000..7cf551d6cf8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/collections/CircularList.java @@ -0,0 +1,89 @@ +/* + * 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.collections; + +import javax.annotation.concurrent.NotThreadSafe; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Set; + +/** + * A circular list that is backed by an ordered list of elements containing no duplicates. The list is ordered by the + * supplied comparator. The iterator keeps track of the current position, so iterating the list multiple times will + * resume from the last location and continue until a caller explicitly terminates it. + *

    + * This class is not thread-safe and must be used from a single thread. + */ +@NotThreadSafe +public class CircularList implements Iterable +{ + private final List elements = new ArrayList<>(); + private int currentPosition; + + public CircularList(final Set elements, final Comparator comparator) + { + this.elements.addAll(elements); + this.elements.sort(comparator); + this.currentPosition = -1; + } + + @Override + public Iterator iterator() + { + return new Iterator() + { + @Override + public boolean hasNext() + { + return elements.size() > 0; + } + + @Override + public T next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + advanceCursor(); + return elements.get(currentPosition); + } + + private void advanceCursor() + { + if (++currentPosition >= elements.size()) { + currentPosition = 0; + } + } + }; + } + + /** + * @return true if the supplied set is equal to the set used to instantiate this circular list, otherwise false. + */ + public boolean equalsSet(final Set inputSet) + { + return new HashSet<>(elements).equals(inputSet); + } +} diff --git a/processing/src/main/java/org/apache/druid/data/input/Firehose.java b/processing/src/main/java/org/apache/druid/data/input/Firehose.java deleted file mode 100644 index a9ff6f33159..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/Firehose.java +++ /dev/null @@ -1,97 +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.data.input; - -import org.apache.druid.guice.annotations.ExtensionPoint; -import org.apache.druid.java.util.common.parsers.ParseException; - -import javax.annotation.Nullable; -import java.io.Closeable; -import java.io.IOException; - -/** - * This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this - * abstraction. - * - * This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends - * Closeable and it is very important that the {@link #close()} method doesn't get forgotten, which is easy to do if - * this gets passed around as an Iterator. Note that {@link #close()} doesn't cut the stream of rows for Firehose users - * immediately, but rather stops the supply of new rows into internal buffers. {@link #hasMore()} and {@link #nextRow()} - * are expected to operate for some time after (or concurrently with) {@link #close()} until the buffered events (if - * any) run out. - * - * Concurrency: - * The two methods {@link #hasMore()} and {@link #nextRow()} are all called from the same thread. - * {@link #close()} might be called concurrently from a thread different from the thread calling {@link #hasMore()} - * and {@link #nextRow()}. - *

    - */ -@Deprecated -@ExtensionPoint -public interface Firehose extends Closeable -{ - /** - * Returns whether there are more rows to process. This is used to indicate that another item is immediately - * available via {@link #nextRow()}. Thus, if the stream is still available but there are no new messages on it, this - * call should block until a new message is available. - * - * If something happens such that the stream is no longer available, this should return false. - * - * @return true if and when there is another row available, false if the stream has dried up - */ - boolean hasMore() throws IOException; - - /** - * The next row available. Should only be called if hasMore returns true. - * The return value can be null which means the caller must skip this row. - * - * @return The next row - */ - @Nullable - InputRow nextRow() throws IOException; - - /** - * Returns an {@link InputRowListPlusRawValues} object containing the InputRow plus the raw, unparsed data corresponding to - * the next row available. Used in the sampler to provide the caller with information to assist in configuring a parse - * spec. If a ParseException is thrown by the parser, it should be caught and returned in the InputRowListPlusRawValues so - * we will be able to provide information on the raw row which failed to be parsed. Should only be called if hasMore - * returns true. - * - * @return an InputRowListPlusRawValues which may contain any of: an InputRow, map of the raw data, or a ParseException - */ - @Deprecated - default InputRowListPlusRawValues nextRowWithRaw() throws IOException - { - try { - return InputRowListPlusRawValues.of(nextRow(), null); - } - catch (ParseException e) { - return InputRowListPlusRawValues.of(null, e); - } - } - - /** - * Closes the "ingestion side" of the Firehose, potentially concurrently with calls to {@link #hasMore()} and {@link - * #nextRow()} being made from a different thread. {@link #hasMore()} and {@link #nextRow()} - * continue to work after close(), but since the ingestion side is closed rows will eventually run out. - */ - @Override - void close() throws IOException; -} diff --git a/processing/src/main/java/org/apache/druid/data/input/FirehoseFactory.java b/processing/src/main/java/org/apache/druid/data/input/FirehoseFactory.java deleted file mode 100644 index 6ec8657d435..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/FirehoseFactory.java +++ /dev/null @@ -1,82 +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.data.input; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.guice.annotations.ExtensionPoint; -import org.apache.druid.java.util.common.parsers.ParseException; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; - -/** - * FirehoseFactory creates a {@link Firehose} which is an interface holding onto the stream of incoming data. - * It currently provides two methods for creating a {@link Firehose} and their default implementations call each other - * for the backward compatibility. Implementations of this interface must implement one of these methods. - * - * This class is deprecated in favor of {@link InputSource} - */ -@Deprecated -@ExtensionPoint -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -public interface FirehoseFactory -{ - /** - * Initialization method that connects up the fire hose. If this method returns successfully it should be safe to - * call hasMore() on the returned Firehose (which might subsequently block). - *

    - * If this method returns null, then any attempt to call hasMore(), nextRow() and close() on the return - * value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on - * invalid configuration is preferred over returning null. - * - * @param parser an input row parser - */ - @Deprecated - default Firehose connect(T parser) throws IOException, ParseException - { - return connect(parser, null); - } - - /** - * Initialization method that connects up the fire hose. If this method returns successfully it should be safe to - * call hasMore() on the returned Firehose (which might subsequently block). - *

    - * If this method returns null, then any attempt to call hasMore(), nextRow() and close() on the return - * value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on - * invalid configuration is preferred over returning null. - * - * @param parser an input row parser - * @param temporaryDirectory a directory where temporary files are stored - */ - default Firehose connect(T parser, @Nullable File temporaryDirectory) throws IOException, ParseException - { - return connect(parser); - } - - @SuppressWarnings("unused") - @JsonIgnore - default boolean isSplittable() - { - return false; - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/SegmentsSplitHintSpec.java b/processing/src/main/java/org/apache/druid/data/input/SegmentsSplitHintSpec.java index ed59d67285a..433a6a2f995 100644 --- a/processing/src/main/java/org/apache/druid/data/input/SegmentsSplitHintSpec.java +++ b/processing/src/main/java/org/apache/druid/data/input/SegmentsSplitHintSpec.java @@ -30,11 +30,10 @@ import java.util.Objects; import java.util.function.Function; /** - * {@link SplitHintSpec} for IngestSegmentFirehoseFactory and DruidInputSource. + * {@link SplitHintSpec} for DruidInputSource. * * In DruidInputSource, this spec is converted into {@link MaxSizeSplitHintSpec}. As a result, its {@link #split} - * method is never called (IngestSegmentFirehoseFactory creates splits on its own instead of calling the - * {@code split()} method). This doesn't necessarily mean this class is deprecated in favor of the MaxSizeSplitHintSpec. + * method is never called. This doesn't necessarily mean this class is deprecated in favor of the MaxSizeSplitHintSpec. * We may want to create more optimized splits in the future. For example, segments can be split to maximize the rollup * ratio if the segments have different sets of columns or even different value ranges of columns. */ diff --git a/processing/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/processing/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 517235a99f9..ca2708700f0 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -19,10 +19,9 @@ package org.apache.druid.java.util.common.guava; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import org.apache.druid.java.util.common.RE; +import com.google.common.util.concurrent.AbstractFuture; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QueryTimeoutException; @@ -63,6 +62,7 @@ import java.util.function.Consumer; public class ParallelMergeCombiningSequence extends YieldingSequenceBase { private static final Logger LOG = new Logger(ParallelMergeCombiningSequence.class); + private static final long BLOCK_TIMEOUT = TimeUnit.NANOSECONDS.convert(500, TimeUnit.MILLISECONDS); // these values were chosen carefully via feedback from benchmarks, // see PR https://github.com/apache/druid/pull/8578 for details @@ -84,7 +84,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase private final long targetTimeNanos; private final Consumer metricsReporter; - private final CancellationGizmo cancellationGizmo; + private final CancellationFuture cancellationFuture; public ParallelMergeCombiningSequence( ForkJoinPool workerPool, @@ -114,14 +114,24 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase this.targetTimeNanos = TimeUnit.NANOSECONDS.convert(targetTimeMillis, TimeUnit.MILLISECONDS); this.queueSize = (1 << 15) / batchSize; // each queue can by default hold ~32k rows this.metricsReporter = reporter; - this.cancellationGizmo = new CancellationGizmo(); + this.cancellationFuture = new CancellationFuture(new CancellationGizmo()); } @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { if (inputSequences.isEmpty()) { - return Sequences.empty().toYielder(initValue, accumulator); + return Sequences.wrap( + Sequences.empty(), + new SequenceWrapper() + { + @Override + public void after(boolean isDone, Throwable thrown) + { + cancellationFuture.set(true); + } + } + ).toYielder(initValue, accumulator); } // we make final output queue larger than the merging queues so if downstream readers are slower to read there is // less chance of blocking the merge @@ -144,27 +154,43 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase hasTimeout, timeoutAtNanos, metricsAccumulator, - cancellationGizmo + cancellationFuture.cancellationGizmo ); workerPool.execute(mergeCombineAction); - Sequence finalOutSequence = makeOutputSequenceForQueue( - outputQueue, - hasTimeout, - timeoutAtNanos, - cancellationGizmo - ).withBaggage(() -> { - if (metricsReporter != null) { - metricsAccumulator.setTotalWallTime(System.nanoTime() - startTimeNanos); - metricsReporter.accept(metricsAccumulator.build()); - } - }); + + final Sequence finalOutSequence = Sequences.wrap( + makeOutputSequenceForQueue( + outputQueue, + hasTimeout, + timeoutAtNanos, + cancellationFuture.cancellationGizmo + ), + new SequenceWrapper() + { + @Override + public void after(boolean isDone, Throwable thrown) + { + if (isDone) { + cancellationFuture.set(true); + } else { + cancellationFuture.cancel(true); + } + if (metricsReporter != null) { + metricsAccumulator.setTotalWallTime(System.nanoTime() - startTimeNanos); + metricsReporter.accept(metricsAccumulator.build()); + } + } + } + ); return finalOutSequence.toYielder(initValue, accumulator); } - @VisibleForTesting - public CancellationGizmo getCancellationGizmo() + /** + * + */ + public CancellationFuture getCancellationFuture() { - return cancellationGizmo; + return cancellationFuture; } /** @@ -181,8 +207,6 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase return new BaseSequence<>( new BaseSequence.IteratorMaker>() { - private boolean shouldCancelOnCleanup = true; - @Override public Iterator make() { @@ -195,7 +219,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase { final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime(); if (hasTimeout && thisTimeoutNanos < 0) { - throw new QueryTimeoutException(); + throw cancellationGizmo.cancelAndThrow(new QueryTimeoutException()); } if (currentBatch != null && !currentBatch.isTerminalResult() && !currentBatch.isDrained()) { @@ -210,33 +234,32 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase } } if (currentBatch == null) { - throw new QueryTimeoutException(); + throw cancellationGizmo.cancelAndThrow(new QueryTimeoutException()); } - if (cancellationGizmo.isCancelled()) { + if (cancellationGizmo.isCanceled()) { throw cancellationGizmo.getRuntimeException(); } if (currentBatch.isTerminalResult()) { - shouldCancelOnCleanup = false; return false; } return true; } catch (InterruptedException e) { - throw new RE(e); + throw cancellationGizmo.cancelAndThrow(e); } } @Override public T next() { - if (cancellationGizmo.isCancelled()) { + if (cancellationGizmo.isCanceled()) { throw cancellationGizmo.getRuntimeException(); } if (currentBatch == null || currentBatch.isDrained() || currentBatch.isTerminalResult()) { - throw new NoSuchElementException(); + throw cancellationGizmo.cancelAndThrow(new NoSuchElementException()); } return currentBatch.next(); } @@ -246,9 +269,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase @Override public void cleanup(Iterator iterFromMake) { - if (shouldCancelOnCleanup) { - cancellationGizmo.cancel(new RuntimeException("Already closed")); - } + // nothing to cleanup } } ); @@ -338,7 +359,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase parallelTaskCount ); - QueuePusher> resultsPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); + QueuePusher resultsPusher = new QueuePusher<>(out, cancellationGizmo, hasTimeout, timeoutAt); for (Sequence s : sequences) { sequenceCursors.add(new YielderBatchedResultsCursor<>(new SequenceBatcher<>(s, batchSize), orderingFn)); @@ -367,10 +388,10 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase catch (Throwable t) { closeAllCursors(sequenceCursors); cancellationGizmo.cancel(t); - // Should be the following, but can' change due to lack of - // unit tests. - // out.offer((ParallelMergeCombiningSequence.ResultBatch) ResultBatch.TERMINAL); - out.offer(ResultBatch.TERMINAL); + // offer terminal result if queue is not full in case out is empty to allow downstream threads waiting on + // stuff to be present to stop blocking immediately. However, if the queue is full, it doesn't matter if we + // write anything because the cancellation signal has been set, which will also terminate processing. + out.offer(ResultBatch.terminal()); } } @@ -387,7 +408,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase for (List> partition : partitions) { BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); intermediaryOutputs.add(outputQueue); - QueuePusher> pusher = new QueuePusher<>(outputQueue, hasTimeout, timeoutAt); + QueuePusher pusher = new QueuePusher<>(outputQueue, cancellationGizmo, hasTimeout, timeoutAt); List> partitionCursors = new ArrayList<>(sequences.size()); for (Sequence s : partition) { @@ -415,11 +436,11 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase getPool().execute(task); } - QueuePusher> outputPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); + QueuePusher outputPusher = new QueuePusher<>(out, cancellationGizmo, hasTimeout, timeoutAt); List> intermediaryOutputsCursors = new ArrayList<>(intermediaryOutputs.size()); for (BlockingQueue> queue : intermediaryOutputs) { intermediaryOutputsCursors.add( - new BlockingQueueuBatchedResultsCursor<>(queue, orderingFn, hasTimeout, timeoutAt) + new BlockingQueueuBatchedResultsCursor<>(queue, cancellationGizmo, orderingFn, hasTimeout, timeoutAt) ); } MergeCombineActionMetricsAccumulator finalMergeMetrics = new MergeCombineActionMetricsAccumulator(); @@ -513,7 +534,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase private final PriorityQueue> pQueue; private final Ordering orderingFn; private final BinaryOperator combineFn; - private final QueuePusher> outputQueue; + private final QueuePusher outputQueue; private final T initialValue; private final int yieldAfter; private final int batchSize; @@ -523,7 +544,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase private MergeCombineAction( PriorityQueue> pQueue, - QueuePusher> outputQueue, + QueuePusher outputQueue, Ordering orderingFn, BinaryOperator combineFn, T initialValue, @@ -550,6 +571,10 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase @Override protected void compute() { + if (cancellationGizmo.isCanceled()) { + cleanup(); + return; + } try { long start = System.nanoTime(); long startCpuNanos = JvmUtils.safeGetThreadCpuTime(); @@ -608,7 +633,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase metricsAccumulator.incrementCpuTimeNanos(elapsedCpuNanos); metricsAccumulator.incrementTaskCount(); - if (!pQueue.isEmpty() && !cancellationGizmo.isCancelled()) { + if (!pQueue.isEmpty() && !cancellationGizmo.isCanceled()) { // if there is still work to be done, execute a new task with the current accumulated value to continue // combining where we left off if (!outputBatch.isDrained()) { @@ -650,29 +675,36 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase metricsAccumulator, cancellationGizmo )); - } else if (cancellationGizmo.isCancelled()) { + } else if (cancellationGizmo.isCanceled()) { // if we got the cancellation signal, go ahead and write terminal value into output queue to help gracefully // allow downstream stuff to stop - LOG.debug("cancelled after %s tasks", metricsAccumulator.getTaskCount()); + LOG.debug("canceled after %s tasks", metricsAccumulator.getTaskCount()); // make sure to close underlying cursors - closeAllCursors(pQueue); - outputQueue.offer(ResultBatch.TERMINAL); + cleanup(); } else { // if priority queue is empty, push the final accumulated value into the output batch and push it out outputBatch.add(currentCombinedValue); metricsAccumulator.incrementOutputRows(batchCounter + 1L); outputQueue.offer(outputBatch); // ... and the terminal value to indicate the blocking queue holding the values is complete - outputQueue.offer(ResultBatch.TERMINAL); + outputQueue.offer(ResultBatch.terminal()); LOG.debug("merge combine complete after %s tasks", metricsAccumulator.getTaskCount()); } } catch (Throwable t) { - closeAllCursors(pQueue); cancellationGizmo.cancel(t); - outputQueue.offer(ResultBatch.TERMINAL); + cleanup(); } } + + private void cleanup() + { + closeAllCursors(pQueue); + // offer terminal result if queue is not full in case out is empty to allow downstream threads waiting on + // stuff to be present to stop blocking immediately. However, if the queue is full, it doesn't matter if we + // write anything because the cancellation signal has been set, which will also terminate processing. + outputQueue.offer(ResultBatch.terminal()); + } } @@ -696,7 +728,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase private final List> partition; private final Ordering orderingFn; private final BinaryOperator combineFn; - private final QueuePusher> outputQueue; + private final QueuePusher outputQueue; private final int yieldAfter; private final int batchSize; private final long targetTimeNanos; @@ -707,7 +739,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase private PrepareMergeCombineInputsAction( List> partition, - QueuePusher> outputQueue, + QueuePusher outputQueue, Ordering orderingFn, BinaryOperator combineFn, int yieldAfter, @@ -744,7 +776,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase cursor.close(); } } - if (cursors.size() > 0) { + if (!cancellationGizmo.isCanceled() && !cursors.isEmpty()) { getPool().execute(new MergeCombineAction( cursors, outputQueue, @@ -758,14 +790,17 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase cancellationGizmo )); } else { - outputQueue.offer(ResultBatch.TERMINAL); + outputQueue.offer(ResultBatch.terminal()); } metricsAccumulator.setPartitionInitializedTime(System.nanoTime() - startTime); } catch (Throwable t) { closeAllCursors(partition); cancellationGizmo.cancel(t); - outputQueue.offer(ResultBatch.TERMINAL); + // offer terminal result if queue is not full in case out is empty to allow downstream threads waiting on + // stuff to be present to stop blocking immediately. However, if the queue is full, it doesn't matter if we + // write anything because the cancellation signal has been set, which will also terminate processing. + outputQueue.tryOfferTerminal(); } } } @@ -779,12 +814,14 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase { final boolean hasTimeout; final long timeoutAtNanos; - final BlockingQueue queue; - volatile E item = null; + final BlockingQueue> queue; + final CancellationGizmo gizmo; + volatile ResultBatch item = null; - QueuePusher(BlockingQueue q, boolean hasTimeout, long timeoutAtNanos) + QueuePusher(BlockingQueue> q, CancellationGizmo gizmo, boolean hasTimeout, long timeoutAtNanos) { this.queue = q; + this.gizmo = gizmo; this.hasTimeout = hasTimeout; this.timeoutAtNanos = timeoutAtNanos; } @@ -795,14 +832,16 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase boolean success = false; if (item != null) { if (hasTimeout) { - final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime(); - if (thisTimeoutNanos < 0) { + final long remainingNanos = timeoutAtNanos - System.nanoTime(); + if (remainingNanos < 0) { item = null; - throw new QueryTimeoutException("QueuePusher timed out offering data"); + throw gizmo.cancelAndThrow(new QueryTimeoutException()); } - success = queue.offer(item, thisTimeoutNanos, TimeUnit.NANOSECONDS); + final long blockTimeoutNanos = Math.min(remainingNanos, BLOCK_TIMEOUT); + success = queue.offer(item, blockTimeoutNanos, TimeUnit.NANOSECONDS); } else { - success = queue.offer(item); + queue.put(item); + success = true; } if (success) { item = null; @@ -817,7 +856,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase return item == null; } - public void offer(E item) + public void offer(ResultBatch item) { try { this.item = item; @@ -828,6 +867,11 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase throw new RuntimeException("Failed to offer result to output queue", e); } } + + public void tryOfferTerminal() + { + this.queue.offer(ResultBatch.terminal()); + } } /** @@ -837,8 +881,10 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase */ static class ResultBatch { - @SuppressWarnings("rawtypes") - static final ResultBatch TERMINAL = new ResultBatch(); + static ResultBatch terminal() + { + return new ResultBatch<>(); + } @Nullable private final Queue values; @@ -855,19 +901,16 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase public void add(E in) { - assert values != null; values.offer(in); } public E get() { - assert values != null; return values.peek(); } public E next() { - assert values != null; return values.poll(); } @@ -925,6 +968,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase Yielder> getBatchYielder() { try { + batchYielder = null; ForkJoinPool.managedBlock(this); return batchYielder; } @@ -1033,8 +1077,8 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase @Override public void initialize() { - yielder = batcher.getBatchYielder(); - resultBatch = yielder.get(); + yielder = null; + nextBatch(); } @Override @@ -1059,6 +1103,10 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase @Override public boolean block() { + if (yielder == null) { + yielder = batcher.getBatchYielder(); + resultBatch = yielder.get(); + } if (yielder.isDone()) { return true; } @@ -1073,7 +1121,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase @Override public boolean isReleasable() { - return yielder.isDone() || (resultBatch != null && !resultBatch.isDrained()); + return (yielder != null && yielder.isDone()) || (resultBatch != null && !resultBatch.isDrained()); } @Override @@ -1092,11 +1140,13 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase static class BlockingQueueuBatchedResultsCursor extends BatchedResultsCursor { final BlockingQueue> queue; + final CancellationGizmo gizmo; final boolean hasTimeout; final long timeoutAtNanos; BlockingQueueuBatchedResultsCursor( BlockingQueue> blockingQueue, + CancellationGizmo cancellationGizmo, Ordering ordering, boolean hasTimeout, long timeoutAtNanos @@ -1104,6 +1154,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase { super(ordering); this.queue = blockingQueue; + this.gizmo = cancellationGizmo; this.hasTimeout = hasTimeout; this.timeoutAtNanos = timeoutAtNanos; } @@ -1142,17 +1193,18 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase { if (resultBatch == null || resultBatch.isDrained()) { if (hasTimeout) { - final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime(); - if (thisTimeoutNanos < 0) { - resultBatch = ResultBatch.TERMINAL; - throw new QueryTimeoutException("BlockingQueue cursor timed out waiting for data"); + final long remainingNanos = timeoutAtNanos - System.nanoTime(); + if (remainingNanos < 0) { + resultBatch = ResultBatch.terminal(); + throw gizmo.cancelAndThrow(new QueryTimeoutException()); } - resultBatch = queue.poll(thisTimeoutNanos, TimeUnit.NANOSECONDS); + final long blockTimeoutNanos = Math.min(remainingNanos, BLOCK_TIMEOUT); + resultBatch = queue.poll(blockTimeoutNanos, TimeUnit.NANOSECONDS); } else { resultBatch = queue.take(); } } - return resultBatch != null; + return resultBatch != null && !resultBatch.isDrained(); } @Override @@ -1164,35 +1216,91 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase } // if we can get a result immediately without blocking, also no need to block resultBatch = queue.poll(); - return resultBatch != null; + return resultBatch != null && !resultBatch.isDrained(); } } /** - * Token to allow any {@link RecursiveAction} signal the others and the output sequence that something bad happened - * and processing should cancel, such as a timeout or connection loss. + * Token used to stop internal parallel processing across all tasks in the merge pool. Allows any + * {@link RecursiveAction} signal the others and the output sequence that something bad happened and + * processing should cancel, such as a timeout, error, or connection loss. */ - static class CancellationGizmo + public static class CancellationGizmo { private final AtomicReference throwable = new AtomicReference<>(null); + RuntimeException cancelAndThrow(Throwable t) + { + throwable.compareAndSet(null, t); + return wrapRuntimeException(t); + } + void cancel(Throwable t) { throwable.compareAndSet(null, t); } - boolean isCancelled() + boolean isCanceled() { return throwable.get() != null; } RuntimeException getRuntimeException() { - Throwable ex = throwable.get(); - if (ex instanceof RuntimeException) { - return (RuntimeException) ex; + return wrapRuntimeException(throwable.get()); + } + + private static RuntimeException wrapRuntimeException(Throwable t) + { + if (t instanceof RuntimeException) { + return (RuntimeException) t; } - return new RE(ex); + return new RuntimeException(t); + } + } + + /** + * {@link com.google.common.util.concurrent.ListenableFuture} that allows {@link ParallelMergeCombiningSequence} to be + * registered with {@link org.apache.druid.query.QueryWatcher#registerQueryFuture} to participate in query + * cancellation or anything else that has a need to watch the activity on the merge pool. Wraps a + * {@link CancellationGizmo} to allow for external threads to signal cancellation of parallel processing on the pool + * by triggering {@link CancellationGizmo#cancel(Throwable)} whenever {@link #cancel(boolean)} is called. + * + * This is not used internally by workers on the pool in favor of using the much simpler {@link CancellationGizmo} + * directly instead. + */ + public static class CancellationFuture extends AbstractFuture + { + private final CancellationGizmo cancellationGizmo; + + public CancellationFuture(CancellationGizmo cancellationGizmo) + { + this.cancellationGizmo = cancellationGizmo; + } + + public CancellationGizmo getCancellationGizmo() + { + return cancellationGizmo; + } + + @Override + public boolean set(Boolean value) + { + return super.set(value); + } + + @Override + public boolean setException(Throwable throwable) + { + cancellationGizmo.cancel(throwable); + return super.setException(throwable); + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) + { + cancellationGizmo.cancel(new RuntimeException("Sequence canceled")); + return super.cancel(mayInterruptIfRunning); } } @@ -1308,8 +1416,8 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase */ static class MergeCombineMetricsAccumulator { - List partitionMetrics; - MergeCombineActionMetricsAccumulator mergeMetrics; + List partitionMetrics = Collections.emptyList(); + MergeCombineActionMetricsAccumulator mergeMetrics = new MergeCombineActionMetricsAccumulator(); private long totalWallTime; @@ -1343,8 +1451,8 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase // partition long totalPoolTasks = 1 + 1 + partitionMetrics.size(); - long fastestPartInitialized = partitionMetrics.size() > 0 ? Long.MAX_VALUE : mergeMetrics.getPartitionInitializedtime(); - long slowestPartInitialied = partitionMetrics.size() > 0 ? Long.MIN_VALUE : mergeMetrics.getPartitionInitializedtime(); + long fastestPartInitialized = !partitionMetrics.isEmpty() ? Long.MAX_VALUE : mergeMetrics.getPartitionInitializedtime(); + long slowestPartInitialied = !partitionMetrics.isEmpty() ? Long.MIN_VALUE : mergeMetrics.getPartitionInitializedtime(); // accumulate input row count, cpu time, and total number of tasks from each partition for (MergeCombineActionMetricsAccumulator partition : partitionMetrics) { diff --git a/processing/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java b/processing/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java index bfb1dc79d78..b02de3ce493 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java @@ -20,8 +20,11 @@ package org.apache.druid.java.util.common.jackson; import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.JsonToken; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.JsonSerializer; @@ -122,6 +125,40 @@ public final class JacksonUtils } } + /** + * Reads an object using the {@link JsonParser}. It reuses the provided {@link DeserializationContext} which offers + * better performance that calling {@link JsonParser#readValueAs(Class)} because it avoids re-creating the {@link DeserializationContext} + * for each readValue call + */ + @Nullable + public static T readObjectUsingDeserializationContext( + final JsonParser jp, + final DeserializationContext deserializationContext, + final Class clazz + ) throws IOException + { + if (jp.currentToken() == JsonToken.VALUE_NULL) { + return null; + } + return deserializationContext.readValue(jp, clazz); + } + + /** + * @see #readObjectUsingDeserializationContext(JsonParser, DeserializationContext, Class) + */ + @Nullable + public static Object readObjectUsingDeserializationContext( + final JsonParser jp, + final DeserializationContext deserializationContext, + final JavaType javaType + ) throws IOException + { + if (jp.currentToken() == JsonToken.VALUE_NULL) { + return null; + } + return deserializationContext.readValue(jp, javaType); + } + /** * Convert the given object to an array of bytes. Use when the object is * known serializable so that the Jackson exception can be suppressed. diff --git a/processing/src/main/java/org/apache/druid/query/Druids.java b/processing/src/main/java/org/apache/druid/query/Druids.java index 7579843636d..824444410c0 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -824,7 +824,6 @@ public class Druids private long limit; private DimFilter dimFilter; private List columns = new ArrayList<>(); - private Boolean legacy; private ScanQuery.Order order; private List orderBy; private List columnTypes = null; @@ -843,7 +842,6 @@ public class Druids orderBy, dimFilter, columns, - legacy, context, columnTypes ); @@ -861,7 +859,6 @@ public class Druids .limit(query.getScanRowsLimit()) .filters(query.getFilter()) .columns(query.getColumns()) - .legacy(query.isLegacy()) .context(query.getContext()) .orderBy(query.getOrderBys()) .columnTypes(query.getColumnTypes()); @@ -959,12 +956,6 @@ public class Druids return this; } - public ScanQueryBuilder legacy(Boolean legacy) - { - this.legacy = legacy; - return this; - } - public ScanQueryBuilder order(ScanQuery.Order order) { this.order = order; diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index fa394beec43..978b4922615 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -88,6 +88,11 @@ public abstract class QueryToolChest + * Jackson's default implementation of deserialization is usually optimised and this method should be overriden + * only if there is a functional requirement of so. The method must be benchmarked in isolation, without other portions + * of the query engine executing as modifying this method can alter the performance of queries where deserializing is + * a major portion of the execution. */ public ObjectMapper decorateObjectMapper(final ObjectMapper objectMapper, final QueryType query) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java index 2034ba21a5b..c4c9a7875ef 100755 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java @@ -28,6 +28,8 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.BaseLongColumnValueSelector; +import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; @@ -35,6 +37,8 @@ import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.FloatColumnSelector; import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.Types; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.apache.druid.segment.virtual.ExpressionSelectors; @@ -428,4 +432,26 @@ public class AggregatorUtil .array(); }); } + + /** + * Whether a simple numeric aggregator should use {@link BaseObjectColumnValueSelector#getObject()}, and coerce the + * result to number, rather than using a primitive method like {@link BaseLongColumnValueSelector#getLong()}. + * + * @param fieldName field name, or null if the aggregator is expression-based + * @param columnSelectorFactory column selector factory + */ + public static boolean shouldUseObjectColumnAggregatorWrapper( + @Nullable final String fieldName, + final ColumnSelectorFactory columnSelectorFactory + ) + { + if (fieldName != null) { + ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName); + + // STRING can be coerced to a number. COMPLEX types can be subclasses of Number (or subclasses of some type + // that is coercible to a number.) + return Types.is(capabilities, ValueType.STRING) || Types.is(capabilities, ValueType.COMPLEX); + } + return false; + } } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingAggregator.java index c1b4b409023..6b0d0fb4969 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingAggregator.java @@ -23,7 +23,7 @@ import javax.annotation.Nullable; /** * An Aggregator that delegates everything. It is used by Aggregator wrappers e.g. - * {@link StringColumnDoubleAggregatorWrapper} that modify some behavior of a delegate. + * {@link ObjectColumnDoubleAggregatorWrapper} that modify some behavior of a delegate. */ public abstract class DelegatingAggregator implements Aggregator { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingBufferAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingBufferAggregator.java index 9b1aa8086b2..87718f33108 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingBufferAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingBufferAggregator.java @@ -26,7 +26,7 @@ import java.nio.ByteBuffer; /** * A BufferAggregator that delegates everything. It is used by BufferAggregator wrappers e.g. - * {@link StringColumnDoubleBufferAggregatorWrapper} that modify some behavior of a delegate. + * {@link ObjectColumnDoubleBufferAggregatorWrapper} that modify some behavior of a delegate. */ public abstract class DelegatingBufferAggregator implements BufferAggregator { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java index c9928f828f7..d85d91936d7 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.common.config.NullHandling; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.segment.BaseNullableColumnValueSelector; +import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ColumnType; @@ -38,8 +39,9 @@ import org.apache.druid.segment.vector.VectorValueSelector; * values to be aggregated are null values, or if no values are aggregated at all. If any of the values are non-null, * the result will be the aggregated value of the non-null values. * - * This superclass should only be extended by aggregators that read primitive numbers. It implements logic that is - * not valid for non-numeric selector methods such as {@link ColumnValueSelector#getObject()}. + * Aggregators that use {@link ColumnValueSelector#getObject()} must override + * {@link #useGetObject(ColumnSelectorFactory)}. Otherwise, the logic in this class is not correct for + * non-numeric selectors. * * @see BaseNullableColumnValueSelector#isNull() for why this only works in the numeric case */ @@ -51,16 +53,18 @@ public abstract class NullableNumericAggregatorFactory combiner = makeAggregateCombiner(); + return NullHandling.sqlCompatible() ? new NullableNumericAggregateCombiner<>(combiner) : combiner; } @Override @@ -85,6 +89,23 @@ public abstract class NullableNumericAggregatorFactory objectSelector = (BaseObjectColumnValueSelector) selector; + return () -> objectSelector.getObject() == null; + } else { + return selector; + } + } + // ---- ABSTRACT METHODS BELOW ------ /** @@ -94,6 +115,17 @@ public abstract class NullableNumericAggregatorFactory delegateBuilder, double nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnDoubleBufferAggregatorWrapper.java b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnDoubleBufferAggregatorWrapper.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/aggregation/StringColumnDoubleBufferAggregatorWrapper.java rename to processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnDoubleBufferAggregatorWrapper.java index fb58ad5cc49..f50a6371f93 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnDoubleBufferAggregatorWrapper.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnDoubleBufferAggregatorWrapper.java @@ -29,15 +29,15 @@ import java.util.List; import java.util.function.Function; /** - * This class can be used to wrap Double BufferAggregator that consume double type columns to handle String type. + * This class can be used to wrap Double BufferAggregator that consume double type columns to handle Object type. */ -public class StringColumnDoubleBufferAggregatorWrapper extends DelegatingBufferAggregator +public class ObjectColumnDoubleBufferAggregatorWrapper extends DelegatingBufferAggregator { private final BaseObjectColumnValueSelector selector; private final double nullValue; private final SettableValueDoubleColumnValueSelector doubleSelector; - public StringColumnDoubleBufferAggregatorWrapper( + public ObjectColumnDoubleBufferAggregatorWrapper( BaseObjectColumnValueSelector selector, Function delegateBuilder, double nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatAggregatorWrapper.java b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatAggregatorWrapper.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatAggregatorWrapper.java rename to processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatAggregatorWrapper.java index bb7cd65c409..0f738b4bc0d 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatAggregatorWrapper.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatAggregatorWrapper.java @@ -28,15 +28,15 @@ import java.util.List; import java.util.function.Function; /** - * This class can be used to wrap Float Aggregator that consume float type columns to handle String type. + * This class can be used to wrap Float Aggregator that consume float type columns to handle Object type. */ -public class StringColumnFloatAggregatorWrapper extends DelegatingAggregator +public class ObjectColumnFloatAggregatorWrapper extends DelegatingAggregator { private final BaseObjectColumnValueSelector selector; private final float nullValue; private final SettableValueFloatColumnValueSelector floatSelector; - public StringColumnFloatAggregatorWrapper( + public ObjectColumnFloatAggregatorWrapper( BaseObjectColumnValueSelector selector, Function delegateBuilder, float nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatBufferAggregatorWrapper.java b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatBufferAggregatorWrapper.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatBufferAggregatorWrapper.java rename to processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatBufferAggregatorWrapper.java index 7c1c5e54881..1c3725e968a 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatBufferAggregatorWrapper.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatBufferAggregatorWrapper.java @@ -29,15 +29,15 @@ import java.util.List; import java.util.function.Function; /** - * This class can be used to wrap Float BufferAggregator that consume float type columns to handle String type. + * This class can be used to wrap Float BufferAggregator that consume float type columns to handle Object type. */ -public class StringColumnFloatBufferAggregatorWrapper extends DelegatingBufferAggregator +public class ObjectColumnFloatBufferAggregatorWrapper extends DelegatingBufferAggregator { private final BaseObjectColumnValueSelector selector; private final float nullValue; private final SettableValueFloatColumnValueSelector floatSelector; - public StringColumnFloatBufferAggregatorWrapper( + public ObjectColumnFloatBufferAggregatorWrapper( BaseObjectColumnValueSelector selector, Function delegateBuilder, float nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongAggregatorWrapper.java b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongAggregatorWrapper.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongAggregatorWrapper.java rename to processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongAggregatorWrapper.java index d218ab38cfb..01b571a4e14 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongAggregatorWrapper.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongAggregatorWrapper.java @@ -28,15 +28,15 @@ import java.util.List; import java.util.function.Function; /** - * This class can be used to wrap Long Aggregator that consume long type columns to handle String type. + * This class can be used to wrap Long Aggregator that consume long type columns to handle Object type. */ -public class StringColumnLongAggregatorWrapper extends DelegatingAggregator +public class ObjectColumnLongAggregatorWrapper extends DelegatingAggregator { private final BaseObjectColumnValueSelector selector; private final long nullValue; private final SettableValueLongColumnValueSelector longSelector; - public StringColumnLongAggregatorWrapper( + public ObjectColumnLongAggregatorWrapper( BaseObjectColumnValueSelector selector, Function delegateBuilder, long nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongBufferAggregatorWrapper.java b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongBufferAggregatorWrapper.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongBufferAggregatorWrapper.java rename to processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongBufferAggregatorWrapper.java index ad2e6c2cbc5..831c9a8f26e 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongBufferAggregatorWrapper.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongBufferAggregatorWrapper.java @@ -29,15 +29,15 @@ import java.util.List; import java.util.function.Function; /** - * This class can be used to wrap Long BufferAggregator that consume long type columns to handle String type. + * This class can be used to wrap Long BufferAggregator that consume long type columns to handle Object type. */ -public class StringColumnLongBufferAggregatorWrapper extends DelegatingBufferAggregator +public class ObjectColumnLongBufferAggregatorWrapper extends DelegatingBufferAggregator { private final BaseObjectColumnValueSelector selector; private final long nullValue; private final SettableValueLongColumnValueSelector longSelector; - public StringColumnLongBufferAggregatorWrapper( + public ObjectColumnLongBufferAggregatorWrapper( BaseObjectColumnValueSelector selector, Function delegateBuilder, long nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java index 324a10bb4d1..0fa96e226ea 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java @@ -31,11 +31,8 @@ import org.apache.druid.segment.BaseDoubleColumnValueSelector; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.Types; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -86,8 +83,8 @@ public abstract class SimpleDoubleAggregatorFactory extends NullableNumericAggre @Override protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnDoubleAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnDoubleAggregatorWrapper( selector, SimpleDoubleAggregatorFactory.this::buildAggregator, nullValue() @@ -103,8 +100,8 @@ public abstract class SimpleDoubleAggregatorFactory extends NullableNumericAggre ColumnValueSelector selector ) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnDoubleBufferAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnDoubleBufferAggregatorWrapper( selector, SimpleDoubleAggregatorFactory.this::buildBufferAggregator, nullValue() @@ -131,13 +128,10 @@ public abstract class SimpleDoubleAggregatorFactory extends NullableNumericAggre return AggregatorUtil.makeVectorValueSelector(columnSelectorFactory, fieldName, expression, fieldExpression); } - private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory columnSelectorFactory) + @Override + protected boolean useGetObject(ColumnSelectorFactory columnSelectorFactory) { - if (fieldName != null) { - ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName); - return Types.is(capabilities, ValueType.STRING); - } - return false; + return AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, columnSelectorFactory); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java index 7633d39d972..5268c454ce1 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java @@ -31,10 +31,7 @@ import org.apache.druid.segment.BaseFloatColumnValueSelector; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.Types; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -76,8 +73,8 @@ public abstract class SimpleFloatAggregatorFactory extends NullableNumericAggreg @Override protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnFloatAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnFloatAggregatorWrapper( selector, SimpleFloatAggregatorFactory.this::buildAggregator, nullValue() @@ -93,8 +90,8 @@ public abstract class SimpleFloatAggregatorFactory extends NullableNumericAggreg ColumnValueSelector selector ) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnFloatBufferAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnFloatBufferAggregatorWrapper( selector, SimpleFloatAggregatorFactory.this::buildBufferAggregator, nullValue() @@ -121,6 +118,12 @@ public abstract class SimpleFloatAggregatorFactory extends NullableNumericAggreg return AggregatorUtil.makeVectorValueSelector(columnSelectorFactory, fieldName, expression, fieldExpression); } + @Override + protected boolean useGetObject(ColumnSelectorFactory columnSelectorFactory) + { + return AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, columnSelectorFactory); + } + @Override public Object deserialize(Object object) { @@ -233,15 +236,6 @@ public abstract class SimpleFloatAggregatorFactory extends NullableNumericAggreg return AggregatorUtil.canVectorize(columnInspector, fieldName, expression, fieldExpression); } - private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory columnSelectorFactory) - { - if (fieldName != null) { - ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName); - return Types.is(capabilities, ValueType.STRING); - } - return false; - } - protected abstract float nullValue(); protected abstract Aggregator buildAggregator(BaseFloatColumnValueSelector selector); diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java index 173a9cb229d..c4bc5307ed4 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java @@ -31,10 +31,7 @@ import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.Types; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -82,8 +79,8 @@ public abstract class SimpleLongAggregatorFactory extends NullableNumericAggrega @Override protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnLongAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnLongAggregatorWrapper( selector, SimpleLongAggregatorFactory.this::buildAggregator, nullValue() @@ -99,8 +96,8 @@ public abstract class SimpleLongAggregatorFactory extends NullableNumericAggrega ColumnValueSelector selector ) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnLongBufferAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnLongBufferAggregatorWrapper( selector, SimpleLongAggregatorFactory.this::buildBufferAggregator, nullValue() @@ -127,6 +124,12 @@ public abstract class SimpleLongAggregatorFactory extends NullableNumericAggrega return AggregatorUtil.makeVectorValueSelector(columnSelectorFactory, fieldName, expression, fieldExpression); } + @Override + protected boolean useGetObject(ColumnSelectorFactory columnSelectorFactory) + { + return AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, columnSelectorFactory); + } + @Override public Object deserialize(Object object) { @@ -236,15 +239,6 @@ public abstract class SimpleLongAggregatorFactory extends NullableNumericAggrega return AggregatorUtil.canVectorize(columnInspector, fieldName, expression, fieldExpression); } - private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory columnSelectorFactory) - { - if (fieldName != null) { - ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName); - return Types.is(capabilities, ValueType.STRING); - } - return false; - } - protected abstract long nullValue(); protected abstract Aggregator buildAggregator(BaseLongColumnValueSelector selector); diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index c1c55a826b1..c90b651fffa 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -21,8 +21,8 @@ package org.apache.druid.query.aggregation.hyperloglog; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.DruidException; import org.apache.druid.hll.HyperLogLogCollector; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.query.aggregation.AggregateCombiner; @@ -107,12 +107,8 @@ public class HyperUniquesAggregatorFactory extends AggregatorFactory if (selector instanceof NilColumnValueSelector) { return NoopAggregator.instance(); } - final Class classOfObject = selector.classOfObject(); - if (classOfObject.equals(Object.class) || HyperLogLogCollector.class.isAssignableFrom(classOfObject)) { - return new HyperUniquesAggregator(selector); - } - - throw new IAE("Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, classOfObject); + validateInputs(metricFactory.getColumnCapabilities(fieldName)); + return new HyperUniquesAggregator(selector); } @Override @@ -122,25 +118,43 @@ public class HyperUniquesAggregatorFactory extends AggregatorFactory if (selector instanceof NilColumnValueSelector) { return NoopBufferAggregator.instance(); } - final Class classOfObject = selector.classOfObject(); - if (classOfObject.equals(Object.class) || HyperLogLogCollector.class.isAssignableFrom(classOfObject)) { - return new HyperUniquesBufferAggregator(selector); - } - - throw new IAE("Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, classOfObject); + validateInputs(metricFactory.getColumnCapabilities(fieldName)); + return new HyperUniquesBufferAggregator(selector); } @Override public VectorAggregator factorizeVector(final VectorColumnSelectorFactory selectorFactory) { - final ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(fieldName); - if (!Types.is(capabilities, ValueType.COMPLEX)) { + final ColumnCapabilities columnCapabilities = selectorFactory.getColumnCapabilities(fieldName); + if (!Types.is(columnCapabilities, ValueType.COMPLEX)) { return NoopVectorAggregator.instance(); } else { + validateInputs(columnCapabilities); return new HyperUniquesVectorAggregator(selectorFactory.makeObjectSelector(fieldName)); } } + /** + * Validates whether the aggregator supports the input column type. + * Supported column types are complex types of hyperUnique, preComputedHyperUnique, as well as UNKNOWN_COMPLEX. + * @param capabilities + */ + private void validateInputs(@Nullable ColumnCapabilities capabilities) + { + if (capabilities != null) { + final ColumnType type = capabilities.toColumnType(); + if (!(ColumnType.UNKNOWN_COMPLEX.equals(type) || TYPE.equals(type) || PRECOMPUTED_TYPE.equals(type))) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build( + "Using aggregator [%s] is not supported for complex columns with type [%s].", + getIntermediateType().getComplexTypeName(), + type + ); + } + } + } + @Override public boolean canVectorize(ColumnInspector columnInspector) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java index dad5cfd98b3..ff863f764b7 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java @@ -20,6 +20,7 @@ package org.apache.druid.query.filter; import com.google.common.base.Function; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.RangeSet; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.timeline.partition.ShardSpec; @@ -116,13 +117,19 @@ public class DimFilterUtils final Map>> dimensionRangeCache ) { + if (dimFilter == null) { + // ImmutableSet retains order from "input". + return ImmutableSet.copyOf(input); + } + + // LinkedHashSet retains order from "input". Set retSet = new LinkedHashSet<>(); for (T obj : input) { ShardSpec shard = converter.apply(obj); boolean include = true; - if (dimFilter != null && shard != null) { + if (shard != null) { Map> filterDomain = new HashMap<>(); List dimensions = shard.getDomainDimensions(); for (String dimension : dimensions) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java index 9950695f28c..dbd6f2869a7 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java @@ -293,7 +293,6 @@ public class GroupByQueryConfig return vectorize; } - @SuppressWarnings("unused") public boolean isIntermediateResultAsMapCompat() { return intermediateResultAsMapCompat; diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index d69e09c9ff0..a4520524968 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -19,23 +19,14 @@ package org.apache.druid.query.groupby; -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonToken; -import com.fasterxml.jackson.core.ObjectCodec; import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.JsonDeserializer; -import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializerProvider; -import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.base.Supplier; import com.google.common.collect.Lists; import com.google.inject.Inject; -import org.apache.druid.data.input.Row; import org.apache.druid.error.DruidException; import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.MemoryAllocatorFactory; @@ -51,7 +42,6 @@ import org.apache.druid.java.util.common.guava.MappedSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.DataSource; import org.apache.druid.query.FrameSignaturePair; @@ -82,7 +72,6 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; import java.io.Closeable; -import java.io.IOException; import java.util.ArrayList; import java.util.BitSet; import java.util.Comparator; @@ -108,6 +97,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest queryConfigSupplier, GroupByQueryMetricsFactory queryMetricsFactory, @Merging GroupByResourcesReservationPool groupByResourcesReservationPool ) { this.groupingEngine = groupingEngine; + this.queryConfig = queryConfigSupplier.get(); this.queryMetricsFactory = queryMetricsFactory; this.groupByResourcesReservationPool = groupByResourcesReservationPool; } @@ -450,96 +443,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest serializer = new JsonSerializer() - { - @Override - public void serialize( - final ResultRow resultRow, - final JsonGenerator jg, - final SerializerProvider serializers - ) throws IOException - { - if (resultAsArray) { - JacksonUtils.writeObjectUsingSerializerProvider(jg, serializers, resultRow.getArray()); - } else { - JacksonUtils.writeObjectUsingSerializerProvider(jg, serializers, resultRow.toMapBasedRow(query)); - } - } - }; - - // Deserializer that can deserialize either array- or map-based rows. - final JsonDeserializer deserializer = new JsonDeserializer() - { - final Class[] dimensionClasses = createDimensionClasses(query); - boolean containsComplexDimensions = query.getDimensions() - .stream() - .anyMatch( - dimensionSpec -> dimensionSpec.getOutputType().is(ValueType.COMPLEX) - ); - - @Override - public ResultRow deserialize(final JsonParser jp, final DeserializationContext ctxt) throws IOException - { - if (jp.isExpectedStartObjectToken()) { - final Row row = jp.readValueAs(Row.class); - final ResultRow resultRow = ResultRow.fromLegacyRow(row, query); - if (containsComplexDimensions) { - final List queryDimensions = query.getDimensions(); - for (int i = 0; i < queryDimensions.size(); ++i) { - if (queryDimensions.get(i).getOutputType().is(ValueType.COMPLEX)) { - final int dimensionIndexInResultRow = query.getResultRowDimensionStart() + i; - resultRow.set( - dimensionIndexInResultRow, - objectMapper.convertValue( - resultRow.get(dimensionIndexInResultRow), - dimensionClasses[i] - ) - ); - } - } - } - return resultRow; - } else { - Object[] objectArray = new Object[query.getResultRowSizeWithPostAggregators()]; - - if (!jp.isExpectedStartArrayToken()) { - throw DruidException.defensive("Expected start token, received [%s]", jp.currentToken()); - } - - ObjectCodec codec = jp.getCodec(); - - jp.nextToken(); - - int numObjects = 0; - while (jp.currentToken() != JsonToken.END_ARRAY) { - if (numObjects >= query.getResultRowDimensionStart() && numObjects < query.getResultRowAggregatorStart()) { - objectArray[numObjects] = codec.readValue(jp, dimensionClasses[numObjects - query.getResultRowDimensionStart()]); - } else { - objectArray[numObjects] = codec.readValue(jp, Object.class); - } - jp.nextToken(); - ++numObjects; - } - return ResultRow.of(objectArray); - } - } - }; - - class GroupByResultRowModule extends SimpleModule - { - private GroupByResultRowModule() - { - addSerializer(ResultRow.class, serializer); - addDeserializer(ResultRow.class, deserializer); - } - } - - final ObjectMapper newObjectMapper = objectMapper.copy(); - newObjectMapper.registerModule(new GroupByResultRowModule()); - return newObjectMapper; + return ResultRowObjectMapperDecoratorUtil.decorateObjectMapper(objectMapper, query, queryConfig); } @Override @@ -828,7 +732,11 @@ public class GroupByQueryQueryToolChest extends QueryToolChest deserializer = getDeserializer(baseObjectMapper, query, groupByQueryConfig); + final JsonSerializer serializer = getSerializer(query, groupByQueryConfig); + if (deserializer == null && serializer == null) { + return baseObjectMapper; + } + + final ObjectMapper decoratedObjectMapper = baseObjectMapper.copy(); + class GroupByResultRowModule extends SimpleModule + { + private GroupByResultRowModule() + { + if (serializer != null) { + addSerializer(ResultRow.class, serializer); + } + if (deserializer != null) { + addDeserializer(ResultRow.class, deserializer); + } + } + } + decoratedObjectMapper.registerModule(new GroupByResultRowModule()); + return decoratedObjectMapper; + } + + /** + * Returns a deserializer required to for the result rows of the provided query. It returns null if no special + * deserialization is required, and type-unaware generic java objects are sufficient. + */ + @Nullable + private static JsonDeserializer getDeserializer( + final ObjectMapper objectMapper, + final GroupByQuery query, + final GroupByQueryConfig groupByQueryConfig + ) + { + final boolean resultAsArray = query.context().getBoolean(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, false); + final boolean intermediateCompatMode = groupByQueryConfig.isIntermediateResultAsMapCompat(); + final boolean arrayBasedRows = resultAsArray && !intermediateCompatMode; + final boolean dimensionsRequireConversion = query.getDimensions() + .stream() + .anyMatch( + dimensionSpec -> dimensionRequiresConversion(dimensionSpec.getOutputType()) + ); + + // Most common case - when array based rows are used, and grouping is done on primitive/array/json types + if (arrayBasedRows && !dimensionsRequireConversion) { + // We can assume ResultRow are serialized and deserialized as arrays. No need for special decoration, + // and we can save the overhead of making a copy of the ObjectMapper + return null; + } else if (!arrayBasedRows && !dimensionsRequireConversion) { + // We have to deserialize map based rows, however we don't have to deserialize the dimensions individually + // Returns a deserializer that can deserialize both map and array based rows simultaneously + return new JsonDeserializer() + { + @Override + public ResultRow deserialize(final JsonParser jp, final DeserializationContext ctxt) throws IOException + { + if (jp.isExpectedStartObjectToken()) { + final Row row = jp.readValueAs(Row.class); + return ResultRow.fromLegacyRow(row, query); + } else { + return ResultRow.of(jp.readValueAs(Object[].class)); + } + } + }; + + } else { + // Dimensions need to be serialized individually because some of them require conversion to specialized types + return new JsonDeserializer() + { + final JavaType[] javaTypes = createJavaTypesForResultRow(query); + + @Override + public ResultRow deserialize(final JsonParser jp, final DeserializationContext ctxt) throws IOException + { + if (jp.isExpectedStartObjectToken()) { + final Row row = jp.readValueAs(Row.class); + final ResultRow resultRow = ResultRow.fromLegacyRow(row, query); + + final List queryDimensions = query.getDimensions(); + for (int i = 0; i < queryDimensions.size(); ++i) { + if (dimensionRequiresConversion(queryDimensions.get(i).getOutputType())) { + final int dimensionIndexInResultRow = query.getResultRowDimensionStart() + i; + resultRow.set( + dimensionIndexInResultRow, + objectMapper.convertValue( + resultRow.get(dimensionIndexInResultRow), + javaTypes[dimensionIndexInResultRow] + ) + ); + } + } + + return resultRow; + } else { + if (!jp.isExpectedStartArrayToken()) { + throw DruidException.defensive("Expected start token, received [%s]", jp.currentToken()); + } + + Object[] objectArray = new Object[query.getResultRowSizeWithPostAggregators()]; + int index = 0; + + while (jp.nextToken() != JsonToken.END_ARRAY) { + objectArray[index] = JacksonUtils.readObjectUsingDeserializationContext(jp, ctxt, javaTypes[index]); + ++index; + } + + return ResultRow.of(objectArray); + } + } + }; + } + } + + /** + * Returns a legacy mode aware serialiazer that serializes the result rows as arrays or maps depending on the query + * configuration + */ + @Nullable + private static JsonSerializer getSerializer( + final GroupByQuery query, + final GroupByQueryConfig groupByQueryConfig + ) + { + final boolean resultAsArray = query.context().getBoolean(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, false); + final boolean intermediateCompatMode = groupByQueryConfig.isIntermediateResultAsMapCompat(); + final boolean arrayBasedRows = resultAsArray && !intermediateCompatMode; + if (arrayBasedRows) { + return null; + } else { + if (resultAsArray) { + return new JsonSerializer() + { + @Override + public void serialize(ResultRow resultRow, JsonGenerator jsonGenerator, SerializerProvider serializerProvider) + throws IOException + { + JacksonUtils.writeObjectUsingSerializerProvider(jsonGenerator, serializerProvider, resultRow.getArray()); + } + }; + + } else { + return new JsonSerializer() + { + @Override + public void serialize(ResultRow resultRow, JsonGenerator jsonGenerator, SerializerProvider serializerProvider) + throws IOException + { + JacksonUtils.writeObjectUsingSerializerProvider( + jsonGenerator, + serializerProvider, + resultRow.toMapBasedRow(query) + ); + } + }; + } + } + } + + /** + * Returns true if the dimension needs to be converted from generic Java objects to the specialized column type. It involves all + * complex types, except for JSON types. JSON types are special in a way that they can work with the generic java objects + * without any conversion + */ + private static boolean dimensionRequiresConversion(final ColumnType dimensionType) + { + return dimensionType.is(ValueType.COMPLEX) && !ColumnType.NESTED_DATA.equals(dimensionType); + } + + /** + * Creates java types for deserializing the result row. For timestamp, aggregators and post-aggregators, it resorts to + * {@code Object.class}. For dimensions requiring conversion (check {@link #dimensionRequiresConversion(ColumnType)}), + * it returns the java type for the associated class of the complex object. + */ + private static JavaType[] createJavaTypesForResultRow(final GroupByQuery groupByQuery) + { + final TypeFactory typeFactory = TypeFactory.defaultInstance(); + final JavaType[] javaTypes = new JavaType[groupByQuery.getResultRowSizeWithPostAggregators()]; + final List dimensions = groupByQuery.getDimensions(); + for (int i = 0; i < groupByQuery.getResultRowSizeWithPostAggregators(); ++i) { + if (i >= groupByQuery.getResultRowDimensionStart() && i < groupByQuery.getResultRowAggregatorStart()) { + DimensionSpec dimension = dimensions.get(i - groupByQuery.getResultRowDimensionStart()); + ColumnType dimensionType = dimensions.get(i - groupByQuery.getResultRowDimensionStart()).getOutputType(); + if (dimensionType.is(ValueType.COMPLEX)) { + //noinspection rawtypes + NullableTypeStrategy nullableTypeStrategy = dimensionType.getNullableStrategy(); + if (!nullableTypeStrategy.groupable()) { + throw DruidException.defensive( + "Ungroupable dimension [%s] with type [%s] found in the query.", + dimension, + dimensionType + ); + } + javaTypes[i] = typeFactory.constructType(nullableTypeStrategy.getClazz()); + } else { + javaTypes[i] = typeFactory.constructType(Object.class); + } + } else { + javaTypes[i] = typeFactory.constructType(Object.class); + } + } + return javaTypes; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index da8a0e04623..0e73d5db6f4 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -22,7 +22,6 @@ package org.apache.druid.query.groupby.epinephelinae; import com.fasterxml.jackson.annotation.JsonValue; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonToken; -import com.fasterxml.jackson.core.ObjectCodec; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.ObjectMapper; @@ -45,6 +44,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.guava.Accumulator; import org.apache.druid.java.util.common.guava.Comparators; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.ColumnSelectorPlus; import org.apache.druid.query.DimensionComparisonUtils; @@ -1378,7 +1378,6 @@ public class RowBasedGrouperHelper } jp.nextToken(); - final ObjectCodec codec = jp.getCodec(); final int timestampAdjustment = includeTimestamp ? 1 : 0; final int dimsToRead = timestampAdjustment + serdeHelpers.length; int dimsReadSoFar = 0; @@ -1389,15 +1388,19 @@ public class RowBasedGrouperHelper jp.currentToken() != JsonToken.END_ARRAY, "Unexpected end of array when deserializing timestamp from the spilled files" ); - objects[dimsReadSoFar] = codec.readValue(jp, Long.class); + objects[dimsReadSoFar] = JacksonUtils.readObjectUsingDeserializationContext(jp, deserializationContext, Long.class); ++dimsReadSoFar; jp.nextToken(); } while (jp.currentToken() != JsonToken.END_ARRAY) { - objects[dimsReadSoFar] = - codec.readValue(jp, serdeHelpers[dimsReadSoFar - timestampAdjustment].getClazz()); + objects[dimsReadSoFar] = JacksonUtils.readObjectUsingDeserializationContext( + jp, + deserializationContext, + serdeHelpers[dimsReadSoFar - timestampAdjustment].getClazz() + ); + ++dimsReadSoFar; jp.nextToken(); diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java index fca50c25b28..2dd827d323e 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java @@ -21,159 +21,192 @@ package org.apache.druid.query.operator.window; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode; + +import javax.annotation.Nullable; -import java.util.Collections; import java.util.List; import java.util.Objects; -import java.util.stream.Collectors; -public class WindowFrame +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "rows", value = WindowFrame.Rows.class), + @JsonSubTypes.Type(name = "groups", value = WindowFrame.Groups.class), +}) +@SubclassesMustOverrideEqualsAndHashCode +public interface WindowFrame { - public static WindowFrame unbounded() + static WindowFrame unbounded() { - return new WindowFrame(PeerType.ROWS, true, 0, true, 0, null); + return rows(null, null); } - @SuppressWarnings("unused") - public enum PeerType + static Rows rows(Integer lowerOffset, Integer upperOffset) { - ROWS, - RANGE + return new WindowFrame.Rows(lowerOffset, upperOffset); } - // Will likely need to add the order by columns to also be able to deal with RANGE peer type. - private final PeerType peerType; - private final boolean lowerUnbounded; - private final int lowerOffset; - private final boolean upperUnbounded; - private final int upperOffset; - private final List orderBy; - - @JsonCreator - public WindowFrame( - @JsonProperty("peerType") PeerType peerType, - @JsonProperty("lowUnbounded") boolean lowerUnbounded, - @JsonProperty("lowOffset") int lowerOffset, - @JsonProperty("uppUnbounded") boolean upperUnbounded, - @JsonProperty("uppOffset") int upperOffset, - @JsonProperty("orderBy") List orderBy - ) + static Groups groups(Integer lowerOffset, Integer upperOffset, List orderByColumns) { - this.peerType = peerType; - this.lowerUnbounded = lowerUnbounded; - this.lowerOffset = lowerOffset; - this.upperUnbounded = upperUnbounded; - this.upperOffset = upperOffset; - this.orderBy = orderBy; + return new WindowFrame.Groups(lowerOffset, upperOffset, orderByColumns); } - @JsonProperty("peerType") - public PeerType getPeerType() + static WindowFrame forOrderBy(String... orderByColumns) { - return peerType; + return groups(null, 0, Lists.newArrayList(orderByColumns)); } - @JsonProperty("lowUnbounded") - public boolean isLowerUnbounded() + abstract class OffsetFrame implements WindowFrame { - return lowerUnbounded; - } + @JsonProperty + public final Integer lowerOffset; + @JsonProperty + public final Integer upperOffset; - @JsonProperty("lowOffset") - public int getLowerOffset() - { - return lowerOffset; - } - - @JsonProperty("uppUnbounded") - public boolean isUpperUnbounded() - { - return upperUnbounded; - } - - @JsonProperty("uppOffset") - public int getUpperOffset() - { - return upperOffset; - } - - @JsonProperty("orderBy") - public List getOrderBy() - { - return orderBy; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; + @JsonCreator + public OffsetFrame( + @JsonProperty("lowerOffset") Integer lowerOffset, + @JsonProperty("upperOffset") Integer upperOffset) + { + this.lowerOffset = lowerOffset; + this.upperOffset = upperOffset; } - if (!(o instanceof WindowFrame)) { - return false; + + @Override + public int hashCode() + { + return Objects.hash(lowerOffset, upperOffset); } - WindowFrame that = (WindowFrame) o; - return lowerUnbounded == that.lowerUnbounded - && lowerOffset == that.lowerOffset - && upperUnbounded == that.upperUnbounded - && upperOffset == that.upperOffset - && peerType == that.peerType - && Objects.equals(orderBy, that.orderBy); - } - @Override - public int hashCode() - { - return Objects.hash(peerType, lowerUnbounded, lowerOffset, upperUnbounded, upperOffset, orderBy); - } - - @Override - public String toString() - { - return "WindowFrame{" + - "peerType=" + peerType + - ", lowerUnbounded=" + lowerUnbounded + - ", lowerOffset=" + lowerOffset + - ", upperUnbounded=" + upperUnbounded + - ", upperOffset=" + upperOffset + - ", orderBy=" + orderBy + - '}'; - } - - public static WindowFrame forOrderBy(ColumnWithDirection... orderBy) - { - return new WindowFrame(PeerType.RANGE, true, 0, false, 0, Lists.newArrayList(orderBy)); - } - - public List getOrderByColNames() - { - if (orderBy == null) { - return Collections.emptyList(); + /** + * Calculates the applicable lower offset if the max number of rows is + * known. + */ + public int getLowerOffsetClamped(int maxRows) + { + if (lowerOffset == null) { + return -maxRows; + } + return Math.max(-maxRows, lowerOffset); } - return orderBy.stream().map(ColumnWithDirection::getColumn).collect(Collectors.toList()); + + /** + * Calculates the applicable upper offset if the max number of rows is + * known. + */ + public int getUpperOffsetClamped(int maxRows) + { + if (upperOffset == null) { + return maxRows; + } + return Math.min(maxRows, upperOffset); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + OffsetFrame other = (OffsetFrame) obj; + return Objects.equals(lowerOffset, other.lowerOffset) && Objects.equals(upperOffset, other.upperOffset); + } + + @Override + public abstract String toString(); } - /** - * Calculates the applicable lower offset if the max number of rows is known. - */ - public int getLowerOffsetClamped(int maxRows) + class Rows extends OffsetFrame { - if (lowerUnbounded) { - return -maxRows; + @JsonCreator + public Rows( + @JsonProperty("lowerOffset") Integer lowerOffset, + @JsonProperty("upperOffset") Integer upperOffset) + { + super(lowerOffset, upperOffset); + } + + @Override + public String toString() + { + return "WindowFrame.Rows [" + + "lowerOffset=" + lowerOffset + + ", upperOffset=" + upperOffset + + "]"; } - return Math.max(-maxRows, lowerOffset); } - /** - * Calculates the applicable upper offset if the max number of rows is known. - */ - public int getUpperOffsetClamped(int maxRows) + class Groups extends OffsetFrame { - if (upperUnbounded) { - return maxRows; + @JsonProperty + private final ImmutableList orderByColumns; + + @JsonCreator + public Groups( + @JsonProperty("lowerOffset") Integer lowerOffset, + @JsonProperty("upperOffset") Integer upperOffset, + @JsonProperty("orderByColumns") List orderByColumns) + { + super(lowerOffset, upperOffset); + this.orderByColumns = ImmutableList.copyOf(orderByColumns); } - return Math.min(maxRows, upperOffset); + + public List getOrderByColumns() + { + return orderByColumns; + } + + @Override + public int hashCode() + { + return Objects.hash(lowerOffset, orderByColumns, upperOffset); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + Groups other = (Groups) obj; + return Objects.equals(lowerOffset, other.lowerOffset) + && Objects.equals(orderByColumns, other.orderByColumns) + && Objects.equals(upperOffset, other.upperOffset); + } + + @Override + public String toString() + { + return "WindowFrame.Groups [" + + "lowerOffset=" + lowerOffset + + ", upperOffset=" + upperOffset + + ", orderByColumns=" + orderByColumns + "]"; + } + } + + @SuppressWarnings("unchecked") + @Nullable + default T unwrap(Class clazz) + { + if (clazz.isInstance(this)) { + return (T) this; + } + return null; } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java index 04f9eddbff0..05b9dee5458 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java @@ -212,15 +212,14 @@ public class ArrayListRowsAndColumns implements AppendableRowsAndColumn @Override public void addColumn(String name, Column column) { - if (rows.size() == numRows()) { + if (rows.size() == numRows() && column.as(ColumnValueSwapper.class) != null) { extraColumns.put(name, column); columnNames.add(name); return; } // When an ArrayListRowsAndColumns is only a partial view, but adds a column, it believes that the same column - // will eventually be added for all of the rows so we pre-allocate storage for the entire set of data and - // copy. + // will eventually be added for all the rows so we pre-allocate storage for the entire set of data and copy. final ColumnAccessor columnAccessor = column.toAccessor(); if (columnAccessor.numRows() != numRows()) { diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java index 83952873050..7130fafd867 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java @@ -21,6 +21,7 @@ package org.apache.druid.query.rowsandcols.semantic; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.aggregation.Aggregator; @@ -28,6 +29,9 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.query.operator.window.WindowFrame; +import org.apache.druid.query.operator.window.WindowFrame.Groups; +import org.apache.druid.query.operator.window.WindowFrame.OffsetFrame; +import org.apache.druid.query.operator.window.WindowFrame.Rows; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; import org.apache.druid.segment.ColumnSelectorFactory; @@ -106,22 +110,38 @@ public class DefaultFramedOnHeapAggregatable implements FramedOnHeapAggregatable public static Iterable buildIteratorFor(AppendableRowsAndColumns rac, WindowFrame frame) { int numRows = rac.numRows(); - if (frame.getLowerOffsetClamped(numRows) == -numRows && frame.getUpperOffsetClamped(numRows) == numRows) { - return buildUnboundedIteratorFor(rac, frame); - } else if (frame.getPeerType() == WindowFrame.PeerType.RANGE) { - return buildGroupIteratorFor(rac, frame); - } else { - return buildRowIteratorFor(rac, frame); + if (isEffectivelyUnbounded(frame, numRows)) { + return buildUnboundedIteratorFor(rac); } + Rows rowsFrame = frame.unwrap(WindowFrame.Rows.class); + if (rowsFrame != null) { + return buildRowIteratorFor(rac, rowsFrame); + } + Groups groupsFrame = frame.unwrap(WindowFrame.Groups.class); + if (groupsFrame != null) { + return buildGroupIteratorFor(rac, groupsFrame); + } + throw DruidException.defensive("Unable to handle WindowFrame [%s]!", frame); } - private static Iterable buildUnboundedIteratorFor(AppendableRowsAndColumns rac, WindowFrame frame) + private static boolean isEffectivelyUnbounded(WindowFrame frame, int numRows) { - int[] groupBoundaries = new int[]{0, rac.numRows()}; - return new GroupIteratorForWindowFrame(frame, groupBoundaries); + OffsetFrame offsetFrame = frame.unwrap(WindowFrame.OffsetFrame.class); + if (offsetFrame.getLowerOffsetClamped(numRows) == -numRows + && offsetFrame.getUpperOffsetClamped(numRows) == numRows) { + // regardless the actual mode; all rows will be inside the frame! + return true; + } + return false; } - private static Iterable buildRowIteratorFor(AppendableRowsAndColumns rac, WindowFrame frame) + private static Iterable buildUnboundedIteratorFor(AppendableRowsAndColumns rac) + { + int[] groupBoundaries = new int[] {0, rac.numRows()}; + return new GroupIteratorForWindowFrame(WindowFrame.rows(null, null), groupBoundaries); + } + + private static Iterable buildRowIteratorFor(AppendableRowsAndColumns rac, WindowFrame.Rows frame) { int[] groupBoundaries = new int[rac.numRows() + 1]; for (int j = 0; j < groupBoundaries.length; j++) { @@ -130,9 +150,9 @@ public class DefaultFramedOnHeapAggregatable implements FramedOnHeapAggregatable return new GroupIteratorForWindowFrame(frame, groupBoundaries); } - private static Iterable buildGroupIteratorFor(AppendableRowsAndColumns rac, WindowFrame frame) + private static Iterable buildGroupIteratorFor(AppendableRowsAndColumns rac, WindowFrame.Groups frame) { - int[] groupBoundaries = ClusteredGroupPartitioner.fromRAC(rac).computeBoundaries(frame.getOrderByColNames()); + int[] groupBoundaries = ClusteredGroupPartitioner.fromRAC(rac).computeBoundaries(frame.getOrderByColumns()); return new GroupIteratorForWindowFrame(frame, groupBoundaries); } @@ -145,7 +165,7 @@ public class DefaultFramedOnHeapAggregatable implements FramedOnHeapAggregatable // upper exclusive private final int upperOffset; - public GroupIteratorForWindowFrame(WindowFrame frame, int[] groupBoundaries) + public GroupIteratorForWindowFrame(WindowFrame.OffsetFrame frame, int[] groupBoundaries) { this.groupBoundaries = groupBoundaries; numGroups = groupBoundaries.length - 1; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index ac36b106c64..96693a7b389 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -187,7 +187,6 @@ public class ScanQuery extends BaseQuery private final long scanRowsLimit; private final DimFilter dimFilter; private final List columns; - private final Boolean legacy; private final Order timeOrder; private final List orderBys; private final Integer maxRowsQueuedForOrdering; @@ -207,7 +206,6 @@ public class ScanQuery extends BaseQuery @JsonProperty("orderBy") List orderBysFromUser, @JsonProperty("filter") DimFilter dimFilter, @JsonProperty("columns") List columns, - @JsonProperty("legacy") Boolean legacy, @JsonProperty("context") Map context, @JsonProperty("columnTypes") List columnTypes ) @@ -232,7 +230,6 @@ public class ScanQuery extends BaseQuery ); this.dimFilter = dimFilter; this.columns = columns; - this.legacy = legacy; this.columnTypes = columnTypes; if (columnTypes != null) { @@ -447,18 +444,15 @@ public class ScanQuery extends BaseQuery } /** - * Compatibility mode with the legacy scan-query extension. - * - * True, false, and null have different meanings: true/false mean "legacy" and "not legacy"; null means use the - * default set by {@link ScanQueryConfig#isLegacy()}. The method {@link #withNonNullLegacy} is provided to help - * with this. + * Prior to PR https://github.com/apache/druid/pull/16659 (Druid 31) data servers require + * the "legacy" parameter to be set to a non-null value. For compatibility with older data + * servers during rolling updates, we need to write out "false". */ - @Nullable - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) + @Deprecated + @JsonProperty("legacy") public Boolean isLegacy() { - return legacy; + return false; } @Override @@ -507,11 +501,6 @@ public class ScanQuery extends BaseQuery return Druids.ScanQueryBuilder.copy(this).limit(newLimit).build(); } - public ScanQuery withNonNullLegacy(final ScanQueryConfig scanQueryConfig) - { - return Druids.ScanQueryBuilder.copy(this).legacy(legacy != null ? legacy : scanQueryConfig.isLegacy()).build(); - } - @Override public ScanQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec) { @@ -546,7 +535,6 @@ public class ScanQuery extends BaseQuery return batchSize == scanQuery.batchSize && scanRowsOffset == scanQuery.scanRowsOffset && scanRowsLimit == scanQuery.scanRowsLimit && - Objects.equals(legacy, scanQuery.legacy) && Objects.equals(virtualColumns, scanQuery.virtualColumns) && Objects.equals(resultFormat, scanQuery.resultFormat) && Objects.equals(dimFilter, scanQuery.dimFilter) && @@ -566,8 +554,7 @@ public class ScanQuery extends BaseQuery scanRowsLimit, dimFilter, columns, - orderBys, - legacy + orderBys ); } @@ -585,7 +572,6 @@ public class ScanQuery extends BaseQuery ", dimFilter=" + dimFilter + ", columns=" + columns + (orderBys.isEmpty() ? "" : ", orderBy=" + orderBys) + - (legacy == null ? "" : ", legacy=" + legacy) + ", context=" + getContext() + '}'; } @@ -710,12 +696,6 @@ public class ScanQuery extends BaseQuery */ @Nullable public RowSignature getRowSignature() - { - return getRowSignature(false); - } - - @Nullable - public RowSignature getRowSignature(boolean defaultIsLegacy) { if (columns == null || columns.isEmpty()) { // Note: if no specific list of columns is provided, then since we can't predict what columns will come back, we @@ -731,15 +711,7 @@ public class ScanQuery extends BaseQuery } return builder.build(); } - return guessRowSignature(defaultIsLegacy); - } - - private RowSignature guessRowSignature(boolean defaultIsLegacy) - { final RowSignature.Builder builder = RowSignature.builder(); - if (Boolean.TRUE.equals(legacy) || (legacy == null && defaultIsLegacy)) { - builder.add(ScanQueryEngine.LEGACY_TIMESTAMP_KEY, null); - } DataSource dataSource = getDataSource(); for (String columnName : columns) { final ColumnType columnType = guessColumnType(columnName, virtualColumns, dataSource); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java index b9269728260..aca2f00446f 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java @@ -29,38 +29,23 @@ public class ScanQueryConfig public static final String CTX_KEY_MAX_SEGMENT_PARTITIONS_FOR_ORDERING = "maxSegmentPartitionsOrderedInMemory"; @JsonProperty - private boolean legacy = false; - - public boolean isLegacy() - { - return legacy; - } - - @SuppressWarnings("unused") // Used by Jackson deserialization? - public ScanQueryConfig setLegacy(final boolean legacy) - { - this.legacy = legacy; - return this; - } + private int maxRowsQueuedForOrdering = 100000; @JsonProperty - private int maxRowsQueuedForOrdering = 100000; + private int maxSegmentPartitionsOrderedInMemory = 50; public int getMaxRowsQueuedForOrdering() { return maxRowsQueuedForOrdering; } - @JsonProperty - private int maxSegmentPartitionsOrderedInMemory = 50; - public int getMaxSegmentPartitionsOrderedInMemory() { return maxSegmentPartitionsOrderedInMemory; } @Override - public boolean equals(final Object o) + public boolean equals(Object o) { if (this == o) { return true; @@ -68,21 +53,23 @@ public class ScanQueryConfig if (o == null || getClass() != o.getClass()) { return false; } - final ScanQueryConfig that = (ScanQueryConfig) o; - return legacy == that.legacy; + ScanQueryConfig that = (ScanQueryConfig) o; + return maxRowsQueuedForOrdering == that.maxRowsQueuedForOrdering + && maxSegmentPartitionsOrderedInMemory == that.maxSegmentPartitionsOrderedInMemory; } @Override public int hashCode() { - return Objects.hash(legacy); + return Objects.hash(maxRowsQueuedForOrdering, maxSegmentPartitionsOrderedInMemory); } @Override public String toString() { return "ScanQueryConfig{" + - "legacy=" + legacy + + "maxRowsQueuedForOrdering=" + maxRowsQueuedForOrdering + + ", maxSegmentPartitionsOrderedInMemory=" + maxSegmentPartitionsOrderedInMemory + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 2246e349cbb..567e07fbdc7 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -23,7 +23,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; @@ -36,6 +35,7 @@ import org.apache.druid.query.QueryTimeoutException; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumn; @@ -59,8 +59,6 @@ import java.util.Set; public class ScanQueryEngine { - static final String LEGACY_TIMESTAMP_KEY = "timestamp"; - public Sequence process( final ScanQuery query, final Segment segment, @@ -68,10 +66,6 @@ public class ScanQueryEngine @Nullable final QueryMetrics queryMetrics ) { - - // "legacy" should be non-null due to toolChest.mergeResults - final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter"); - final Long numScannedRows = responseContext.getRowScanCount(); if (numScannedRows != null && numScannedRows >= query.getScanRowsLimit() && query.getTimeOrder().equals(ScanQuery.Order.NONE)) { return Sequences.empty(); @@ -93,9 +87,6 @@ public class ScanQueryEngine final List allColumns = new ArrayList<>(); if (query.getColumns() != null && !query.getColumns().isEmpty()) { - if (legacy && !query.getColumns().contains(LEGACY_TIMESTAMP_KEY)) { - allColumns.add(LEGACY_TIMESTAMP_KEY); - } // Unless we're in legacy mode, allColumns equals query.getColumns() exactly. This is nice since it makes // the compactedList form easier to use. @@ -103,7 +94,7 @@ public class ScanQueryEngine } else { final Set availableColumns = Sets.newLinkedHashSet( Iterables.concat( - Collections.singleton(legacy ? LEGACY_TIMESTAMP_KEY : ColumnHolder.TIME_COLUMN_NAME), + Collections.singleton(ColumnHolder.TIME_COLUMN_NAME), Iterables.transform( Arrays.asList(query.getVirtualColumns().getVirtualColumns()), VirtualColumn::getOutputName @@ -114,10 +105,6 @@ public class ScanQueryEngine ); allColumns.addAll(availableColumns); - - if (legacy) { - allColumns.remove(ColumnHolder.TIME_COLUMN_NAME); - } } final List intervals = query.getQuerySegmentSpec().getIntervals(); @@ -149,28 +136,15 @@ public class ScanQueryEngine { final List columnSelectors = new ArrayList<>(allColumns.size()); final RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); for (String column : allColumns) { - final BaseObjectColumnValueSelector selector; - - if (legacy && LEGACY_TIMESTAMP_KEY.equals(column)) { - selector = cursor.getColumnSelectorFactory() - .makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME); - ColumnCapabilities columnCapabilities = cursor.getColumnSelectorFactory() - .getColumnCapabilities(ColumnHolder.TIME_COLUMN_NAME); - rowSignatureBuilder.add( - column, - columnCapabilities == null ? null : columnCapabilities.toColumnType() - ); - } else { - selector = cursor.getColumnSelectorFactory().makeColumnValueSelector(column); - ColumnCapabilities columnCapabilities = cursor.getColumnSelectorFactory() - .getColumnCapabilities(column); - rowSignatureBuilder.add( - column, - columnCapabilities == null ? null : columnCapabilities.toColumnType() - ); - } + final BaseObjectColumnValueSelector selector = factory.makeColumnValueSelector(column); + ColumnCapabilities columnCapabilities = factory.getColumnCapabilities(column); + rowSignatureBuilder.add( + column, + columnCapabilities == null ? null : columnCapabilities.toColumnType() + ); columnSelectors.add(selector); } @@ -246,14 +220,7 @@ public class ScanQueryEngine private Object getColumnValue(int i) { final BaseObjectColumnValueSelector selector = columnSelectors.get(i); - final Object value; - - if (legacy && allColumns.get(i).equals(LEGACY_TIMESTAMP_KEY)) { - value = DateTimes.utc((long) selector.getObject()); - } else { - value = selector == null ? null : selector.getObject(); - } - + final Object value = selector == null ? null : selector.getObject(); return value; } }; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 63722db74af..717a40d1062 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -50,16 +50,13 @@ public class ScanQueryQueryToolChest extends QueryToolChest results; @@ -154,8 +148,7 @@ public class ScanQueryQueryToolChest extends QueryToolChest> return context().getBoolean(SKIP_EMPTY_BUCKETS, false); } + public RowSignature getResultSignature(final RowSignature.Finalization finalization) + { + final RowSignature.Builder builder = RowSignature.builder(); + builder.addTimeColumn(); + String timestampResultField = getTimestampResultField(); + if (StringUtils.isNotEmpty(timestampResultField)) { + builder.add(timestampResultField, ColumnType.LONG); + } + builder.addAggregators(aggregatorSpecs, finalization); + builder.addPostAggregators(postAggregatorSpecs); + return builder.build(); + } + @Nullable @Override public Set getRequiredColumns() diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 67c36fe7603..3ad58270f8b 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -62,7 +62,6 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedColumnSelectorFactory; -import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.joda.time.DateTime; @@ -439,14 +438,7 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence( resultsAsArrays(query, resultSequence), rowSignature diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java index 349e5a02d16..21729022d61 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java @@ -37,6 +37,7 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; import java.util.ArrayList; @@ -185,6 +186,16 @@ public class TopNQuery extends BaseQuery> topNMetricSpec.initTopNAlgorithmSelector(selector); } + public RowSignature getResultSignature(final RowSignature.Finalization finalization) + { + return RowSignature.builder() + .addTimeColumn() + .addDimensions(Collections.singletonList(getDimensionSpec())) + .addAggregators(getAggregatorSpecs(), finalization) + .addPostAggregators(getPostAggregatorSpecs()) + .build(); + } + @Override public TopNQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec) { diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java index 21bc336438a..c5f195615f3 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java @@ -68,7 +68,6 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; import java.io.Closeable; import java.util.ArrayList; -import java.util.Collections; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; @@ -518,12 +517,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence( resultsAsArrays(query, resultSequence), rowSignature diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java index 2b32860698f..1168e34b28a 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java @@ -142,4 +142,10 @@ public class FilteredStorageAdapter implements StorageAdapter { return baseStorageAdapter.getMetadata(); } + + @Override + public boolean isFromTombstone() + { + return baseStorageAdapter.isFromTombstone(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/Metadata.java b/processing/src/main/java/org/apache/druid/segment/Metadata.java index e6b5b1f6588..c3ac554a0fa 100644 --- a/processing/src/main/java/org/apache/druid/segment/Metadata.java +++ b/processing/src/main/java/org/apache/druid/segment/Metadata.java @@ -40,7 +40,7 @@ import java.util.concurrent.ConcurrentHashMap; public class Metadata { // container is used for arbitrary key-value pairs in segment metadata e.g. - // kafka firehose uses it to store commit offset + // kafka input reader uses it to store commit offset private final Map container; @Nullable private final AggregatorFactory[] aggregators; diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index ff4994210e1..752c6f48678 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -256,6 +256,12 @@ public class UnnestStorageAdapter implements StorageAdapter return baseAdapter.getMetadata(); } + @Override + public boolean isFromTombstone() + { + return baseAdapter.isFromTombstone(); + } + public VirtualColumn getUnnestColumn() { return unnestColumn; diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index e6f67a1683e..9efbcaecdee 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -177,7 +177,7 @@ public class HashJoinEngine @Override public void advance() { - advanceUninterruptibly(); + advance(true); BaseQuery.checkInterrupted(); } @@ -196,6 +196,11 @@ public class HashJoinEngine @Override public void advanceUninterruptibly() + { + advance(false); + } + + private void advance(boolean interruptibly) { joinColumnSelectorFactory.advanceRowId(); @@ -217,7 +222,11 @@ public class HashJoinEngine do { // No more right-hand side matches; advance the left-hand side. - leftCursor.advanceUninterruptibly(); + if (interruptibly) { + leftCursor.advance(); + } else { + leftCursor.advanceUninterruptibly(); + } // Update joinMatcher state to match new cursor position. matchCurrentPosition(); diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index fd9c07a253f..9dade664f8a 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -227,6 +227,12 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter return clauses.isEmpty() && baseAdapter.canVectorize(baseFilterAnd(filter), virtualColumns, descending); } + @Override + public boolean isFromTombstone() + { + return baseAdapter.isFromTombstone(); + } + @Nullable @Override public VectorCursor makeVectorCursor( diff --git a/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java index f4176db220c..6827497f7a6 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java @@ -99,42 +99,27 @@ public final class DictionaryIdLookup implements Closeable this.arrayDictionaryWriter = arrayDictionaryWriter; } + @Nullable + public Object getDictionaryValue(int id) + { + ensureStringDictionaryLoaded(); + ensureLongDictionaryLoaded(); + ensureDoubleDictionaryLoaded(); + ensureArrayDictionaryLoaded(); + if (id < longOffset()) { + return StringUtils.fromUtf8Nullable(stringDictionary.get(id)); + } else if (id < doubleOffset()) { + return longDictionary.get(id - longOffset()); + } else if (id < arrayOffset()) { + return doubleDictionary.get(id - doubleOffset()); + } else { + return arrayDictionary.get(id - arrayOffset()); + } + } + public int lookupString(@Nullable String value) { - if (stringDictionary == null) { - // GenericIndexed v2 can write to multiple files if the dictionary is larger than 2gb, so we use a smooshfile - // for strings because of this. if other type dictionary writers could potentially use multiple internal files - // in the future, we should transition them to using this approach as well (or build a combination smoosher and - // mapper so that we can have a mutable smoosh) - File stringSmoosh = FileUtils.createTempDirInLocation(tempBasePath, StringUtils.urlEncode(name) + "__stringTempSmoosh"); - stringDictionaryFile = stringSmoosh.toPath(); - final String fileName = NestedCommonFormatColumnSerializer.getInternalFileName( - name, - NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME - ); - - try ( - final FileSmoosher smoosher = new FileSmoosher(stringSmoosh); - final SmooshedWriter writer = smoosher.addWithSmooshedWriter( - fileName, - stringDictionaryWriter.getSerializedSize() - ) - ) { - stringDictionaryWriter.writeTo(writer, smoosher); - writer.close(); - smoosher.close(); - stringBufferMapper = SmooshedFileMapper.load(stringSmoosh); - final ByteBuffer stringBuffer = stringBufferMapper.mapFile(fileName); - stringDictionary = StringEncodingStrategies.getStringDictionarySupplier( - stringBufferMapper, - stringBuffer, - ByteOrder.nativeOrder() - ).get(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } + ensureStringDictionaryLoaded(); final byte[] bytes = StringUtils.toUtf8Nullable(value); final int index = stringDictionary.indexOf(bytes == null ? null : ByteBuffer.wrap(bytes)); if (index < 0) { @@ -145,13 +130,7 @@ public final class DictionaryIdLookup implements Closeable public int lookupLong(@Nullable Long value) { - if (longDictionary == null) { - longDictionaryFile = makeTempFile(name + NestedCommonFormatColumnSerializer.LONG_DICTIONARY_FILE_NAME); - longBuffer = mapWriter(longDictionaryFile, longDictionaryWriter); - longDictionary = FixedIndexed.read(longBuffer, TypeStrategies.LONG, ByteOrder.nativeOrder(), Long.BYTES).get(); - // reset position - longBuffer.position(0); - } + ensureLongDictionaryLoaded(); final int index = longDictionary.indexOf(value); if (index < 0) { throw DruidException.defensive("Value not found in column[%s] long dictionary", name); @@ -161,18 +140,7 @@ public final class DictionaryIdLookup implements Closeable public int lookupDouble(@Nullable Double value) { - if (doubleDictionary == null) { - doubleDictionaryFile = makeTempFile(name + NestedCommonFormatColumnSerializer.DOUBLE_DICTIONARY_FILE_NAME); - doubleBuffer = mapWriter(doubleDictionaryFile, doubleDictionaryWriter); - doubleDictionary = FixedIndexed.read( - doubleBuffer, - TypeStrategies.DOUBLE, - ByteOrder.nativeOrder(), - Double.BYTES - ).get(); - // reset position - doubleBuffer.position(0); - } + ensureDoubleDictionaryLoaded(); final int index = doubleDictionary.indexOf(value); if (index < 0) { throw DruidException.defensive("Value not found in column[%s] double dictionary", name); @@ -182,13 +150,7 @@ public final class DictionaryIdLookup implements Closeable public int lookupArray(@Nullable int[] value) { - if (arrayDictionary == null) { - arrayDictionaryFile = makeTempFile(name + NestedCommonFormatColumnSerializer.ARRAY_DICTIONARY_FILE_NAME); - arrayBuffer = mapWriter(arrayDictionaryFile, arrayDictionaryWriter); - arrayDictionary = FrontCodedIntArrayIndexed.read(arrayBuffer, ByteOrder.nativeOrder()).get(); - // reset position - arrayBuffer.position(0); - } + ensureArrayDictionaryLoaded(); final int index = arrayDictionary.indexOf(value); if (index < 0) { throw DruidException.defensive("Value not found in column[%s] array dictionary", name); @@ -256,6 +218,82 @@ public final class DictionaryIdLookup implements Closeable return doubleOffset() + (doubleDictionaryWriter != null ? doubleDictionaryWriter.getCardinality() : 0); } + private void ensureStringDictionaryLoaded() + { + if (stringDictionary == null) { + // GenericIndexed v2 can write to multiple files if the dictionary is larger than 2gb, so we use a smooshfile + // for strings because of this. if other type dictionary writers could potentially use multiple internal files + // in the future, we should transition them to using this approach as well (or build a combination smoosher and + // mapper so that we can have a mutable smoosh) + File stringSmoosh = FileUtils.createTempDirInLocation(tempBasePath, StringUtils.urlEncode(name) + "__stringTempSmoosh"); + stringDictionaryFile = stringSmoosh.toPath(); + final String fileName = NestedCommonFormatColumnSerializer.getInternalFileName( + name, + NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME + ); + + try ( + final FileSmoosher smoosher = new FileSmoosher(stringSmoosh); + final SmooshedWriter writer = smoosher.addWithSmooshedWriter( + fileName, + stringDictionaryWriter.getSerializedSize() + ) + ) { + stringDictionaryWriter.writeTo(writer, smoosher); + writer.close(); + smoosher.close(); + stringBufferMapper = SmooshedFileMapper.load(stringSmoosh); + final ByteBuffer stringBuffer = stringBufferMapper.mapFile(fileName); + stringDictionary = StringEncodingStrategies.getStringDictionarySupplier( + stringBufferMapper, + stringBuffer, + ByteOrder.nativeOrder() + ).get(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + private void ensureLongDictionaryLoaded() + { + if (longDictionary == null) { + longDictionaryFile = makeTempFile(name + NestedCommonFormatColumnSerializer.LONG_DICTIONARY_FILE_NAME); + longBuffer = mapWriter(longDictionaryFile, longDictionaryWriter); + longDictionary = FixedIndexed.read(longBuffer, TypeStrategies.LONG, ByteOrder.nativeOrder(), Long.BYTES).get(); + // reset position + longBuffer.position(0); + } + } + + private void ensureDoubleDictionaryLoaded() + { + if (doubleDictionary == null) { + doubleDictionaryFile = makeTempFile(name + NestedCommonFormatColumnSerializer.DOUBLE_DICTIONARY_FILE_NAME); + doubleBuffer = mapWriter(doubleDictionaryFile, doubleDictionaryWriter); + doubleDictionary = FixedIndexed.read( + doubleBuffer, + TypeStrategies.DOUBLE, + ByteOrder.nativeOrder(), + Double.BYTES + ).get(); + // reset position + doubleBuffer.position(0); + } + } + + private void ensureArrayDictionaryLoaded() + { + if (arrayDictionary == null && arrayDictionaryWriter != null) { + arrayDictionaryFile = makeTempFile(name + NestedCommonFormatColumnSerializer.ARRAY_DICTIONARY_FILE_NAME); + arrayBuffer = mapWriter(arrayDictionaryFile, arrayDictionaryWriter); + arrayDictionary = FrontCodedIntArrayIndexed.read(arrayBuffer, ByteOrder.nativeOrder()).get(); + // reset position + arrayBuffer.position(0); + } + } + private Path makeTempFile(String name) { try { diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java index aa6a71ae754..d9f00bb2321 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java @@ -117,8 +117,8 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter } /** - * Hook to allow implementors the chance to do additional operations during {@link #addValue(int, Object)}, such as - * writing an additional value column + * Hook to allow implementors the chance to do additional operations during {@link #writeTo(int, FileSmoosher)}, such + * as writing an additional value column */ void writeValue(@Nullable T value) throws IOException { @@ -159,7 +159,6 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter localId = localDictionary.add(globalId); } intermediateValueWriter.write(localId); - writeValue(value); cursorPosition++; } @@ -168,11 +167,9 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter */ private void fillNull(int row) throws IOException { - final T value = processValue(row, null); final int localId = localDictionary.add(0); while (cursorPosition < row) { intermediateValueWriter.write(localId); - writeValue(value); cursorPosition++; } } @@ -252,6 +249,7 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter final int unsortedLocalId = rows.nextInt(); final int sortedLocalId = unsortedToSorted[unsortedLocalId]; encodedValueSerializer.addValue(sortedLocalId); + writeValue((T) globalDictionaryIdLookup.getDictionaryValue(unsortedToGlobal[unsortedLocalId])); bitmaps[sortedLocalId].add(rowCount++); } @@ -307,7 +305,7 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter } } - private void openColumnSerializer(SegmentWriteOutMedium medium, int maxId) throws IOException + public void openColumnSerializer(SegmentWriteOutMedium medium, int maxId) throws IOException { if (indexSpec.getDimensionCompression() != CompressionStrategy.UNCOMPRESSED) { this.version = DictionaryEncodedColumnPartSerde.VERSION.COMPRESSED; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleFieldColumnWriter.java index 8ccd528715b..09e8dc121c8 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleFieldColumnWriter.java @@ -58,21 +58,6 @@ public final class ScalarDoubleFieldColumnWriter extends GlobalDictionaryEncoded return globalDictionaryIdLookup.lookupDouble(value); } - @Override - public void open() throws IOException - { - super.open(); - doublesSerializer = CompressionFactory.getDoubleSerializer( - fieldName, - segmentWriteOutMedium, - StringUtils.format("%s.double_column", fieldName), - ByteOrder.nativeOrder(), - indexSpec.getDimensionCompression(), - fieldResourceCloser - ); - doublesSerializer.open(); - } - @Override void writeValue(@Nullable Double value) throws IOException { @@ -83,6 +68,21 @@ public final class ScalarDoubleFieldColumnWriter extends GlobalDictionaryEncoded } } + @Override + public void openColumnSerializer(SegmentWriteOutMedium medium, int maxId) throws IOException + { + super.openColumnSerializer(medium, maxId); + doublesSerializer = CompressionFactory.getDoubleSerializer( + fieldName, + medium, + StringUtils.format("%s.double_column", fieldName), + ByteOrder.nativeOrder(), + indexSpec.getDimensionCompression(), + fieldResourceCloser + ); + doublesSerializer.open(); + } + @Override void writeColumnTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongFieldColumnWriter.java index 66b5eca18d9..d9191c4e805 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongFieldColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongFieldColumnWriter.java @@ -58,22 +58,6 @@ public final class ScalarLongFieldColumnWriter extends GlobalDictionaryEncodedFi return globalDictionaryIdLookup.lookupLong(value); } - @Override - public void open() throws IOException - { - super.open(); - longsSerializer = CompressionFactory.getLongSerializer( - fieldName, - segmentWriteOutMedium, - StringUtils.format("%s.long_column", fieldName), - ByteOrder.nativeOrder(), - indexSpec.getLongEncoding(), - indexSpec.getDimensionCompression(), - fieldResourceCloser - ); - longsSerializer.open(); - } - @Override void writeValue(@Nullable Long value) throws IOException { @@ -84,6 +68,22 @@ public final class ScalarLongFieldColumnWriter extends GlobalDictionaryEncodedFi } } + @Override + public void openColumnSerializer(SegmentWriteOutMedium medium, int maxId) throws IOException + { + super.openColumnSerializer(medium, maxId); + longsSerializer = CompressionFactory.getLongSerializer( + fieldName, + medium, + StringUtils.format("%s.long_column", fieldName), + ByteOrder.nativeOrder(), + indexSpec.getLongEncoding(), + indexSpec.getDimensionCompression(), + fieldResourceCloser + ); + longsSerializer.open(); + } + @Override void writeColumnTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java index 98588de1d53..77188db5857 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java @@ -98,8 +98,8 @@ public class BuildingHashBasedNumberedShardSpec implements BuildingShardSpec PartitionChunk createChunk(T obj) { - // This method can be called in AppenderatorImpl to create a sinkTimeline. - // The sinkTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now. + // This method can be called in StreamAppenderator to create a sinkTimeline. + // The sinkTimeline isn't used in batch ingestion, let's set 'chunks' to 0 for now. // HashBasedNumberedShardSpec is using NumberedPartitionChunk, so we use it here too. return new NumberedPartitionChunk<>(partitionId, 0, obj); } diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java index 52b3069f0b4..3989340c7b4 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java @@ -61,8 +61,8 @@ public class BuildingNumberedShardSpec implements BuildingShardSpec PartitionChunk createChunk(T obj) { - // This method can be called in AppenderatorImpl to create a sinkTimeline. - // The sinkTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now. + // This method can be called in StreamAppenderator to create a sinkTimeline. + // The sinkTimeline isn't used in batch ingestion, let's set 'chunks' to 0 for now. return new NumberedPartitionChunk<>(partitionId, 0, obj); } diff --git a/processing/src/test/java/org/apache/druid/collections/CircularListTest.java b/processing/src/test/java/org/apache/druid/collections/CircularListTest.java new file mode 100644 index 00000000000..51518a254e3 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/collections/CircularListTest.java @@ -0,0 +1,135 @@ +/* + * 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.collections; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Set; + +public class CircularListTest +{ + @Test + public void testIterateInNaturalOrder() + { + final Set input = ImmutableSet.of("b", "a", "c"); + final CircularList circularList = new CircularList<>(input, Comparator.naturalOrder()); + final List observedElements = new ArrayList<>(); + int cnt = 0; + for (String x : circularList) { + observedElements.add(x); + if (++cnt >= input.size()) { + break; + } + } + Assert.assertEquals(ImmutableList.of("a", "b", "c"), observedElements); + } + + @Test + public void testIterateInReverseOrder() + { + final Set input = ImmutableSet.of(-1, 100, 0, -4); + final CircularList circularList = new CircularList<>(input, Comparator.reverseOrder()); + final List observedElements = new ArrayList<>(); + int cnt = 0; + for (Integer x : circularList) { + observedElements.add(x); + if (++cnt >= 2 * input.size()) { + break; + } + } + + Assert.assertEquals(ImmutableList.of(100, 0, -1, -4, 100, 0, -1, -4), observedElements); + } + + @Test + public void testIteratorResumesFromLastPosition() + { + final Set input = ImmutableSet.of("a", "b", "c", "d", "e", "f"); + final CircularList circularList = new CircularList<>(input, Comparator.naturalOrder()); + + List observedElements = new ArrayList<>(); + int cnt = 0; + for (String element : circularList) { + observedElements.add(element); + if (++cnt >= input.size() / 2) { + break; + } + } + + Assert.assertEquals(ImmutableList.of("a", "b", "c"), observedElements); + + observedElements = new ArrayList<>(); + for (String element : circularList) { + observedElements.add(element); + // Resume and go till the end, and add two more elements looping around + if (++cnt == input.size() + 2) { + break; + } + } + + Assert.assertEquals(ImmutableList.of("d", "e", "f", "a", "b"), observedElements); + } + + @Test + public void testEqualsSet() + { + final Set input = ImmutableSet.of("a", "b", "c"); + final CircularList circularList = new CircularList<>(input, Comparator.naturalOrder()); + Assert.assertTrue(circularList.equalsSet(ImmutableSet.of("b", "a", "c"))); + Assert.assertFalse(circularList.equalsSet(ImmutableSet.of("c"))); + Assert.assertFalse(circularList.equalsSet(ImmutableSet.of("a", "c"))); + } + + @Test + public void testEmptyIterator() + { + final Set input = ImmutableSet.of(); + final CircularList circularList = new CircularList<>(input, Comparator.naturalOrder()); + final List observedElements = new ArrayList<>(); + + int cnt = 0; + for (String x : circularList) { + observedElements.add(x); + if (++cnt >= input.size()) { + break; + } + } + Assert.assertEquals(ImmutableList.of(), observedElements); + } + + @Test + public void testNextOnEmptyIteratorThrowsException() + { + final Set input = ImmutableSet.of(); + final CircularList circularList = new CircularList<>(input, Comparator.naturalOrder()); + + final Iterator iterator = circularList.iterator(); + Assert.assertFalse(iterator.hasNext()); + Assert.assertThrows(NoSuchElementException.class, iterator::next); + } +} diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java index 744c29dba2a..80602d0508a 100644 --- a/processing/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java +++ b/processing/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java @@ -187,7 +187,7 @@ public class InputEntityIteratingReaderTest extends InitializedNullHandlingTest @Test public void testIncorrectURI() throws IOException, URISyntaxException { - final InputEntityIteratingReader firehose = new InputEntityIteratingReader( + final InputEntityIteratingReader inputReader = new InputEntityIteratingReader( new InputRowSchema( new TimestampSpec(null, null, null), new DimensionsSpec( @@ -220,7 +220,7 @@ public class InputEntityIteratingReaderTest extends InitializedNullHandlingTest temporaryFolder.newFolder() ); - try (CloseableIterator readIterator = firehose.read()) { + try (CloseableIterator readIterator = inputReader.read()) { String expectedMessage = "Error occurred while trying to read uri: testscheme://some/path"; Exception exception = Assert.assertThrows(RuntimeException.class, readIterator::hasNext); Assert.assertTrue(exception.getMessage().contains(expectedMessage)); diff --git a/processing/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/processing/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index ca34c364dca..5b76afb9022 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -143,7 +143,7 @@ public class ParallelMergeCombiningSequenceTest if (!currentBatch.isDrained()) { outputQueue.offer(currentBatch); } - outputQueue.offer(ParallelMergeCombiningSequence.ResultBatch.TERMINAL); + outputQueue.offer(ParallelMergeCombiningSequence.ResultBatch.terminal()); rawYielder.close(); cursor.close(); @@ -211,16 +211,18 @@ public class ParallelMergeCombiningSequenceTest if (!currentBatch.isDrained()) { outputQueue.offer(currentBatch); } - outputQueue.offer(ParallelMergeCombiningSequence.ResultBatch.TERMINAL); + outputQueue.offer(ParallelMergeCombiningSequence.ResultBatch.terminal()); rawYielder.close(); cursor.close(); rawYielder = Yielders.each(rawSequence); + ParallelMergeCombiningSequence.CancellationGizmo gizmo = new ParallelMergeCombiningSequence.CancellationGizmo(); ParallelMergeCombiningSequence.BlockingQueueuBatchedResultsCursor queueCursor = new ParallelMergeCombiningSequence.BlockingQueueuBatchedResultsCursor<>( outputQueue, + gizmo, INT_PAIR_ORDERING, false, -1L @@ -551,14 +553,14 @@ public class ParallelMergeCombiningSequenceTest } @Test - public void testTimeoutExceptionDueToStalledReader() + public void testTimeoutExceptionDueToSlowReader() { - final int someSize = 2048; + final int someSize = 50_000; List> input = new ArrayList<>(); - input.add(nonBlockingSequence(someSize)); - input.add(nonBlockingSequence(someSize)); - input.add(nonBlockingSequence(someSize)); - input.add(nonBlockingSequence(someSize)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); Throwable t = Assert.assertThrows(QueryTimeoutException.class, () -> assertException(input, 8, 64, 1000, 1500)); Assert.assertEquals("Query did not complete within configured timeout period. " + @@ -567,6 +569,110 @@ public class ParallelMergeCombiningSequenceTest Assert.assertTrue(pool.isQuiescent()); } + @Test + public void testTimeoutExceptionDueToStoppedReader() throws InterruptedException + { + final int someSize = 150_000; + List reporters = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + List> input = new ArrayList<>(); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + + TestingReporter reporter = new TestingReporter(); + final ParallelMergeCombiningSequence parallelMergeCombineSequence = new ParallelMergeCombiningSequence<>( + pool, + input, + INT_PAIR_ORDERING, + INT_PAIR_MERGE_FN, + true, + 1000, + 0, + TEST_POOL_SIZE, + 512, + 128, + ParallelMergeCombiningSequence.DEFAULT_TASK_TARGET_RUN_TIME_MILLIS, + reporter + ); + Yielder parallelMergeCombineYielder = Yielders.each(parallelMergeCombineSequence); + reporter.future = parallelMergeCombineSequence.getCancellationFuture(); + reporter.yielder = parallelMergeCombineYielder; + reporter.yielder = parallelMergeCombineYielder.next(null); + Assert.assertFalse(parallelMergeCombineYielder.isDone()); + reporters.add(reporter); + } + + // sleep until timeout + Thread.sleep(1000); + Assert.assertTrue(pool.awaitQuiescence(10, TimeUnit.SECONDS)); + Assert.assertTrue(pool.isQuiescent()); + Assert.assertFalse(pool.hasQueuedSubmissions()); + for (TestingReporter reporter : reporters) { + Assert.assertThrows(QueryTimeoutException.class, () -> reporter.yielder.next(null)); + Assert.assertTrue(reporter.future.isCancelled()); + Assert.assertTrue(reporter.future.getCancellationGizmo().isCanceled()); + } + Assert.assertTrue(pool.awaitQuiescence(10, TimeUnit.SECONDS)); + Assert.assertTrue(pool.isQuiescent()); + } + + @Test + public void testManyBigSequencesAllAtOnce() throws IOException + { + final int someSize = 50_000; + List reporters = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + List> input = new ArrayList<>(); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + + TestingReporter reporter = new TestingReporter(); + final ParallelMergeCombiningSequence parallelMergeCombineSequence = new ParallelMergeCombiningSequence<>( + pool, + input, + INT_PAIR_ORDERING, + INT_PAIR_MERGE_FN, + true, + 30 * 1000, + 0, + TEST_POOL_SIZE, + 512, + 128, + ParallelMergeCombiningSequence.DEFAULT_TASK_TARGET_RUN_TIME_MILLIS, + reporter + ); + Yielder parallelMergeCombineYielder = Yielders.each(parallelMergeCombineSequence); + reporter.future = parallelMergeCombineSequence.getCancellationFuture(); + reporter.yielder = parallelMergeCombineYielder; + parallelMergeCombineYielder.next(null); + Assert.assertFalse(parallelMergeCombineYielder.isDone()); + reporters.add(reporter); + } + + for (TestingReporter testingReporter : reporters) { + Yielder parallelMergeCombineYielder = testingReporter.yielder; + while (!parallelMergeCombineYielder.isDone()) { + parallelMergeCombineYielder = parallelMergeCombineYielder.next(parallelMergeCombineYielder.get()); + } + Assert.assertTrue(parallelMergeCombineYielder.isDone()); + parallelMergeCombineYielder.close(); + Assert.assertTrue(testingReporter.future.isDone()); + } + + Assert.assertTrue(pool.awaitQuiescence(10, TimeUnit.SECONDS)); + Assert.assertTrue(pool.isQuiescent()); + Assert.assertEquals(0, pool.getRunningThreadCount()); + Assert.assertFalse(pool.hasQueuedSubmissions()); + Assert.assertEquals(0, pool.getActiveThreadCount()); + for (TestingReporter reporter : reporters) { + Assert.assertTrue(reporter.done); + } + } + @Test public void testGracefulCloseOfYielderCancelsPool() throws IOException { @@ -666,7 +772,9 @@ public class ParallelMergeCombiningSequenceTest parallelMergeCombineYielder.close(); // cancellation trigger should not be set if sequence was fully yielded and close is called // (though shouldn't actually matter even if it was...) - Assert.assertFalse(parallelMergeCombineSequence.getCancellationGizmo().isCancelled()); + Assert.assertFalse(parallelMergeCombineSequence.getCancellationFuture().isCancelled()); + Assert.assertTrue(parallelMergeCombineSequence.getCancellationFuture().isDone()); + Assert.assertFalse(parallelMergeCombineSequence.getCancellationFuture().getCancellationGizmo().isCanceled()); } private void assertResult( @@ -713,13 +821,15 @@ public class ParallelMergeCombiningSequenceTest Assert.assertTrue(combiningYielder.isDone()); Assert.assertTrue(parallelMergeCombineYielder.isDone()); - Assert.assertTrue(pool.awaitQuiescence(1, TimeUnit.SECONDS)); + Assert.assertTrue(pool.awaitQuiescence(5, TimeUnit.SECONDS)); Assert.assertTrue(pool.isQuiescent()); combiningYielder.close(); parallelMergeCombineYielder.close(); // cancellation trigger should not be set if sequence was fully yielded and close is called // (though shouldn't actually matter even if it was...) - Assert.assertFalse(parallelMergeCombineSequence.getCancellationGizmo().isCancelled()); + Assert.assertFalse(parallelMergeCombineSequence.getCancellationFuture().isCancelled()); + Assert.assertFalse(parallelMergeCombineSequence.getCancellationFuture().getCancellationGizmo().isCanceled()); + Assert.assertTrue(parallelMergeCombineSequence.getCancellationFuture().isDone()); } private void assertResultWithEarlyClose( @@ -773,20 +883,21 @@ public class ParallelMergeCombiningSequenceTest } } // trying to next the yielder creates sadness for you - final String expectedExceptionMsg = "Already closed"; + final String expectedExceptionMsg = "Sequence canceled"; Assert.assertEquals(combiningYielder.get(), parallelMergeCombineYielder.get()); final Yielder finalYielder = parallelMergeCombineYielder; Throwable t = Assert.assertThrows(RuntimeException.class, () -> finalYielder.next(finalYielder.get())); Assert.assertEquals(expectedExceptionMsg, t.getMessage()); // cancellation gizmo of sequence should be cancelled, and also should contain our expected message - Assert.assertTrue(parallelMergeCombineSequence.getCancellationGizmo().isCancelled()); + Assert.assertTrue(parallelMergeCombineSequence.getCancellationFuture().getCancellationGizmo().isCanceled()); Assert.assertEquals( expectedExceptionMsg, - parallelMergeCombineSequence.getCancellationGizmo().getRuntimeException().getMessage() + parallelMergeCombineSequence.getCancellationFuture().getCancellationGizmo().getRuntimeException().getMessage() ); + Assert.assertTrue(parallelMergeCombineSequence.getCancellationFuture().isCancelled()); - Assert.assertTrue(pool.awaitQuiescence(1, TimeUnit.SECONDS)); + Assert.assertTrue(pool.awaitQuiescence(10, TimeUnit.SECONDS)); Assert.assertTrue(pool.isQuiescent()); Assert.assertFalse(combiningYielder.isDone()); @@ -1082,4 +1193,19 @@ public class ParallelMergeCombiningSequenceTest { return new IntPair(mergeKey, ThreadLocalRandom.current().nextInt(1, 100)); } + + static class TestingReporter implements Consumer + { + ParallelMergeCombiningSequence.CancellationFuture future; + Yielder yielder; + volatile ParallelMergeCombiningSequence.MergeCombineMetrics metrics; + volatile boolean done = false; + + @Override + public void accept(ParallelMergeCombiningSequence.MergeCombineMetrics mergeCombineMetrics) + { + metrics = mergeCombineMetrics; + done = true; + } + } } diff --git a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java index 44d5725ba3d..03b5fe7e45e 100644 --- a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java +++ b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java @@ -91,7 +91,6 @@ public class DoubleStorageTest extends InitializedNullHandlingTest ); private static final ScanQueryQueryToolChest SCAN_QUERY_QUERY_TOOL_CHEST = new ScanQueryQueryToolChest( - new ScanQueryConfig(), DefaultGenericQueryMetricsFactory.instance() ); @@ -107,8 +106,7 @@ public class DoubleStorageTest extends InitializedNullHandlingTest .dataSource(new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE)) .columns(Collections.emptyList()) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) - .limit(Integer.MAX_VALUE) - .legacy(false); + .limit(Integer.MAX_VALUE); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java index 2ad9f90148a..e21b185d5cd 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java @@ -278,7 +278,6 @@ public class AggregationTestHelper implements Closeable ObjectMapper mapper = TestHelper.makeJsonMapper(); ScanQueryQueryToolChest toolchest = new ScanQueryQueryToolChest( - new ScanQueryConfig(), DefaultGenericQueryMetricsFactory.instance() ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java index e0dc85a4450..21c8aeacc25 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java @@ -63,7 +63,7 @@ public class DoubleMaxAggregationTest selector = new TestDoubleColumnSelectorImpl(values); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(selector); - EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null); + EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null).anyTimes(); EasyMock.replay(colSelectorFactory); VectorValueSelector vectorValueSelector = EasyMock.createMock(VectorValueSelector.class); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java index 5b4041b8422..40f72719f7f 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java @@ -63,7 +63,7 @@ public class DoubleMinAggregationTest selector = new TestDoubleColumnSelectorImpl(values); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(selector); - EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null); + EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null).anyTimes(); EasyMock.replay(colSelectorFactory); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java index d063658d76a..535efb02faf 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java @@ -62,7 +62,7 @@ public class LongMaxAggregationTest selector = new TestLongColumnSelector(values); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(selector); - EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null); + EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null).anyTimes(); EasyMock.replay(colSelectorFactory); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java index f651ce074b9..129f81d245a 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java @@ -62,7 +62,7 @@ public class LongMinAggregationTest selector = new TestLongColumnSelector(values); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(selector); - EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null); + EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null).anyTimes(); EasyMock.replay(colSelectorFactory); VectorValueSelector vectorValueSelector = EasyMock.createMock(VectorValueSelector.class); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java index 421a457999d..c4df70a88de 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java @@ -22,20 +22,39 @@ package org.apache.druid.query.aggregation.hyperloglog; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.hll.HyperLogLogCollector; import org.apache.druid.hll.VersionZeroHyperLogLogCollector; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.NoopAggregator; +import org.apache.druid.query.aggregation.NoopBufferAggregator; +import org.apache.druid.query.aggregation.NoopVectorAggregator; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.NilColumnValueSelector; +import org.apache.druid.segment.TestColumnSelectorFactory; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.vector.TestVectorColumnSelectorFactory; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.nio.ByteBuffer; import java.util.Comparator; import java.util.Random; +import static org.junit.jupiter.api.Assertions.assertThrows; + public class HyperUniquesAggregatorFactoryTest { + static { + NullHandling.initializeForTests(); + } + static final HyperUniquesAggregatorFactory AGGREGATOR_FACTORY = new HyperUniquesAggregatorFactory( "hyperUnique", "uniques" @@ -44,6 +63,19 @@ public class HyperUniquesAggregatorFactoryTest private final HashFunction fn = Hashing.murmur3_128(); + private ColumnSelectorFactory metricFactory; + private VectorColumnSelectorFactory vectorFactory; + + @Before + public void setup() + { + final ColumnCapabilitiesImpl columnCapabilities = ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA); + metricFactory = new TestColumnSelectorFactory() + .addCapabilities("uniques", columnCapabilities) + .addColumnSelector("uniques", null); + vectorFactory = new TestVectorColumnSelectorFactory().addCapabilities("uniques", columnCapabilities); + } + @Test public void testDeserializeV0() { @@ -216,4 +248,39 @@ public class HyperUniquesAggregatorFactoryTest Assert.assertEquals(factory, factory2); } + + @Test + public void testFactorizeOnPrimitiveColumnType() + { + final ColumnCapabilitiesImpl columnCapabilities = ColumnCapabilitiesImpl.createDefault().setType(ColumnType.LONG); + final ColumnSelectorFactory metricFactory = new TestColumnSelectorFactory() + .addCapabilities("uniques", columnCapabilities) + .addColumnSelector("uniques", NilColumnValueSelector.instance()); + final VectorColumnSelectorFactory vectorFactory = new TestVectorColumnSelectorFactory().addCapabilities("uniques", columnCapabilities); + + Assert.assertEquals(NoopAggregator.instance(), AGGREGATOR_FACTORY.factorize(metricFactory)); + Assert.assertEquals(NoopBufferAggregator.instance(), AGGREGATOR_FACTORY.factorizeBuffered(metricFactory)); + Assert.assertEquals(NoopVectorAggregator.instance(), AGGREGATOR_FACTORY.factorizeVector(vectorFactory)); + } + + @Test + public void testFactorizeOnUnsupportedComplexColumn() + { + Throwable exception = assertThrows(DruidException.class, () -> AGGREGATOR_FACTORY.factorize(metricFactory)); + Assert.assertEquals("Using aggregator [hyperUnique] is not supported for complex columns with type [COMPLEX].", exception.getMessage()); + } + + @Test + public void testFactorizeBufferedOnUnsupportedComplexColumn() + { + Throwable exception = assertThrows(DruidException.class, () -> AGGREGATOR_FACTORY.factorizeBuffered(metricFactory)); + Assert.assertEquals("Using aggregator [hyperUnique] is not supported for complex columns with type [COMPLEX].", exception.getMessage()); + } + + @Test + public void testFactorizeVectorOnUnsupportedComplexColumn() + { + Throwable exception = assertThrows(DruidException.class, () -> AGGREGATOR_FACTORY.factorizeVector(vectorFactory)); + Assert.assertEquals("Using aggregator [hyperUnique] is not supported for complex columns with type [COMPLEX].", exception.getMessage()); + } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java index 7279ca938bd..d9aefd5f55e 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java @@ -792,7 +792,19 @@ public class GroupByQueryQueryToolChestTest extends InitializedNullHandlingTest .setGranularity(QueryRunnerTestHelper.DAY_GRAN) .build(); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(null, null, null); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( + null, + () -> new GroupByQueryConfig() + { + @Override + public boolean isIntermediateResultAsMapCompat() + { + return true; + } + }, + null, + null + ); final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); final ObjectMapper arraysObjectMapper = toolChest.decorateObjectMapper( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index a5dbb49bca5..475848edbdc 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -363,6 +363,7 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest ); final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( groupingEngine, + () -> config, DefaultGroupByQueryMetricsFactory.instance(), groupByResourcesReservationPool ); diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/WindowFrameTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFrameTest.java new file mode 100644 index 00000000000..855f4694f43 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFrameTest.java @@ -0,0 +1,70 @@ +/* + * 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.query.operator.window; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.query.operator.window.WindowFrame.OffsetFrame; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class WindowFrameTest +{ + @Test + public void testEqualsRows() + { + EqualsVerifier.forClass(WindowFrame.Rows.class) + .usingGetClass() + .verify(); + } + + @Test + public void testEqualsGroups() + { + EqualsVerifier.forClass(WindowFrame.Groups.class) + .usingGetClass() + .verify(); + } + + @Test + public void testOffsetFrameUnbounded() + { + OffsetFrame of = new WindowFrame.Rows(null, null); + assertEquals(-100, of.getLowerOffsetClamped(100)); + assertEquals(100, of.getUpperOffsetClamped(100)); + } + + @Test + public void testOffsetFrameNormal() + { + OffsetFrame of = new WindowFrame.Rows(-1, 2); + assertEquals(-1, of.getLowerOffsetClamped(100)); + assertEquals(2, of.getUpperOffsetClamped(100)); + } + + @Test + public void testOffsetFrameUnbounded2() + { + OffsetFrame of = new WindowFrame.Rows(-200, 200); + assertEquals(-100, of.getLowerOffsetClamped(100)); + assertEquals(100, of.getUpperOffsetClamped(100)); + } + +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java index 5af321b53c8..9bae78bc2cc 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java @@ -46,7 +46,7 @@ public class WindowFramedAggregateProcessorTest @Test public void testIsPassThruWhenRACReturnsSemanticInterface() { - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, false, 0, null); + final WindowFrame theFrame = WindowFrame.rows(null, 0); final AggregatorFactory[] theAggs = { new LongMaxAggregatorFactory("cummMax", "intCol"), new DoubleSumAggregatorFactory("cummSum", "doubleCol") @@ -78,7 +78,7 @@ public class WindowFramedAggregateProcessorTest @Test public void testDoesStuffWhenNoSemanticInterfacesAvailable() { - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, false, 0, null); + final WindowFrame theFrame = WindowFrame.rows(null, 0); final AggregatorFactory[] theAggs = { new LongSumAggregatorFactory("sum", "intCol") }; diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java index d5b11f7a612..41ceb315a04 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java @@ -25,10 +25,8 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongMinAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.operator.ColumnWithDirection; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; import org.apache.druid.query.operator.window.WindowFrame; -import org.apache.druid.query.operator.window.WindowFrame.PeerType; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; @@ -65,7 +63,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 0, null), + WindowFrame.rows(0, 0), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -91,7 +89,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -1, false, 2, null), + WindowFrame.rows(-1, 2), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -117,7 +115,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 2, null), + WindowFrame.rows(0, 2), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -143,7 +141,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -2, false, 0, null), + WindowFrame.rows(-2, 0), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -169,7 +167,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -5, false, 7, null), + WindowFrame.rows(-5, 7), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -197,7 +195,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -5, false, 1, null), + WindowFrame.rows(-5, 1), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -225,7 +223,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -5, false, 0, null), + WindowFrame.rows(-5, 0), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -253,7 +251,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -1, false, 7, null), + WindowFrame.rows(-1, 7), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -281,7 +279,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 7, null), + WindowFrame.rows(0, 7), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -309,7 +307,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 7, null), + WindowFrame.rows(0, 7), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -337,7 +335,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -5, false, 0, null), + WindowFrame.rows(-5, 0), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -371,7 +369,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null), + WindowFrame.unbounded(), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new LongSumAggregatorFactory("sumFromDouble", "doubleCol"), @@ -409,7 +407,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, false, 0, null), + WindowFrame.rows(null, 0), new AggregatorFactory[]{ new LongMaxAggregatorFactory("cummMax", "intCol"), new DoubleSumAggregatorFactory("cummSum", "doubleCol") @@ -443,7 +441,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, true, 0, null), + WindowFrame.rows(0, null), new AggregatorFactory[]{ new LongMaxAggregatorFactory("cummMax", "intCol"), new DoubleSumAggregatorFactory("cummSum", "doubleCol") @@ -465,7 +463,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase @Test public void testRangeOrderBy() { - WindowFrame frame = WindowFrame.forOrderBy(ColumnWithDirection.ascending("c1")); + WindowFrame frame = WindowFrame.forOrderBy("c1"); int[] c1Vals = new int[] {0, 0, 0, 1, 1, 1, 2, 2, 2, 2}; int[] c2Vals = new int[] {1, 1, 2, 1, 1, 2, 1, 1, 1, 2}; int[] resVals = new int[] {4, 4, 4, 8, 8, 8, 13, 13, 13, 13}; @@ -476,14 +474,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase @Test public void testRangeB1() { - WindowFrame frame = new WindowFrame( - PeerType.RANGE, - false, - -1, - false, - 0, - Collections.singletonList(ColumnWithDirection.ascending("c1")) - ); + WindowFrame frame = WindowFrame.groups(-1, 0, Collections.singletonList("c1")); int[] c1Vals = new int[] {0, 1, 2, 2, 3, 4, 5}; int[] c2Vals = new int[] {0, 1, 1, 1, 3, 4, 5}; @@ -495,14 +486,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase @Test public void testRangeA1() { - WindowFrame frame = new WindowFrame( - PeerType.RANGE, - false, - 0, - false, - 1, - Collections.singletonList(ColumnWithDirection.ascending("c1")) - ); + WindowFrame frame = WindowFrame.groups(0, 1, Collections.singletonList("c1")); int[] c1Vals = new int[] {0, 1, 2, 2, 3, 4, 5}; int[] c2Vals = new int[] {0, 1, 1, 1, 3, 4, 5}; @@ -514,14 +498,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase @Test public void testRangeB1A1() { - WindowFrame frame = new WindowFrame( - PeerType.RANGE, - false, - -1, - false, - 1, - Collections.singletonList(ColumnWithDirection.ascending("c1")) - ); + WindowFrame frame = WindowFrame.groups(-1, 1, Collections.singletonList("c1")); int[] c1Vals = new int[] {0, 1, 2, 3, 4, 5}; int[] c2Vals = new int[] {0, 1, 2, 3, 4, 5}; @@ -534,14 +511,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase @Test public void testRangeB1A1_2() { - WindowFrame frame = new WindowFrame( - PeerType.RANGE, - false, - -1, - false, - 1, - Collections.singletonList(ColumnWithDirection.ascending("c1")) - ); + WindowFrame frame = WindowFrame.groups(-1, 1, Collections.singletonList("c1")); int[] c1Vals = new int[] {0, 0, 1, 2, 3, 3, 4, 4, 5}; int[] c2Vals = new int[] {0, 0, 1, 2, 2, 1, 2, 2, 5}; @@ -553,14 +523,7 @@ public class FramedOnHeapAggregatableTest extends SemanticTestBase @Test public void testRangeB1A2() { - WindowFrame frame = new WindowFrame( - PeerType.RANGE, - false, - -1, - false, - 2, - Collections.singletonList(ColumnWithDirection.ascending("c1")) - ); + WindowFrame frame = WindowFrame.groups(-1, 2, Collections.singletonList("c1")); int[] c1Vals = new int[] {0, 0, 0, 1, 1, 1, 2, 2, 2, 2, 3, 3, 3}; int[] c2Vals = new int[] {1, 1, 2, 1, 1, 2, 1, 1, 1, 2, 1, 1, 1}; diff --git a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java index cc276c66328..e3ea3b1ef6d 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java @@ -69,7 +69,6 @@ import java.util.List; public class MultiSegmentScanQueryTest extends InitializedNullHandlingTest { private static final ScanQueryQueryToolChest TOOL_CHEST = new ScanQueryQueryToolChest( - new ScanQueryConfig(), DefaultGenericQueryMetricsFactory.instance() ); @@ -193,7 +192,6 @@ public class MultiSegmentScanQueryTest extends InitializedNullHandlingTest .intervals(I_0112_0114_SPEC) .batchSize(batchSize) .columns(Collections.emptyList()) - .legacy(false) .limit(limit) .offset(offset); } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java index 7a71c44e28c..f4214c19456 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java @@ -33,12 +33,10 @@ public class ScanQueryConfigTest .builder() .put("maxSegmentPartitionsOrderedInMemory", "1") .put("maxRowsQueuedForOrdering", "1") - .put("legacy", "true") .build(); private final ImmutableMap CONFIG_MAP2 = ImmutableMap .builder() - .put("legacy", "false") .put("maxSegmentPartitionsOrderedInMemory", "42") .build(); @@ -52,16 +50,13 @@ public class ScanQueryConfigTest final ScanQueryConfig config = MAPPER.convertValue(CONFIG_MAP, ScanQueryConfig.class); Assert.assertEquals(1, config.getMaxRowsQueuedForOrdering()); Assert.assertEquals(1, config.getMaxSegmentPartitionsOrderedInMemory()); - Assert.assertTrue(config.isLegacy()); final ScanQueryConfig config2 = MAPPER.convertValue(CONFIG_MAP2, ScanQueryConfig.class); Assert.assertEquals(100000, config2.getMaxRowsQueuedForOrdering()); Assert.assertEquals(42, config2.getMaxSegmentPartitionsOrderedInMemory()); - Assert.assertFalse(config2.isLegacy()); final ScanQueryConfig config3 = MAPPER.convertValue(CONFIG_MAP_EMPTY, ScanQueryConfig.class); Assert.assertEquals(100000, config3.getMaxRowsQueuedForOrdering()); Assert.assertEquals(50, config3.getMaxSegmentPartitionsOrderedInMemory()); - Assert.assertFalse(config3.isLegacy()); } } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index 63c5b381921..76e7f6bdd3e 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -72,8 +72,7 @@ public class ScanQueryQueryToolChestTest ); private final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest( - new ScanQueryConfig(), - new DefaultGenericQueryMetricsFactory() + DefaultGenericQueryMetricsFactory.instance() ); @Test @@ -95,7 +94,6 @@ public class ScanQueryQueryToolChestTest Druids.newScanQueryBuilder() .dataSource("foo") .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000")))) - .legacy(true) .build(); Assert.assertEquals(RowSignature.empty(), toolChest.resultArraySignature(scanQuery)); @@ -117,23 +115,6 @@ public class ScanQueryQueryToolChestTest ); } - @Test - public void test_resultArraySignature_columnsSpecifiedLegacyMode() - { - final ScanQuery scanQuery = - Druids.newScanQueryBuilder() - .dataSource("foo") - .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000")))) - .columns("foo", "bar") - .legacy(true) - .build(); - - Assert.assertEquals( - RowSignature.builder().add("timestamp", null).add("foo", null).add("bar", null).build(), - toolChest.resultArraySignature(scanQuery) - ); - } - @Test public void test_resultsAsArrays_columnsNotSpecifiedListResults() { diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java index fee5b4b9de7..5b155aa4a98 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java @@ -190,10 +190,7 @@ public class ScanQueryResultOrderingTest public void setUp() { queryRunnerFactory = new ScanQueryRunnerFactory( - new ScanQueryQueryToolChest( - new ScanQueryConfig(), - new DefaultGenericQueryMetricsFactory() - ), + new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()), new ScanQueryEngine(), new ScanQueryConfig() ); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index 1c9a6c67ded..55c23a8d8c9 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -74,10 +74,7 @@ public class ScanQueryRunnerFactoryTest }; private static final ScanQueryRunnerFactory FACTORY = new ScanQueryRunnerFactory( - new ScanQueryQueryToolChest( - CONFIG, - DefaultGenericQueryMetricsFactory.instance() - ), + new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()), new ScanQueryEngine(), CONFIG ); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index b003f9580e8..f2c3a9ea18f 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -62,7 +62,6 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.testing.InitializedNullHandlingTest; -import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -138,7 +137,6 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class); private static final ScanQueryQueryToolChest TOOL_CHEST = new ScanQueryQueryToolChest( - new ScanQueryConfig(), DefaultGenericQueryMetricsFactory.instance() ); @@ -148,28 +146,25 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest new ScanQueryConfig() ); - @Parameterized.Parameters(name = "{0}, legacy = {1}") + @Parameterized.Parameters(name = "{0}") public static Iterable constructorFeeder() { - - return QueryRunnerTestHelper.cartesian( + return Iterables.transform( QueryRunnerTestHelper.makeQueryRunners( FACTORY ), - ImmutableList.of(false, true) + (runner) -> new Object[]{runner} ); } private final QueryRunner runner; - private final boolean legacy; private final List columns; - public ScanQueryRunnerTest(final QueryRunner runner, final boolean legacy) + public ScanQueryRunnerTest(final QueryRunner runner) { this.runner = runner; - this.legacy = legacy; this.columns = Lists.newArrayList( - getTimestampName(), + ColumnHolder.TIME_COLUMN_NAME, "expr", "market", "quality", @@ -200,8 +195,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest .dataSource(new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE)) .columns(Collections.emptyList()) .eternityInterval() - .limit(3) - .legacy(legacy); + .limit(3); } @Test @@ -270,7 +264,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest final List>> expectedEvents = toEvents( new String[]{ - getTimestampName() + ":TIME", + ColumnHolder.TIME_COLUMN_NAME + ":TIME", QueryRunnerTestHelper.MARKET_DIMENSION + ":STRING", null, null, @@ -284,24 +278,12 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest null, QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE" }, - legacy, V_0112_0114 ); - // Add "__time" to all the expected events in legacy mode - if (legacy) { - for (List> batch : expectedEvents) { - for (Map event : batch) { - event.put("__time", ((DateTime) event.get("timestamp")).getMillis()); - } - } - } - List expectedResults = toExpected( expectedEvents, - legacy - ? Lists.newArrayList(getTimestampName(), "__time", "market", "index") - : Lists.newArrayList("__time", "market", "index"), + Lists.newArrayList("__time", "market", "index"), 0, 3 ); @@ -321,7 +303,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest List expectedResults = toExpected( toEvents( new String[]{ - legacy ? getTimestampName() + ":TIME" : null, + null, QueryRunnerTestHelper.MARKET_DIMENSION + ":STRING", null, null, @@ -335,10 +317,9 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest null, QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE" }, - legacy, V_0112_0114 ), - legacy ? Lists.newArrayList(getTimestampName(), "market", "index") : Lists.newArrayList("market", "index"), + Lists.newArrayList("market", "index"), 0, 3 ); @@ -359,7 +340,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest List expectedResults = toExpected( toEvents( new String[]{ - legacy ? getTimestampName() + ":TIME" : null, + null, QueryRunnerTestHelper.MARKET_DIMENSION + ":STRING", null, null, @@ -373,10 +354,9 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest null, QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE" }, - legacy, V_0112_0114 ), - legacy ? Lists.newArrayList(getTimestampName(), "market", "index") : Lists.newArrayList("market", "index"), + Lists.newArrayList("market", "index"), 0, 3 ); @@ -399,14 +379,13 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest final List>> events = toEvents( new String[]{ - legacy ? getTimestampName() + ":TIME" : null, + null, null, QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING", null, null, QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE" }, - legacy, // filtered values with day granularity new String[]{ "2011-01-12T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t100.000000", @@ -434,7 +413,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest List expectedResults = toExpected( events, - legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), + Lists.newArrayList("quality", "index"), 0, limit ); @@ -463,14 +442,13 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest final List>> events = toEvents( new String[]{ - legacy ? getTimestampName() + ":TIME" : null, + null, null, QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING", null, null, QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE" }, - legacy, // filtered values with day granularity new String[]{ "2011-01-12T00:00:00.000Z\ttotal_market\tmezzanine\tpreferred\tmpreferred\t1000.000000", @@ -484,11 +462,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest List expectedResults = toExpected( events, - legacy ? Lists.newArrayList( - getTimestampName(), - QueryRunnerTestHelper.QUALITY_DIMENSION, - QueryRunnerTestHelper.INDEX_METRIC - ) : Lists.newArrayList( + Lists.newArrayList( QueryRunnerTestHelper.QUALITY_DIMENSION, QueryRunnerTestHelper.INDEX_METRIC ), @@ -533,14 +507,13 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest Iterable results = runner.run(QueryPlus.wrap(query)).toList(); final List>> events = toEvents( - legacy ? new String[]{getTimestampName() + ":TIME"} : new String[0], - legacy, + new String[0], V_0112_0114 ); List expectedResults = toExpected( events, - legacy ? Lists.newArrayList(getTimestampName(), "foo", "foo2") : Lists.newArrayList("foo", "foo2"), + Lists.newArrayList("foo", "foo2"), 0, 3 ); @@ -591,40 +564,23 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest }; final List>> ascendingEvents = toEvents( new String[]{ - legacy ? getTimestampName() + ":TIME" : ColumnHolder.TIME_COLUMN_NAME, + ColumnHolder.TIME_COLUMN_NAME, null, QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING", null, null, QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE" }, - legacy, (String[]) ArrayUtils.addAll(seg1Results, seg2Results) ); - - if (legacy) { - for (List> batch : ascendingEvents) { - for (Map event : batch) { - event.put("__time", ((DateTime) event.get("timestamp")).getMillis()); - } - } - } else { - for (List> batch : ascendingEvents) { - for (Map event : batch) { - event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis()); - } + for (List> batch : ascendingEvents) { + for (Map event : batch) { + event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis()); } } List ascendingExpectedResults = toExpected( ascendingEvents, - legacy ? - Lists.newArrayList( - QueryRunnerTestHelper.TIME_DIMENSION, - getTimestampName(), - "quality", - "index" - ) : Lists.newArrayList( QueryRunnerTestHelper.TIME_DIMENSION, "quality", @@ -681,39 +637,23 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest ArrayUtils.reverse(expectedRet); final List>> descendingEvents = toEvents( new String[]{ - legacy ? getTimestampName() + ":TIME" : ColumnHolder.TIME_COLUMN_NAME, + ColumnHolder.TIME_COLUMN_NAME, null, QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING", null, null, QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE" }, - legacy, expectedRet ); - if (legacy) { - for (List> batch : descendingEvents) { - for (Map event : batch) { - event.put("__time", ((DateTime) event.get("timestamp")).getMillis()); - } - } - } else { - for (List> batch : descendingEvents) { - for (Map event : batch) { - event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis()); - } + + for (List> batch : descendingEvents) { + for (Map event : batch) { + event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis()); } } List descendingExpectedResults = toExpected( descendingEvents, - legacy ? - Lists.newArrayList( - QueryRunnerTestHelper.TIME_DIMENSION, - getTimestampName(), - // getTimestampName() always returns the legacy timestamp when legacy is true - "quality", - "index" - ) : Lists.newArrayList( QueryRunnerTestHelper.TIME_DIMENSION, "quality", @@ -770,39 +710,22 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest Iterable results = runner.run(QueryPlus.wrap(query)).toList(); final List>> ascendingEvents = toEvents( new String[]{ - legacy ? getTimestampName() + ":TIME" : ColumnHolder.TIME_COLUMN_NAME, + ColumnHolder.TIME_COLUMN_NAME, null, QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING", null, null, QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE" }, - legacy, (String[]) ArrayUtils.addAll(seg1Results, seg2Results) ); - if (legacy) { - for (List> batch : ascendingEvents) { - for (Map event : batch) { - event.put("__time", ((DateTime) event.get("timestamp")).getMillis()); - } - } - } else { - for (List> batch : ascendingEvents) { - for (Map event : batch) { - event.put("__time", ((DateTimes.of((String) event.get("__time"))).getMillis())); - } + for (List> batch : ascendingEvents) { + for (Map event : batch) { + event.put("__time", ((DateTimes.of((String) event.get("__time"))).getMillis())); } } List ascendingExpectedResults = toExpected( ascendingEvents, - legacy ? - Lists.newArrayList( - QueryRunnerTestHelper.TIME_DIMENSION, - getTimestampName(), - // getTimestampName() always returns the legacy timestamp when legacy is true - "quality", - "index" - ) : Lists.newArrayList( QueryRunnerTestHelper.TIME_DIMENSION, "quality", @@ -863,39 +786,22 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest ArrayUtils.reverse(expectedRet); final List>> descendingEvents = toEvents( new String[]{ - legacy ? getTimestampName() + ":TIME" : ColumnHolder.TIME_COLUMN_NAME, + ColumnHolder.TIME_COLUMN_NAME, null, QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING", null, null, QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE" }, - legacy, expectedRet //segments in reverse order from above ); - if (legacy) { - for (List> batch : descendingEvents) { - for (Map event : batch) { - event.put("__time", ((DateTime) event.get("timestamp")).getMillis()); - } - } - } else { - for (List> batch : descendingEvents) { - for (Map event : batch) { - event.put("__time", ((DateTimes.of((String) event.get("__time"))).getMillis())); - } + for (List> batch : descendingEvents) { + for (Map event : batch) { + event.put("__time", ((DateTimes.of((String) event.get("__time"))).getMillis())); } } List descendingExpectedResults = toExpected( descendingEvents, - legacy ? - Lists.newArrayList( - QueryRunnerTestHelper.TIME_DIMENSION, - getTimestampName(), - // getTimestampName() always returns the legacy timestamp when legacy is true - "quality", - "index" - ) : Lists.newArrayList( QueryRunnerTestHelper.TIME_DIMENSION, "quality", @@ -996,7 +902,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest { return toEvents( new String[]{ - getTimestampName() + ":TIME", + ColumnHolder.TIME_COLUMN_NAME + ":TIME", QueryRunnerTestHelper.MARKET_DIMENSION + ":STRING", QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING", "qualityLong" + ":LONG", @@ -1018,12 +924,11 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest "indexMaxFloat", "quality_uniques" }, - legacy, valueSet ); } - public static List>> toEvents(final String[] dimSpecs, boolean legacy, final String[]... valueSet) + public static List>> toEvents(final String[] dimSpecs, final String[]... valueSet) { List values = new ArrayList<>(); for (String[] vSet : valueSet) { @@ -1085,7 +990,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest if (specs.length == 1 || specs[1].equals("STRING")) { eventVal = values1[i]; } else if (specs[1].equals("TIME")) { - eventVal = toTimestamp(values1[i], legacy); + eventVal = DateTimes.of(values1[i]).getMillis(); } else if (specs[1].equals("FLOAT")) { try { eventVal = values1[i].isEmpty() ? NullHandling.defaultFloatValue() : Float.valueOf(values1[i]); @@ -1127,20 +1032,6 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest return events; } - private static Object toTimestamp(final String value, boolean legacy) - { - if (legacy) { - return DateTimes.of(value); - } else { - return DateTimes.of(value).getMillis(); - } - } - - private String getTimestampName() - { - return legacy ? "timestamp" : ColumnHolder.TIME_COLUMN_NAME; - } - private List toExpected( List>> targets, List columns, diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java index 1eaa299927f..e5069743a9e 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java @@ -47,12 +47,7 @@ public class ScanQuerySpecTest + "\"limit\":3}"; String current = - "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," - + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]}," - + "\"resultFormat\":\"list\"," - + "\"limit\":3," - + "\"columns\":[\"market\",\"quality\",\"index\"]," - + "\"granularity\":{\"type\":\"all\"}}"; + "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},\"resultFormat\":\"list\",\"limit\":3,\"columns\":[\"market\",\"quality\",\"index\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false}"; ScanQuery query = new ScanQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), @@ -67,7 +62,6 @@ public class ScanQuerySpecTest null, Arrays.asList("market", "quality", "index"), null, - null, null ); @@ -81,13 +75,7 @@ public class ScanQuerySpecTest public void testSerializationWithTimeOrder() throws Exception { String originalJson = - "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," - + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]}," - + "\"resultFormat\":\"list\"," - + "\"limit\":3," - + "\"order\":\"ascending\"," - + "\"columns\":[\"market\",\"quality\",\"index\",\"__time\"]," - + "\"granularity\":{\"type\":\"all\"}}"; + "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},\"resultFormat\":\"list\",\"limit\":3,\"order\":\"ascending\",\"columns\":[\"market\",\"quality\",\"index\",\"__time\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false}"; ScanQuery expectedQuery = new ScanQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), @@ -102,7 +90,6 @@ public class ScanQuerySpecTest null, Arrays.asList("market", "quality", "index", "__time"), null, - null, null ); @@ -120,13 +107,7 @@ public class ScanQuerySpecTest public void testSerializationWithOrderBy() throws Exception { String originalJson = - "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," - + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]}," - + "\"resultFormat\":\"list\"," - + "\"limit\":3," - + "\"orderBy\":[{\"columnName\":\"quality\",\"order\":\"ascending\"}]," - + "\"columns\":[\"market\",\"quality\",\"index\",\"__time\"]," - + "\"granularity\":{\"type\":\"all\"}}"; + "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},\"resultFormat\":\"list\",\"limit\":3,\"orderBy\":[{\"columnName\":\"quality\",\"order\":\"ascending\"}],\"columns\":[\"market\",\"quality\",\"index\",\"__time\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false}"; ScanQuery expectedQuery = new ScanQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), @@ -141,7 +122,6 @@ public class ScanQuerySpecTest null, Arrays.asList("market", "quality", "index", "__time"), null, - null, null ); @@ -171,7 +151,6 @@ public class ScanQuerySpecTest null, Arrays.asList("market", "quality", "index"), null, - null, null ); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java index a79936b8818..b0d3432c0db 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java @@ -19,6 +19,8 @@ package org.apache.druid.query.scan; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.druid.java.util.common.DateTimes; @@ -26,8 +28,10 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.Druids; +import org.apache.druid.query.Query; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -45,6 +49,7 @@ import java.util.Set; public class ScanQueryTest { + private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); private static QuerySegmentSpec intervalSpec; private static ScanResultValue s1; private static ScanResultValue s2; @@ -94,6 +99,20 @@ public class ScanQueryTest ); } + @Test + public void testSerdeAndLegacyBackwardsCompat() throws JsonProcessingException + { + ScanQuery query = Druids.newScanQueryBuilder() + .columns(ImmutableList.of("__time", "quality")) + .dataSource("source") + .intervals(intervalSpec) + .build(); + Assert.assertFalse(query.isLegacy()); + String json = JSON_MAPPER.writeValueAsString(query); + Assert.assertTrue(json.contains("\"legacy\":false")); + Assert.assertEquals(query, JSON_MAPPER.readValue(json, Query.class)); + } + @Test(expected = IllegalArgumentException.class) public void testAscendingScanQueryWithInvalidColumns() { diff --git a/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java index 8238786998c..23d7f7a96af 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/UnnestScanQueryRunnerTest.java @@ -20,7 +20,6 @@ package org.apache.druid.query.scan; import com.google.common.collect.Lists; -import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.Druids; @@ -39,23 +38,17 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.testing.InitializedNullHandlingTest; -import org.joda.time.DateTime; import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; -@RunWith(Parameterized.class) public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest { public static final QuerySegmentSpec I_0112_0114 = ScanQueryRunnerTest.I_0112_0114; private static final ScanQueryQueryToolChest TOOL_CHEST = new ScanQueryQueryToolChest( - new ScanQueryConfig(), DefaultGenericQueryMetricsFactory.instance() ); private static final ScanQueryRunnerFactory FACTORY = new ScanQueryRunnerFactory( @@ -63,25 +56,7 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest new ScanQueryEngine(), new ScanQueryConfig() ); - private final IncrementalIndex index; - private final boolean legacy; - - public UnnestScanQueryRunnerTest(final IncrementalIndex index, final boolean legacy) - { - this.index = index; - this.legacy = legacy; - } - - @Parameterized.Parameters(name = "{0}") - public static Iterable constructorFeeder() - { - NullHandling.initializeForTests(); - final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex(); - final List constructors = new ArrayList<>(); - constructors.add(new Object[]{rtIndex, true}); - constructors.add(new Object[]{rtIndex, false}); - return constructors; - } + private final IncrementalIndex index = TestIndex.getIncrementalTestIndex(); private Druids.ScanQueryBuilder newTestUnnestQuery() { @@ -89,8 +64,7 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest .dataSource(QueryRunnerTestHelper.UNNEST_DATA_SOURCE) .columns(Collections.emptyList()) .eternityInterval() - .limit(3) - .legacy(legacy); + .limit(3); } private Druids.ScanQueryBuilder newTestUnnestQueryWithFilterDataSource() @@ -99,8 +73,7 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest .dataSource(QueryRunnerTestHelper.UNNEST_FILTER_DATA_SOURCE) .columns(Collections.emptyList()) .eternityInterval() - .limit(3) - .legacy(legacy); + .limit(3); } @Test @@ -123,38 +96,19 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest ); Iterable results = queryRunner.run(QueryPlus.wrap(query)).toList(); - String[] columnNames; - if (legacy) { - columnNames = new String[]{ - getTimestampName() + ":TIME", - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } else { - columnNames = new String[]{ - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } - String[] values; - if (legacy) { - values = new String[]{ - "2011-01-12T00:00:00.000Z\ta", - "2011-01-12T00:00:00.000Z\tpreferred", - "2011-01-12T00:00:00.000Z\tb" - }; - } else { - values = new String[]{ - "a", - "preferred", - "b" - }; - } + String[] columnNames = new String[]{ + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST + }; + String[] values = new String[]{ + "a", + "preferred", + "b" + }; - final List>> events = ScanQueryRunnerTest.toEvents(columnNames, legacy, values); + final List>> events = ScanQueryRunnerTest.toEvents(columnNames, values); List expectedResults = toExpected( events, - legacy - ? Lists.newArrayList(getTimestampName(), QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST) - : Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST), + Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST), 0, 3 ); @@ -181,38 +135,19 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest ); Iterable results = queryRunner.run(QueryPlus.wrap(query)).toList(); - String[] columnNames; - if (legacy) { - columnNames = new String[]{ - getTimestampName() + ":TIME", - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } else { - columnNames = new String[]{ - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } - String[] values; - if (legacy) { - values = new String[]{ - "2011-01-12T00:00:00.000Z\ta", - "2011-01-12T00:00:00.000Z\tpreferred", - "2011-01-12T00:00:00.000Z\tb" - }; - } else { - values = new String[]{ - "a", - "preferred", - "b" - }; - } + String[] columnNames = new String[]{ + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST + }; + String[] values = new String[]{ + "a", + "preferred", + "b" + }; - final List>> events = ScanQueryRunnerTest.toEvents(columnNames, legacy, values); + final List>> events = ScanQueryRunnerTest.toEvents(columnNames, values); List expectedResults = toExpected( events, - legacy - ? Lists.newArrayList(getTimestampName(), QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST) - : Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST), + Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST), 0, 3 ); @@ -237,7 +172,6 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest )) .columns(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST) .eternityInterval() - .legacy(legacy) .limit(3) .build(); @@ -251,38 +185,19 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest "rtIndexvc" ); Iterable results = vcrunner.run(QueryPlus.wrap(query)).toList(); - String[] columnNames; - if (legacy) { - columnNames = new String[]{ - getTimestampName() + ":TIME", - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } else { - columnNames = new String[]{ - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } - String[] values; - if (legacy) { - values = new String[]{ - "2011-01-12T00:00:00.000Z\ta", - "2011-01-12T00:00:00.000Z\tpreferred", - "2011-01-12T00:00:00.000Z\tb" - }; - } else { - values = new String[]{ - "a", - "preferred", - "b" - }; - } + String[] columnNames = new String[]{ + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST + }; + String[] values = new String[]{ + "a", + "preferred", + "b" + }; - final List>> events = ScanQueryRunnerTest.toEvents(columnNames, legacy, values); + final List>> events = ScanQueryRunnerTest.toEvents(columnNames, values); List expectedResults = toExpected( events, - legacy - ? Lists.newArrayList(getTimestampName(), QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST) - : Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST), + Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST), 0, 3 ); @@ -307,7 +222,6 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest )) .columns(QueryRunnerTestHelper.MARKET_DIMENSION, QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST) .eternityInterval() - .legacy(legacy) .limit(4) .build(); @@ -322,46 +236,21 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest ); Iterable results = vcrunner.run(QueryPlus.wrap(query)).toList(); - String[] columnNames; - if (legacy) { - columnNames = new String[]{ - getTimestampName() + ":TIME", - QueryRunnerTestHelper.MARKET_DIMENSION, - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } else { - columnNames = new String[]{ - QueryRunnerTestHelper.MARKET_DIMENSION, - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } - String[] values; - if (legacy) { - values = new String[]{ - "2011-01-12T00:00:00.000Z\tspot\tspot", - "2011-01-12T00:00:00.000Z\tspot\tautomotive", - "2011-01-12T00:00:00.000Z\tspot\tspot", - "2011-01-12T00:00:00.000Z\tspot\tbusiness", - }; - } else { - values = new String[]{ - "spot\tspot", - "spot\tautomotive", - "spot\tspot", - "spot\tbusiness" - }; - } + String[] columnNames = new String[]{ + QueryRunnerTestHelper.MARKET_DIMENSION, + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST + }; + String[] values = new String[]{ + "spot\tspot", + "spot\tautomotive", + "spot\tspot", + "spot\tbusiness" + }; - final List>> events = ScanQueryRunnerTest.toEvents(columnNames, legacy, values); + final List>> events = ScanQueryRunnerTest.toEvents(columnNames, values); List expectedResults = toExpected( events, - legacy - ? Lists.newArrayList( - getTimestampName(), - QueryRunnerTestHelper.MARKET_DIMENSION, - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - ) - : Lists.newArrayList( + Lists.newArrayList( QueryRunnerTestHelper.MARKET_DIMENSION, QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST ), @@ -392,38 +281,19 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest ); Iterable results = queryRunner.run(QueryPlus.wrap(query)).toList(); - String[] columnNames; - if (legacy) { - columnNames = new String[]{ - getTimestampName() + ":TIME", - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } else { - columnNames = new String[]{ - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } - String[] values; - if (legacy) { - values = new String[]{ - "2011-01-12T00:00:00.000Z\ta", - "2011-01-12T00:00:00.000Z\tpreferred", - "2011-01-12T00:00:00.000Z\tb" - }; - } else { - values = new String[]{ - "a", - "preferred", - "b" - }; - } + String[] columnNames = new String[]{ + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST + }; + String[] values = new String[]{ + "a", + "preferred", + "b" + }; - final List>> events = ScanQueryRunnerTest.toEvents(columnNames, legacy, values); + final List>> events = ScanQueryRunnerTest.toEvents(columnNames, values); List expectedResults = toExpected( events, - legacy - ? Lists.newArrayList(getTimestampName(), QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST) - : Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST), + Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST), 0, 3 ); @@ -453,47 +323,25 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest ); Iterable results = queryRunner.run(QueryPlus.wrap(query)).toList(); - String[] columnNames; - if (legacy) { - columnNames = new String[]{ - getTimestampName() + ":TIME", - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } else { - columnNames = new String[]{ - ColumnHolder.TIME_COLUMN_NAME, - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - }; - } - String[] values; - values = new String[]{ + String[] columnNames = new String[]{ + ColumnHolder.TIME_COLUMN_NAME, + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST + }; + String[] values = new String[]{ "2011-01-12T00:00:00.000Z\ta", "2011-01-12T00:00:00.000Z\tpreferred", "2011-01-12T00:00:00.000Z\tb" }; - final List>> ascendingEvents = ScanQueryRunnerTest.toEvents(columnNames, legacy, values); - if (legacy) { - for (List> batch : ascendingEvents) { - for (Map event : batch) { - event.put("__time", ((DateTime) event.get("timestamp")).getMillis()); - } - } - } else { - for (List> batch : ascendingEvents) { - for (Map event : batch) { - event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis()); - } + final List>> ascendingEvents = ScanQueryRunnerTest.toEvents(columnNames, values); + + for (List> batch : ascendingEvents) { + for (Map event : batch) { + event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis()); } } List ascendingExpectedResults = toExpected( ascendingEvents, - legacy ? - Lists.newArrayList( - QueryRunnerTestHelper.TIME_DIMENSION, - getTimestampName(), - QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST - ) : Lists.newArrayList( QueryRunnerTestHelper.TIME_DIMENSION, QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST @@ -505,12 +353,6 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest ScanQueryRunnerTest.verify(ascendingExpectedResults, results); } - - private String getTimestampName() - { - return legacy ? "timestamp" : ColumnHolder.TIME_COLUMN_NAME; - } - private List toExpected( List>> targets, List columns, diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java index 0746c104732..86d614cf4ac 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -342,6 +342,7 @@ public class JoinTestHelper .collect(Collectors.toList()); final List rows = new ArrayList<>(); + boolean interruptible = false; // test both advance() and advanceUninterruptibly() while (!cursor.isDone()) { final Object[] row = new Object[columns.size()]; @@ -351,7 +352,13 @@ public class JoinTestHelper } rows.add(row); - cursor.advance(); + if (interruptible) { + cursor.advance(); + } else { + cursor.advanceUninterruptibly(); + } + + interruptible = !interruptible; } return Sequences.simple(rows); diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 5fa34d6699d..e4027bcd357 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -384,7 +384,7 @@ public class CachingClusteredClient implements QuerySegmentWalker BinaryOperator mergeFn = toolChest.createMergeFn(query); final QueryContext queryContext = query.context(); if (parallelMergeConfig.useParallelMergePool() && queryContext.getEnableParallelMerges() && mergeFn != null) { - return new ParallelMergeCombiningSequence<>( + final ParallelMergeCombiningSequence parallelSequence = new ParallelMergeCombiningSequence<>( pool, sequencesByInterval, query.getResultOrdering(), @@ -414,6 +414,8 @@ public class CachingClusteredClient implements QuerySegmentWalker } } ); + scheduler.registerQueryFuture(query, parallelSequence.getCancellationFuture()); + return parallelSequence; } else { return Sequences .simple(sequencesByInterval) diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java index ed9e22dfaa2..0f92d99db10 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -32,7 +32,6 @@ import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -92,7 +91,7 @@ public class ClientCompactionRunnerInfo { CompactionEngine compactionEngine = newConfig.getEngine() == null ? defaultCompactionEngine : newConfig.getEngine(); if (compactionEngine == CompactionEngine.NATIVE) { - return new CompactionConfigValidationResult(true, null); + return CompactionConfigValidationResult.success(); } else { return compactionConfigSupportedByMSQEngine(newConfig); } @@ -121,11 +120,10 @@ public class ClientCompactionRunnerInfo )); } validationResults.add(validateMaxNumTasksForMSQ(newConfig.getTaskContext())); - validationResults.add(validateMetricsSpecForMSQ(newConfig.getMetricsSpec())); return validationResults.stream() .filter(result -> !result.isValid()) .findFirst() - .orElse(new CompactionConfigValidationResult(true, null)); + .orElse(CompactionConfigValidationResult.success()); } /** @@ -135,22 +133,19 @@ public class ClientCompactionRunnerInfo { if (!(partitionsSpec instanceof DimensionRangePartitionsSpec || partitionsSpec instanceof DynamicPartitionsSpec)) { - return new CompactionConfigValidationResult( - false, - "Invalid partitionsSpec type[%s] for MSQ engine. Type must be either 'dynamic' or 'range'.", + return CompactionConfigValidationResult.failure( + "MSQ: Invalid partitioning type[%s]. Must be either 'dynamic' or 'range'", partitionsSpec.getClass().getSimpleName() ); } if (partitionsSpec instanceof DynamicPartitionsSpec && ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() != null) { - return new CompactionConfigValidationResult( - false, - "maxTotalRows[%d] in DynamicPartitionsSpec not supported for MSQ engine.", - ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() + return CompactionConfigValidationResult.failure( + "MSQ: 'maxTotalRows' not supported with 'dynamic' partitioning" ); } - return new CompactionConfigValidationResult(true, null); + return CompactionConfigValidationResult.success(); } /** @@ -162,12 +157,11 @@ public class ClientCompactionRunnerInfo ) { if (metricsSpec != null && isRollup != null && !isRollup) { - return new CompactionConfigValidationResult( - false, - "rollup in granularitySpec must be set to True if metricsSpec is specifed for MSQ engine." + return CompactionConfigValidationResult.failure( + "MSQ: 'granularitySpec.rollup' must be true if 'metricsSpec' is specified" ); } - return new CompactionConfigValidationResult(true, null); + return CompactionConfigValidationResult.success(); } /** @@ -179,38 +173,12 @@ public class ClientCompactionRunnerInfo int maxNumTasks = QueryContext.of(context) .getInt(ClientMSQContext.CTX_MAX_NUM_TASKS, ClientMSQContext.DEFAULT_MAX_NUM_TASKS); if (maxNumTasks < 2) { - return new CompactionConfigValidationResult(false, - "MSQ context maxNumTasks [%,d] cannot be less than 2, " - + "since at least 1 controller and 1 worker is necessary.", - maxNumTasks + return CompactionConfigValidationResult.failure( + "MSQ: Context maxNumTasks[%,d] must be at least 2 (1 controller + 1 worker)", + maxNumTasks ); } } - return new CompactionConfigValidationResult(true, null); - } - - /** - * Validate each metric has output column name same as the input name. - */ - public static CompactionConfigValidationResult validateMetricsSpecForMSQ(AggregatorFactory[] metricsSpec) - { - if (metricsSpec == null) { - return new CompactionConfigValidationResult(true, null); - } - return Arrays.stream(metricsSpec) - .filter(aggregatorFactory -> - !(aggregatorFactory.requiredFields().isEmpty() - || aggregatorFactory.requiredFields().size() == 1 - && aggregatorFactory.requiredFields() - .get(0) - .equals(aggregatorFactory.getName()))) - .findFirst() - .map(aggregatorFactory -> - new CompactionConfigValidationResult( - false, - "Different name[%s] and fieldName(s)[%s] for aggregator unsupported for MSQ engine.", - aggregatorFactory.getName(), - aggregatorFactory.requiredFields() - )).orElse(new CompactionConfigValidationResult(true, null)); + return CompactionConfigValidationResult.success(); } } diff --git a/server/src/main/java/org/apache/druid/guice/FirehoseModule.java b/server/src/main/java/org/apache/druid/guice/FirehoseModule.java deleted file mode 100644 index fe6461bf215..00000000000 --- a/server/src/main/java/org/apache/druid/guice/FirehoseModule.java +++ /dev/null @@ -1,55 +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.guice; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.inject.Binder; -import org.apache.druid.initialization.DruidModule; -import org.apache.druid.segment.realtime.firehose.ClippedFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.FixedCountFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; - -import java.util.Collections; -import java.util.List; - -public class FirehoseModule implements DruidModule -{ - @Override - public void configure(Binder binder) - { - } - - @Override - public List getJacksonModules() - { - return Collections.singletonList( - new SimpleModule("FirehoseModule") - .registerSubtypes( - new NamedType(ClippedFirehoseFactory.class, "clipped"), - new NamedType(TimedShutoffFirehoseFactory.class, "timed"), - new NamedType(EventReceiverFirehoseFactory.class, "receiver"), - new NamedType(FixedCountFirehoseFactory.class, "fixedCount") - ) - ); - } -} diff --git a/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java b/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java index 59516f8d4c4..e07ac5ed115 100644 --- a/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java +++ b/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java @@ -29,7 +29,6 @@ import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.guice.DruidSecondaryModule; import org.apache.druid.guice.ExpressionModule; import org.apache.druid.guice.ExtensionsModule; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.guice.JacksonConfigManagerModule; import org.apache.druid.guice.JavaScriptModule; import org.apache.druid.guice.LifecycleModule; @@ -123,7 +122,6 @@ public class CoreInjectorBuilder extends DruidInjectorBuilder new CoordinatorDiscoveryModule(), new LocalDataStorageDruidModule(), new TombstoneDataStorageModule(), - new FirehoseModule(), new JavaScriptModule(), new AuthenticatorModule(), new AuthenticatorMapperModule(), diff --git a/server/src/main/java/org/apache/druid/metadata/BasicDataSourceExt.java b/server/src/main/java/org/apache/druid/metadata/BasicDataSourceExt.java index 6ddfd378fb4..cff9308a697 100644 --- a/server/src/main/java/org/apache/druid/metadata/BasicDataSourceExt.java +++ b/server/src/main/java/org/apache/druid/metadata/BasicDataSourceExt.java @@ -50,7 +50,7 @@ public class BasicDataSourceExt extends BasicDataSource * Note that these properties are not currently checked against any security configuration such as * an allow list for JDBC properties. Instead, they are supposed to be checked before adding to this class. * - * @see SQLFirehoseDatabaseConnector#validateConfigs + * @see SQLInputSourceDatabaseConnector#validateConfigs */ private Properties connectionProperties; diff --git a/server/src/main/java/org/apache/druid/metadata/LockFilterPolicy.java b/server/src/main/java/org/apache/druid/metadata/LockFilterPolicy.java index 88ab4673aa8..019fd22807c 100644 --- a/server/src/main/java/org/apache/druid/metadata/LockFilterPolicy.java +++ b/server/src/main/java/org/apache/druid/metadata/LockFilterPolicy.java @@ -21,10 +21,12 @@ package org.apache.druid.metadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Collections; +import java.util.List; import java.util.Map; -import java.util.Objects; /** * Specifies a policy to filter active locks held by a datasource @@ -33,17 +35,20 @@ public class LockFilterPolicy { private final String datasource; private final int priority; + private final List intervals; private final Map context; @JsonCreator public LockFilterPolicy( @JsonProperty("datasource") String datasource, @JsonProperty("priority") int priority, - @JsonProperty("context") Map context + @JsonProperty("intervals") @Nullable List intervals, + @JsonProperty("context") @Nullable Map context ) { this.datasource = datasource; this.priority = priority; + this.intervals = intervals; this.context = context == null ? Collections.emptyMap() : context; } @@ -65,24 +70,10 @@ public class LockFilterPolicy return context; } - @Override - public boolean equals(Object o) + @Nullable + @JsonProperty + public List getIntervals() { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - LockFilterPolicy that = (LockFilterPolicy) o; - return Objects.equals(datasource, that.datasource) - && priority == that.priority - && Objects.equals(context, that.context); - } - - @Override - public int hashCode() - { - return Objects.hash(datasource, priority, context); + return intervals; } } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLFirehoseDatabaseConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLInputSourceDatabaseConnector.java similarity index 98% rename from server/src/main/java/org/apache/druid/metadata/SQLFirehoseDatabaseConnector.java rename to server/src/main/java/org/apache/druid/metadata/SQLInputSourceDatabaseConnector.java index 11d467323f4..02c2d4269c2 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLFirehoseDatabaseConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLInputSourceDatabaseConnector.java @@ -38,7 +38,7 @@ import java.sql.SQLTransientException; import java.util.Set; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -public abstract class SQLFirehoseDatabaseConnector +public abstract class SQLInputSourceDatabaseConnector { static final int MAX_RETRIES = 10; diff --git a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java index b0aaa54d5ba..1bf942df9f8 100644 --- a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java @@ -201,7 +201,7 @@ public interface SegmentsMetadataManager */ List getUnusedSegmentIntervals( String dataSource, - DateTime minStartTime, + @Nullable DateTime minStartTime, DateTime maxEndTime, int limit, DateTime maxUsedStatusLastUpdatedTime diff --git a/server/src/main/java/org/apache/druid/metadata/input/SqlEntity.java b/server/src/main/java/org/apache/druid/metadata/input/SqlEntity.java index abc64baae5a..0d55e0ed7b5 100644 --- a/server/src/main/java/org/apache/druid/metadata/input/SqlEntity.java +++ b/server/src/main/java/org/apache/druid/metadata/input/SqlEntity.java @@ -27,7 +27,7 @@ import org.apache.druid.data.input.InputEntity; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.SQLInputSourceDatabaseConnector; import org.apache.druid.metadata.SQLMetadataStorageActionHandler; import org.skife.jdbi.v2.ResultIterator; import org.skife.jdbi.v2.exceptions.ResultSetException; @@ -52,19 +52,19 @@ public class SqlEntity implements InputEntity private final String sql; private final ObjectMapper objectMapper; - private final SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector; + private final SQLInputSourceDatabaseConnector sqlInputSourceDatabaseConnector; private final boolean foldCase; public SqlEntity( String sql, - SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector, + SQLInputSourceDatabaseConnector sqlInputSourceDatabaseConnector, boolean foldCase, ObjectMapper objectMapper ) { this.sql = sql; - this.sqlFirehoseDatabaseConnector = Preconditions.checkNotNull( - sqlFirehoseDatabaseConnector, + this.sqlInputSourceDatabaseConnector = Preconditions.checkNotNull( + sqlInputSourceDatabaseConnector, "SQL Metadata Connector not configured!" ); this.foldCase = foldCase; @@ -93,7 +93,7 @@ public class SqlEntity implements InputEntity public CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws IOException { final File tempFile = File.createTempFile("druid-sql-entity", ".tmp", temporaryDirectory); - return openCleanableFile(sql, sqlFirehoseDatabaseConnector, objectMapper, foldCase, tempFile); + return openCleanableFile(sql, sqlInputSourceDatabaseConnector, objectMapper, foldCase, tempFile); } @@ -102,7 +102,7 @@ public class SqlEntity implements InputEntity * The result file is deleted if the query execution or the file write fails. * * @param sql The SQL query to be executed - * @param sqlFirehoseDatabaseConnector The database connector + * @param sqlInputSourceDatabaseConnector The database connector * @param objectMapper An object mapper, used for deserialization * @param foldCase A boolean flag used to enable or disabling case sensitivity while handling database column names * @@ -111,7 +111,7 @@ public class SqlEntity implements InputEntity public static CleanableFile openCleanableFile( String sql, - SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector, + SQLInputSourceDatabaseConnector sqlInputSourceDatabaseConnector, ObjectMapper objectMapper, boolean foldCase, File tempFile @@ -124,7 +124,7 @@ public class SqlEntity implements InputEntity // Execute the sql query and lazily retrieve the results into the file in json format. // foldCase is useful to handle differences in case sensitivity behavior across databases. - sqlFirehoseDatabaseConnector.retryWithHandle( + sqlInputSourceDatabaseConnector.retryWithHandle( (handle) -> { ResultIterator> resultIterator = handle.createQuery( sql @@ -161,7 +161,7 @@ public class SqlEntity implements InputEntity jg.close(); return null; }, - (exception) -> sqlFirehoseDatabaseConnector.isTransientException(exception) + (exception) -> sqlInputSourceDatabaseConnector.isTransientException(exception) && !(SQLMetadataStorageActionHandler.isStatementException(exception)) ); return new CleanableFile() diff --git a/server/src/main/java/org/apache/druid/metadata/input/SqlInputSource.java b/server/src/main/java/org/apache/druid/metadata/input/SqlInputSource.java index 8d886b058fc..27147babfa5 100644 --- a/server/src/main/java/org/apache/druid/metadata/input/SqlInputSource.java +++ b/server/src/main/java/org/apache/druid/metadata/input/SqlInputSource.java @@ -36,7 +36,7 @@ import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.data.input.impl.systemfield.SystemFieldDecoratorFactory; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.java.util.common.CloseableIterators; -import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.SQLInputSourceDatabaseConnector; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -51,7 +51,7 @@ public class SqlInputSource extends AbstractInputSource implements SplittableInp { static final String TYPE_KEY = "sql"; private final List sqls; - private final SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector; + private final SQLInputSourceDatabaseConnector sqlInputSourceDatabaseConnector; private final ObjectMapper objectMapper; private final boolean foldCase; @@ -59,7 +59,7 @@ public class SqlInputSource extends AbstractInputSource implements SplittableInp public SqlInputSource( @JsonProperty("sqls") List sqls, @JsonProperty("foldCase") boolean foldCase, - @JsonProperty("database") SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector, + @JsonProperty("database") SQLInputSourceDatabaseConnector sqlInputSourceDatabaseConnector, @JacksonInject @Smile ObjectMapper objectMapper ) { @@ -67,8 +67,8 @@ public class SqlInputSource extends AbstractInputSource implements SplittableInp this.sqls = sqls; this.foldCase = foldCase; - this.sqlFirehoseDatabaseConnector = Preconditions.checkNotNull( - sqlFirehoseDatabaseConnector, + this.sqlInputSourceDatabaseConnector = Preconditions.checkNotNull( + sqlInputSourceDatabaseConnector, "SQL Metadata Connector not configured!" ); this.objectMapper = objectMapper; @@ -95,9 +95,9 @@ public class SqlInputSource extends AbstractInputSource implements SplittableInp } @JsonProperty("database") - public SQLFirehoseDatabaseConnector getSQLFirehoseDatabaseConnector() + public SQLInputSourceDatabaseConnector getSQLInputSourceDatabaseConnector() { - return sqlFirehoseDatabaseConnector; + return sqlInputSourceDatabaseConnector; } @Override @@ -118,7 +118,7 @@ public class SqlInputSource extends AbstractInputSource implements SplittableInp return new SqlInputSource( Collections.singletonList(split.get()), foldCase, - sqlFirehoseDatabaseConnector, + sqlInputSourceDatabaseConnector, objectMapper ); } @@ -131,7 +131,8 @@ public class SqlInputSource extends AbstractInputSource implements SplittableInp inputRowSchema, inputFormat, CloseableIterators.withEmptyBaggage(createSplits(inputFormat, null) - .map(split -> new SqlEntity(split.get(), sqlFirehoseDatabaseConnector, foldCase, objectMapper)).iterator()), + .map(split -> new SqlEntity(split.get(), + sqlInputSourceDatabaseConnector, foldCase, objectMapper)).iterator()), SystemFieldDecoratorFactory.NONE, temporaryDirectory ); @@ -155,12 +156,12 @@ public class SqlInputSource extends AbstractInputSource implements SplittableInp SqlInputSource that = (SqlInputSource) o; return foldCase == that.foldCase && sqls.equals(that.sqls) && - sqlFirehoseDatabaseConnector.equals(that.sqlFirehoseDatabaseConnector); + sqlInputSourceDatabaseConnector.equals(that.sqlInputSourceDatabaseConnector); } @Override public int hashCode() { - return Objects.hash(sqls, sqlFirehoseDatabaseConnector, foldCase); + return Objects.hash(sqls, sqlInputSourceDatabaseConnector, foldCase); } } diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java b/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java index 36d84f2dc4e..057201b8564 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java @@ -23,7 +23,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.rpc.StandardRetryPolicy; -import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; +import org.apache.druid.segment.realtime.ChatHandlerResource; import org.jboss.netty.handler.codec.http.HttpResponse; import org.jboss.netty.handler.codec.http.HttpResponseStatus; diff --git a/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java new file mode 100644 index 00000000000..b2cb90bc0ce --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.indexing; + +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.transform.TransformSpec; + +import javax.annotation.Nullable; + +/** + * Class representing the combined DataSchema of a set of segments, currently used only by Compaction. + */ +public class CombinedDataSchema extends DataSchema +{ + private final boolean hasRolledUpSegments; + + public CombinedDataSchema( + String dataSource, + @Nullable TimestampSpec timestampSpec, + @Nullable DimensionsSpec dimensionsSpec, + AggregatorFactory[] aggregators, + GranularitySpec granularitySpec, + TransformSpec transformSpec, + @Nullable boolean hasRolledUpSegments + ) + { + super( + dataSource, + timestampSpec, + dimensionsSpec, + aggregators, + granularitySpec, + transformSpec, + null, + null + ); + this.hasRolledUpSegments = hasRolledUpSegments; + } + + public boolean hasRolledUpSegments() + { + return hasRolledUpSegments; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandler.java b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandler.java similarity index 95% rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandler.java rename to server/src/main/java/org/apache/druid/segment/realtime/ChatHandler.java index 004b6c04d48..56f8330fdcb 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandler.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandler.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; /** * Objects that can be registered with a {@link ServiceAnnouncingChatHandlerProvider} and provide http endpoints for indexing-related diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerProvider.java b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerProvider.java similarity index 97% rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerProvider.java rename to server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerProvider.java index 3af0d5c37f0..f19e25f3756 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerProvider.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerProvider.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; import com.google.common.base.Optional; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerResource.java similarity index 98% rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java rename to server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerResource.java index 1cd579d2734..4423a0d0f0f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerResource.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; import com.google.common.base.Optional; import com.google.common.collect.Iterables; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlers.java b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlers.java similarity index 97% rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlers.java rename to server/src/main/java/org/apache/druid/segment/realtime/ChatHandlers.java index 4f6e99217d5..4d971db81e0 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlers.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlers.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java b/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java index 5f1a88f2ea9..b051fa72724 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java @@ -44,6 +44,7 @@ public class FireHydrant { private final int count; private final AtomicReference adapter; + @Nullable private volatile IncrementalIndex index; public FireHydrant(IncrementalIndex index, int count, SegmentId segmentId) @@ -62,6 +63,7 @@ public class FireHydrant this.count = count; } + @Nullable public IncrementalIndex getIndex() { return index; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/NoopChatHandlerProvider.java b/server/src/main/java/org/apache/druid/segment/realtime/NoopChatHandlerProvider.java similarity index 96% rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/NoopChatHandlerProvider.java rename to server/src/main/java/org/apache/druid/segment/realtime/NoopChatHandlerProvider.java index 5c49b9ac292..9480bdf2bc7 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/NoopChatHandlerProvider.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/NoopChatHandlerProvider.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; import com.google.common.base.Optional; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java b/server/src/main/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProvider.java similarity index 98% rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java rename to server/src/main/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProvider.java index 802be54cc11..da3975e4545 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProvider.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; import com.google.common.base.Optional; import com.google.inject.Inject; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/WindowedStorageAdapter.java b/server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java similarity index 96% rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/WindowedStorageAdapter.java rename to server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java index 26d0b41013d..9b4bd1e4ea7 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/WindowedStorageAdapter.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; import org.apache.druid.segment.StorageAdapter; import org.joda.time.Interval; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java deleted file mode 100644 index 734abc6ed54..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ /dev/null @@ -1,1638 +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.segment.realtime.appenderator; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; -import com.google.common.base.Stopwatch; -import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.google.common.primitives.Ints; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.commons.lang.mutable.MutableLong; -import org.apache.druid.client.cache.Cache; -import org.apache.druid.data.input.Committer; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.RE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QuerySegmentWalker; -import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.segment.BaseProgressIndicator; -import org.apache.druid.segment.DataSegmentWithMetadata; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMerger; -import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.SchemaPayload; -import org.apache.druid.segment.SchemaPayloadPlus; -import org.apache.druid.segment.Segment; -import org.apache.druid.segment.SegmentSchemaMapping; -import org.apache.druid.segment.incremental.IncrementalIndexAddResult; -import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.incremental.ParseExceptionHandler; -import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.FingerprintGenerator; -import org.apache.druid.segment.realtime.FireHydrant; -import org.apache.druid.segment.realtime.SegmentGenerationMetrics; -import org.apache.druid.segment.realtime.sink.Sink; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.VersionedIntervalTimeline; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.io.Closeable; -import java.io.File; -import java.io.IOException; -import java.nio.channels.FileChannel; -import java.nio.channels.FileLock; -import java.nio.file.StandardOpenOption; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.IdentityHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import java.util.stream.Collectors; - -/** - * This class is to support OPEN_SEGMENTS and CLOSED_SEGMENTS appenderators. It is mostly taken - * from 0.21 and it is meant to keep for backward compatibility. For now though this class - * with isLegacy constructor argument set to false is the default. When {@link BatchAppenderator} - * proves stable then the plan is to remove this class - */ -@SuppressWarnings("CheckReturnValue") -public class AppenderatorImpl implements Appenderator -{ - // Rough estimate of memory footprint of a ColumnHolder based on actual heap dumps - public static final int ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER = 1000; - public static final int ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER = 700; - public static final int ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER = 600; - // Rough estimate of memory footprint of empty Sink based on actual heap dumps - public static final int ROUGH_OVERHEAD_PER_SINK = 5000; - // Rough estimate of memory footprint of empty FireHydrant based on actual heap dumps - public static final int ROUGH_OVERHEAD_PER_HYDRANT = 1000; - - private static final EmittingLogger log = new EmittingLogger(AppenderatorImpl.class); - private static final int WARN_DELAY = 1000; - private static final String IDENTIFIER_FILE_NAME = "identifier.json"; - - private final String myId; - private final DataSchema schema; - private final AppenderatorConfig tuningConfig; - private final SegmentGenerationMetrics metrics; - private final DataSegmentPusher dataSegmentPusher; - private final ObjectMapper objectMapper; - private final DataSegmentAnnouncer segmentAnnouncer; - private final IndexIO indexIO; - private final IndexMerger indexMerger; - private final Cache cache; - /** - * This map needs to be concurrent because it's accessed and mutated from multiple threads: both the thread from where - * this Appenderator is used (and methods like {@link #add(SegmentIdWithShardSpec, InputRow, Supplier, boolean)} are - * called) and from {@link #persistExecutor}. It could also be accessed (but not mutated) potentially in the context - * of any thread from {@link #drop}. - */ - private final ConcurrentMap sinks = new ConcurrentHashMap<>(); - private final Set droppingSinks = Sets.newConcurrentHashSet(); - private final VersionedIntervalTimeline sinkTimeline; - private final long maxBytesTuningConfig; - private final boolean skipBytesInMemoryOverheadCheck; - - private final QuerySegmentWalker texasRanger; - // This variable updated in add(), persist(), and drop() - private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger(); - private final AtomicInteger totalRows = new AtomicInteger(); - private final AtomicLong bytesCurrentlyInMemory = new AtomicLong(); - private final RowIngestionMeters rowIngestionMeters; - private final ParseExceptionHandler parseExceptionHandler; - // Synchronize persisting commitMetadata so that multiple persist threads (if present) - // and abandon threads do not step over each other - private final Lock commitLock = new ReentrantLock(); - - private final AtomicBoolean closed = new AtomicBoolean(false); - - private volatile ListeningExecutorService persistExecutor = null; - private volatile ListeningExecutorService pushExecutor = null; - // use intermediate executor so that deadlock conditions can be prevented - // where persist and push Executor try to put tasks in each other queues - // thus creating circular dependency - private volatile ListeningExecutorService intermediateTempExecutor = null; - private volatile long nextFlush; - private volatile FileLock basePersistDirLock = null; - private volatile FileChannel basePersistDirLockChannel = null; - - private volatile Throwable persistError; - - private final boolean isOpenSegments; - private final boolean useMaxMemoryEstimates; - - /** - * Use next Map to store metadata (File, SegmentId) for a hydrant for batch appenderator - * in order to facilitate the mapping of the QueryableIndex associated with a given hydrant - * at merge time. This is necessary since batch appenderator will not map the QueryableIndex - * at persist time in order to minimize its memory footprint. This has to be synchronized since the - * map may be accessed from multiple threads. - * Use {@link IdentityHashMap} to better reflect the fact that the key needs to be interpreted - * with reference semantics. - */ - private final Map> persistedHydrantMetadata = - Collections.synchronizedMap(new IdentityHashMap<>()); - - private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - - private final FingerprintGenerator fingerprintGenerator; - - /** - * This constructor allows the caller to provide its own SinkQuerySegmentWalker. - * - * The sinkTimeline is set to the sink timeline of the provided SinkQuerySegmentWalker. - * If the SinkQuerySegmentWalker is null, a new sink timeline is initialized. - * - * It is used by UnifiedIndexerAppenderatorsManager which allows queries on data associated with multiple - * Appenderators. - */ - AppenderatorImpl( - String id, - DataSchema schema, - AppenderatorConfig tuningConfig, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - DataSegmentAnnouncer segmentAnnouncer, - @Nullable SinkQuerySegmentWalker sinkQuerySegmentWalker, - IndexIO indexIO, - IndexMerger indexMerger, - Cache cache, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean isOpenSegments, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - this.myId = id; - this.schema = Preconditions.checkNotNull(schema, "schema"); - this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); - this.metrics = Preconditions.checkNotNull(metrics, "metrics"); - this.dataSegmentPusher = Preconditions.checkNotNull(dataSegmentPusher, "dataSegmentPusher"); - this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); - this.segmentAnnouncer = Preconditions.checkNotNull(segmentAnnouncer, "segmentAnnouncer"); - this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO"); - this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger"); - this.cache = cache; - this.texasRanger = sinkQuerySegmentWalker; - this.rowIngestionMeters = Preconditions.checkNotNull(rowIngestionMeters, "rowIngestionMeters"); - this.parseExceptionHandler = Preconditions.checkNotNull(parseExceptionHandler, "parseExceptionHandler"); - this.isOpenSegments = isOpenSegments; - this.useMaxMemoryEstimates = useMaxMemoryEstimates; - - if (sinkQuerySegmentWalker == null) { - this.sinkTimeline = new VersionedIntervalTimeline<>( - String.CASE_INSENSITIVE_ORDER - ); - } else { - this.sinkTimeline = sinkQuerySegmentWalker.getSinkTimeline(); - } - - maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault(); - skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck(); - - if (isOpenSegments) { - log.debug("Running open segments appenderator"); - } else { - log.debug("Running closed segments appenderator"); - } - this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; - this.fingerprintGenerator = new FingerprintGenerator(objectMapper); - } - - @Override - public String getId() - { - return myId; - } - - @Override - public String getDataSource() - { - return schema.getDataSource(); - } - - @Override - public Object startJob() - { - lockBasePersistDirectory(); - final Object retVal = bootstrapSinksFromDisk(); - initializeExecutors(); - resetNextFlush(); - return retVal; - } - - private void throwPersistErrorIfExists() - { - if (persistError != null) { - throw new RE(persistError, "Error while persisting"); - } - } - - @Override - public AppenderatorAddResult add( - final SegmentIdWithShardSpec identifier, - final InputRow row, - @Nullable final Supplier committerSupplier, - final boolean allowIncrementalPersists - ) throws IndexSizeExceededException, SegmentNotWritableException - { - throwPersistErrorIfExists(); - - if (!identifier.getDataSource().equals(schema.getDataSource())) { - throw new IAE( - "Expected dataSource[%s] but was asked to insert row for dataSource[%s]?!", - schema.getDataSource(), - identifier.getDataSource() - ); - } - - final Sink sink = getOrCreateSink(identifier); - metrics.reportMessageMaxTimestamp(row.getTimestampFromEpoch()); - final int sinkRowsInMemoryBeforeAdd = sink.getNumRowsInMemory(); - final int sinkRowsInMemoryAfterAdd; - final long bytesInMemoryBeforeAdd = sink.getBytesInMemory(); - final long bytesInMemoryAfterAdd; - final IncrementalIndexAddResult addResult; - - try { - addResult = sink.add(row, !allowIncrementalPersists); - sinkRowsInMemoryAfterAdd = addResult.getRowCount(); - bytesInMemoryAfterAdd = addResult.getBytesInMemory(); - } - catch (IndexSizeExceededException e) { - // Uh oh, we can't do anything about this! We can't persist (commit metadata would be out of sync) and we - // can't add the row (it just failed). This should never actually happen, though, because we check - // sink.canAddRow after returning from add. - log.error(e, "Sink for segment[%s] was unexpectedly full!", identifier); - throw e; - } - - if (sinkRowsInMemoryAfterAdd < 0) { - throw new SegmentNotWritableException("Attempt to add row to swapped-out sink for segment[%s].", identifier); - } - - if (addResult.isRowAdded()) { - rowIngestionMeters.incrementProcessed(); - } else if (addResult.hasParseException()) { - parseExceptionHandler.handle(addResult.getParseException()); - } - - final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd; - rowsCurrentlyInMemory.addAndGet(numAddedRows); - bytesCurrentlyInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd); - totalRows.addAndGet(numAddedRows); - - boolean isPersistRequired = false; - boolean persist = false; - List persistReasons = new ArrayList<>(); - - if (!sink.canAppendRow()) { - persist = true; - persistReasons.add("No more rows can be appended to sink"); - } - if (System.currentTimeMillis() > nextFlush) { - persist = true; - persistReasons.add(StringUtils.format( - "current time[%d] is greater than nextFlush[%d]", - System.currentTimeMillis(), - nextFlush - )); - } - if (rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory()) { - persist = true; - persistReasons.add(StringUtils.format( - "rowsCurrentlyInMemory[%d] is greater than maxRowsInMemory[%d]", - rowsCurrentlyInMemory.get(), - tuningConfig.getMaxRowsInMemory() - )); - } - if (bytesCurrentlyInMemory.get() >= maxBytesTuningConfig) { - persist = true; - persistReasons.add(StringUtils.format( - "(estimated) bytesCurrentlyInMemory[%d] is greater than maxBytesInMemory[%d]", - bytesCurrentlyInMemory.get(), - maxBytesTuningConfig - )); - } - if (persist) { - if (allowIncrementalPersists) { - // persistAll clears rowsCurrentlyInMemory, no need to update it. - log.info("Flushing in-memory data to disk because %s.", String.join(",", persistReasons)); - - long bytesToBePersisted = 0L; - for (Map.Entry entry : sinks.entrySet()) { - final Sink sinkEntry = entry.getValue(); - if (sinkEntry != null) { - bytesToBePersisted += sinkEntry.getBytesInMemory(); - if (sinkEntry.swappable()) { - // After swapping the sink, we use memory mapped segment instead (but only for real time appenderators!). - // However, the memory mapped segment still consumes memory. - // These memory mapped segments are held in memory throughout the ingestion phase and permanently add to the bytesCurrentlyInMemory - int memoryStillInUse = calculateMMappedHydrantMemoryInUsed(sink.getCurrHydrant()); - bytesCurrentlyInMemory.addAndGet(memoryStillInUse); - } - } - } - - if (!skipBytesInMemoryOverheadCheck - && bytesCurrentlyInMemory.get() - bytesToBePersisted > maxBytesTuningConfig) { - // We are still over maxBytesTuningConfig even after persisting. - // This means that we ran out of all available memory to ingest (due to overheads created as part of ingestion) - final String alertMessage = StringUtils.format( - "Task has exceeded safe estimated heap usage limits, failing " - + "(numSinks: [%d] numHydrantsAcrossAllSinks: [%d] totalRows: [%d])" - + "(bytesCurrentlyInMemory: [%d] - bytesToBePersisted: [%d] > maxBytesTuningConfig: [%d])", - sinks.size(), - sinks.values().stream().mapToInt(Iterables::size).sum(), - getTotalRowCount(), - bytesCurrentlyInMemory.get(), - bytesToBePersisted, - maxBytesTuningConfig - ); - final String errorMessage = StringUtils.format( - "%s.\nThis can occur when the overhead from too many intermediary segment persists becomes to " - + "great to have enough space to process additional input rows. This check, along with metering the overhead " - + "of these objects to factor into the 'maxBytesInMemory' computation, can be disabled by setting " - + "'skipBytesInMemoryOverheadCheck' to 'true' (note that doing so might allow the task to naturally encounter " - + "a 'java.lang.OutOfMemoryError'). Alternatively, 'maxBytesInMemory' can be increased which will cause an " - + "increase in heap footprint, but will allow for more intermediary segment persists to occur before " - + "reaching this condition.", - alertMessage - ); - log.makeAlert(alertMessage) - .addData("dataSource", schema.getDataSource()) - .emit(); - throw new RuntimeException(errorMessage); - } - - Futures.addCallback( - persistAll(committerSupplier == null ? null : committerSupplier.get()), - new FutureCallback() - { - @Override - public void onSuccess(@Nullable Object result) - { - // do nothing - } - - @Override - public void onFailure(Throwable t) - { - persistError = t; - } - }, - MoreExecutors.directExecutor() - ); - } else { - isPersistRequired = true; - } - } - return new AppenderatorAddResult(identifier, sink.getNumRows(), isPersistRequired); - } - - @Override - public List getSegments() - { - return ImmutableList.copyOf(sinks.keySet()); - } - - @Override - public int getRowCount(final SegmentIdWithShardSpec identifier) - { - final Sink sink = sinks.get(identifier); - - if (sink == null) { - throw new ISE("No such sink: %s", identifier); - } else { - return sink.getNumRows(); - } - } - - @Override - public int getTotalRowCount() - { - return totalRows.get(); - } - - @VisibleForTesting - int getRowsInMemory() - { - return rowsCurrentlyInMemory.get(); - } - - @VisibleForTesting - long getBytesCurrentlyInMemory() - { - return bytesCurrentlyInMemory.get(); - } - - @VisibleForTesting - long getBytesInMemory(SegmentIdWithShardSpec identifier) - { - final Sink sink = sinks.get(identifier); - - if (sink == null) { - throw new ISE("No such sink: %s", identifier); - } else { - return sink.getBytesInMemory(); - } - } - - private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) - { - Sink retVal = sinks.get(identifier); - - if (retVal == null) { - retVal = new Sink( - identifier.getInterval(), - schema, - identifier.getShardSpec(), - identifier.getVersion(), - tuningConfig.getAppendableIndexSpec(), - tuningConfig.getMaxRowsInMemory(), - maxBytesTuningConfig, - useMaxMemoryEstimates - ); - bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed()); - - try { - segmentAnnouncer.announceSegment(retVal.getSegment()); - } - catch (IOException e) { - log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource()) - .addData("interval", retVal.getInterval()) - .emit(); - } - - sinks.put(identifier, retVal); - metrics.setSinkCount(sinks.size()); - sinkTimeline.add(retVal.getInterval(), retVal.getVersion(), identifier.getShardSpec().createChunk(retVal)); - } - - return retVal; - } - - @Override - public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) - { - if (texasRanger == null) { - throw new IllegalStateException("Don't query me, bro."); - } - - return texasRanger.getQueryRunnerForIntervals(query, intervals); - } - - @Override - public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) - { - if (texasRanger == null) { - throw new IllegalStateException("Don't query me, bro."); - } - - return texasRanger.getQueryRunnerForSegments(query, specs); - } - - @Override - public void clear() throws InterruptedException - { - // Drop commit metadata, then abandon all segments. - - try { - throwPersistErrorIfExists(); - - if (persistExecutor != null) { - final ListenableFuture uncommitFuture = persistExecutor.submit( - () -> { - try { - commitLock.lock(); - objectMapper.writeValue(computeCommitFile(), Committed.nil()); - } - finally { - commitLock.unlock(); - } - return null; - } - ); - - // Await uncommit. - uncommitFuture.get(); - - // Drop everything. - final List> futures = new ArrayList<>(); - for (Map.Entry entry : sinks.entrySet()) { - futures.add(abandonSegment(entry.getKey(), entry.getValue(), true)); - } - - // Re-initialize hydrant map: - persistedHydrantMetadata.clear(); - - // Await dropping. - Futures.allAsList(futures).get(); - } - } - catch (ExecutionException e) { - throw new RuntimeException(e); - } - } - - @Override - public ListenableFuture drop(final SegmentIdWithShardSpec identifier) - { - final Sink sink = sinks.get(identifier); - if (sink != null) { - return abandonSegment(identifier, sink, true); - } else { - return Futures.immediateFuture(null); - } - } - - @Override - public ListenableFuture persistAll(@Nullable final Committer committer) - { - throwPersistErrorIfExists(); - final Map currentHydrants = new HashMap<>(); - final List> indexesToPersist = new ArrayList<>(); - int numPersistedRows = 0; - long bytesPersisted = 0L; - MutableLong totalHydrantsCount = new MutableLong(); - MutableLong totalHydrantsPersisted = new MutableLong(); - final long totalSinks = sinks.size(); - for (Map.Entry entry : sinks.entrySet()) { - final SegmentIdWithShardSpec identifier = entry.getKey(); - final Sink sink = entry.getValue(); - if (sink == null) { - throw new ISE("No sink for identifier: %s", identifier); - } - final List hydrants = Lists.newArrayList(sink); - totalHydrantsCount.add(hydrants.size()); - currentHydrants.put(identifier.toString(), hydrants.size()); - numPersistedRows += sink.getNumRowsInMemory(); - bytesPersisted += sink.getBytesInMemory(); - - final int limit = sink.isWritable() ? hydrants.size() - 1 : hydrants.size(); - - // gather hydrants that have not been persisted: - for (FireHydrant hydrant : hydrants.subList(0, limit)) { - if (!hydrant.hasSwapped()) { - log.debug("Hydrant[%s] hasn't persisted yet, persisting. Segment[%s]", hydrant, identifier); - indexesToPersist.add(Pair.of(hydrant, identifier)); - totalHydrantsPersisted.add(1); - } - } - - if (sink.swappable()) { - // It is swappable. Get the old one to persist it and create a new one: - indexesToPersist.add(Pair.of(sink.swap(), identifier)); - totalHydrantsPersisted.add(1); - } - } - log.debug("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); - - final Object commitMetadata = committer == null ? null : committer.getMetadata(); - final Stopwatch runExecStopwatch = Stopwatch.createStarted(); - final Stopwatch persistStopwatch = Stopwatch.createStarted(); - AtomicLong totalPersistedRows = new AtomicLong(numPersistedRows); - final ListenableFuture future = persistExecutor.submit( - new Callable() - { - @Override - public Object call() throws IOException - { - try { - for (Pair pair : indexesToPersist) { - metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs)); - } - - if (committer != null) { - log.debug( - "Committing metadata[%s] for sinks[%s].", - commitMetadata, - Joiner.on(", ").join( - currentHydrants.entrySet() - .stream() - .map(entry -> StringUtils.format( - "%s:%d", - entry.getKey(), - entry.getValue() - )) - .collect(Collectors.toList()) - ) - ); - - committer.run(); - - try { - commitLock.lock(); - final Map commitHydrants = new HashMap<>(); - final Committed oldCommit = readCommit(); - if (oldCommit != null) { - // merge current hydrants with existing hydrants - commitHydrants.putAll(oldCommit.getHydrants()); - } - commitHydrants.putAll(currentHydrants); - writeCommit(new Committed(commitHydrants, commitMetadata)); - } - finally { - commitLock.unlock(); - } - } - - log.info( - "Flushed in-memory data with commit metadata [%s] for segments: %s", - commitMetadata, - indexesToPersist.stream() - .map(itp -> itp.rhs.asSegmentId().toString()) - .distinct() - .collect(Collectors.joining(", ")) - ); - log.info( - "Persisted stats: processed rows: [%d], persisted rows[%d], sinks: [%d], total fireHydrants (across sinks): [%d], persisted fireHydrants (across sinks): [%d]", - rowIngestionMeters.getProcessed(), - totalPersistedRows.get(), - totalSinks, - totalHydrantsCount.longValue(), - totalHydrantsPersisted.longValue() - ); - - // return null if committer is null - return commitMetadata; - } - catch (IOException e) { - metrics.incrementFailedPersists(); - throw e; - } - finally { - metrics.incrementNumPersists(); - metrics.incrementPersistTimeMillis(persistStopwatch.elapsed(TimeUnit.MILLISECONDS)); - persistStopwatch.stop(); - } - } - } - ); - - final long startDelay = runExecStopwatch.elapsed(TimeUnit.MILLISECONDS); - metrics.incrementPersistBackPressureMillis(startDelay); - if (startDelay > WARN_DELAY) { - log.warn("Ingestion was throttled for [%,d] millis because persists were pending.", startDelay); - } - runExecStopwatch.stop(); - resetNextFlush(); - - // NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes. - rowsCurrentlyInMemory.addAndGet(-numPersistedRows); - bytesCurrentlyInMemory.addAndGet(-bytesPersisted); - - log.info("Persisted rows[%,d] and (estimated) bytes[%,d]", numPersistedRows, bytesPersisted); - - return future; - } - - @Override - public ListenableFuture push( - final Collection identifiers, - @Nullable final Committer committer, - final boolean useUniquePath - ) - { - final Map theSinks = new HashMap<>(); - AtomicLong pushedHydrantsCount = new AtomicLong(); - for (final SegmentIdWithShardSpec identifier : identifiers) { - final Sink sink = sinks.get(identifier); - if (sink == null) { - throw new ISE("No sink for identifier: %s", identifier); - } - theSinks.put(identifier, sink); - if (sink.finishWriting()) { - totalRows.addAndGet(-sink.getNumRows()); - } - // count hydrants for stats: - pushedHydrantsCount.addAndGet(Iterables.size(sink)); - } - - return Futures.transform( - // We should always persist all segments regardless of the input because metadata should be committed for all - // segments. - persistAll(committer), - (Function) commitMetadata -> { - final List dataSegments = new ArrayList<>(); - final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); - - log.info("Preparing to push (stats): processed rows: [%d], sinks: [%d], fireHydrants (across sinks): [%d]", - rowIngestionMeters.getProcessed(), theSinks.size(), pushedHydrantsCount.get() - ); - - log.debug( - "Building and pushing segments: %s", - theSinks.keySet().stream().map(SegmentIdWithShardSpec::toString).collect(Collectors.joining(", ")) - ); - - for (Map.Entry entry : theSinks.entrySet()) { - if (droppingSinks.contains(entry.getKey())) { - log.warn("Skipping push of currently-dropping sink[%s]", entry.getKey()); - continue; - } - - final DataSegmentWithMetadata dataSegmentWithMetadata = mergeAndPush( - entry.getKey(), - entry.getValue(), - useUniquePath - ); - - if (dataSegmentWithMetadata != null) { - DataSegment segment = dataSegmentWithMetadata.getDataSegment(); - dataSegments.add(segment); - SchemaPayloadPlus schemaPayloadPlus = dataSegmentWithMetadata.getSegmentSchemaMetadata(); - if (schemaPayloadPlus != null) { - SchemaPayload schemaPayload = schemaPayloadPlus.getSchemaPayload(); - segmentSchemaMapping.addSchema( - segment.getId(), - schemaPayloadPlus, - fingerprintGenerator.generateFingerprint( - schemaPayload, - segment.getDataSource(), - CentralizedDatasourceSchemaConfig.SCHEMA_VERSION - ) - ); - } - } else { - log.warn("mergeAndPush[%s] returned null, skipping.", entry.getKey()); - } - } - - log.info("Push complete..."); - - return new SegmentsAndCommitMetadata(dataSegments, commitMetadata, segmentSchemaMapping); - }, - pushExecutor - ); - } - - /** - * Insert a barrier into the merge-and-push queue. When this future resolves, all pending pushes will have finished. - * This is useful if we're going to do something that would otherwise potentially break currently in-progress - * pushes. - */ - private ListenableFuture pushBarrier() - { - return intermediateTempExecutor.submit( - (Runnable) () -> pushExecutor.submit(() -> {}) - ); - } - - /** - * Merge segment, push to deep storage. Should only be used on segments that have been fully persisted. Must only - * be run in the single-threaded pushExecutor. - * - * @param identifier sink identifier - * @param sink sink to push - * @param useUniquePath true if the segment should be written to a path with a unique identifier - * - * @return segment descriptor, or null if the sink is no longer valid - */ - @Nullable - private DataSegmentWithMetadata mergeAndPush( - final SegmentIdWithShardSpec identifier, - final Sink sink, - final boolean useUniquePath - ) - { - // Bail out if this sink is null or otherwise not what we expect. - //noinspection ObjectEquality - if (sinks.get(identifier) != sink) { - log.warn("Sink for segment[%s] no longer valid, bailing out of mergeAndPush.", identifier); - return null; - } - - // Use a descriptor file to indicate that pushing has completed. - final File persistDir = computePersistDir(identifier); - final File mergedTarget = new File(persistDir, "merged"); - final File descriptorFile = computeDescriptorFile(identifier); - - // Sanity checks - for (FireHydrant hydrant : sink) { - if (sink.isWritable()) { - throw new ISE("Expected sink to be no longer writable before mergeAndPush for segment[%s].", identifier); - } - - synchronized (hydrant) { - if (!hydrant.hasSwapped()) { - throw new ISE("Expected sink to be fully persisted before mergeAndPush for segment[%s].", identifier); - } - } - } - - try { - if (descriptorFile.exists()) { - // Already pushed. - - if (useUniquePath) { - // Don't reuse the descriptor, because the caller asked for a unique path. Leave the old one as-is, since - // it might serve some unknown purpose. - log.debug( - "Segment[%s] already pushed, but we want a unique path, so will push again with a new path.", - identifier - ); - } else { - log.info("Segment[%s] already pushed, skipping.", identifier); - return new DataSegmentWithMetadata( - objectMapper.readValue(descriptorFile, DataSegment.class), - centralizedDatasourceSchemaConfig.isEnabled() ? TaskSegmentSchemaUtil.getSegmentSchema( - mergedTarget, - indexIO - ) : null - ); - } - } - - removeDirectory(mergedTarget); - - if (mergedTarget.exists()) { - throw new ISE("Merged target[%s] exists after removing?!", mergedTarget); - } - - final File mergedFile; - final long mergeFinishTime; - final long startTime = System.nanoTime(); - List indexes = new ArrayList<>(); - Closer closer = Closer.create(); - try { - for (FireHydrant fireHydrant : sink) { - - // if batch, swap/persist did not memory map the incremental index, we need it mapped now: - if (!isOpenSegments()) { - - // sanity - Pair persistedMetadata = persistedHydrantMetadata.get(fireHydrant); - if (persistedMetadata == null) { - throw new ISE("Persisted metadata for batch hydrant [%s] is null!", fireHydrant); - } - - File persistedFile = persistedMetadata.lhs; - SegmentId persistedSegmentId = persistedMetadata.rhs; - - // sanity: - if (persistedFile == null) { - throw new ISE("Persisted file for batch hydrant [%s] is null!", fireHydrant); - } else if (persistedSegmentId == null) { - throw new ISE( - "Persisted segmentId for batch hydrant in file [%s] is null!", - persistedFile.getPath() - ); - } - fireHydrant.swapSegment(new QueryableIndexSegment( - indexIO.loadIndex(persistedFile), - persistedSegmentId - )); - } - - Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); - final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); - log.debug("Segment[%s] adding hydrant[%s]", identifier, fireHydrant); - indexes.add(queryableIndex); - closer.register(segmentAndCloseable.rhs); - } - - mergedFile = indexMerger.mergeQueryableIndex( - indexes, - schema.getGranularitySpec().isRollup(), - schema.getAggregators(), - schema.getDimensionsSpec(), - mergedTarget, - tuningConfig.getIndexSpec(), - tuningConfig.getIndexSpecForIntermediatePersists(), - new BaseProgressIndicator(), - tuningConfig.getSegmentWriteOutMediumFactory(), - tuningConfig.getMaxColumnsToMerge() - ); - - mergeFinishTime = System.nanoTime(); - - log.debug("Segment[%s] built in %,dms.", identifier, (mergeFinishTime - startTime) / 1000000); - } - catch (Throwable t) { - throw closer.rethrow(t); - } - finally { - closer.close(); - } - - final DataSegment segmentToPush = sink.getSegment().withDimensions( - IndexMerger.getMergedDimensionsFromQueryableIndexes(indexes, schema.getDimensionsSpec()) - ); - - // The appenderator is currently being used for the local indexing task and the Kafka indexing task. For the - // Kafka indexing task, pushers must use unique file paths in deep storage in order to maintain exactly-once - // semantics. - // - // dataSegmentPusher retries internally when appropriate; no need for retries here. - final DataSegment segment = dataSegmentPusher.push(mergedFile, segmentToPush, useUniquePath); - - if (!isOpenSegments()) { - // Drop the queryable indexes behind the hydrants... they are not needed anymore and their - // mapped file references - // can generate OOMs during merge if enough of them are held back... - for (FireHydrant fireHydrant : sink) { - fireHydrant.swapSegment(null); - } - } - - final long pushFinishTime = System.nanoTime(); - - objectMapper.writeValue(descriptorFile, segment); - - log.info( - "Segment[%s] of %,d bytes " - + "built from %d incremental persist(s) in %,dms; " - + "pushed to deep storage in %,dms. " - + "Load spec is: %s", - identifier, - segment.getSize(), - indexes.size(), - (mergeFinishTime - startTime) / 1000000, - (pushFinishTime - mergeFinishTime) / 1000000, - objectMapper.writeValueAsString(segment.getLoadSpec()) - ); - - return new DataSegmentWithMetadata( - segment, - centralizedDatasourceSchemaConfig.isEnabled() - ? TaskSegmentSchemaUtil.getSegmentSchema(mergedTarget, indexIO) - : null - ); - } - catch (Exception e) { - metrics.incrementFailedHandoffs(); - log.warn(e, "Failed to push merged index for segment[%s].", identifier); - throw new RuntimeException(e); - } - } - - @Override - public void close() - { - if (!closed.compareAndSet(false, true)) { - log.debug("Appenderator already closed, skipping close() call."); - return; - } - - log.debug("Shutting down..."); - - final List> futures = new ArrayList<>(); - for (Map.Entry entry : sinks.entrySet()) { - futures.add(abandonSegment(entry.getKey(), entry.getValue(), false)); - } - - try { - Futures.allAsList(futures).get(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - log.warn(e, "Interrupted during close()"); - } - catch (ExecutionException e) { - log.warn(e, "Unable to abandon existing segments during close()"); - } - - try { - shutdownExecutors(); - Preconditions.checkState( - persistExecutor == null || persistExecutor.awaitTermination(365, TimeUnit.DAYS), - "persistExecutor not terminated" - ); - Preconditions.checkState( - pushExecutor == null || pushExecutor.awaitTermination(365, TimeUnit.DAYS), - "pushExecutor not terminated" - ); - Preconditions.checkState( - intermediateTempExecutor == null || intermediateTempExecutor.awaitTermination(365, TimeUnit.DAYS), - "intermediateTempExecutor not terminated" - ); - persistExecutor = null; - pushExecutor = null; - intermediateTempExecutor = null; - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new ISE("Failed to shutdown executors during close()"); - } - - // Only unlock if executors actually shut down. - unlockBasePersistDirectory(); - } - - /** - * Unannounce the segments and wait for outstanding persists to finish. - * Do not unlock base persist dir as we are not waiting for push executor to shut down - * relying on current JVM to shutdown to not cause any locking problem if the task is restored. - * In case when task is restored and current task is still active because of push executor (which it shouldn't be - * since push executor starts daemon threads) then the locking should fail and new task should fail to start. - * This also means that this method should only be called when task is shutting down. - */ - @Override - public void closeNow() - { - if (!closed.compareAndSet(false, true)) { - log.debug("Appenderator already closed, skipping closeNow() call."); - return; - } - - log.debug("Shutting down immediately..."); - for (Map.Entry entry : sinks.entrySet()) { - try { - segmentAnnouncer.unannounceSegment(entry.getValue().getSegment()); - } - catch (Exception e) { - log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) - .addData("identifier", entry.getKey().toString()) - .emit(); - } - } - try { - shutdownExecutors(); - // We don't wait for pushExecutor to be terminated. See Javadoc for more details. - Preconditions.checkState( - persistExecutor == null || persistExecutor.awaitTermination(365, TimeUnit.DAYS), - "persistExecutor not terminated" - ); - Preconditions.checkState( - intermediateTempExecutor == null || intermediateTempExecutor.awaitTermination(365, TimeUnit.DAYS), - "intermediateTempExecutor not terminated" - ); - persistExecutor = null; - intermediateTempExecutor = null; - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new ISE("Failed to shutdown executors during close()"); - } - } - - public boolean isOpenSegments() - { - return isOpenSegments; - } - - private void lockBasePersistDirectory() - { - if (basePersistDirLock == null) { - try { - FileUtils.mkdirp(tuningConfig.getBasePersistDirectory()); - - basePersistDirLockChannel = FileChannel.open( - computeLockFile().toPath(), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE - ); - - basePersistDirLock = basePersistDirLockChannel.tryLock(); - if (basePersistDirLock == null) { - throw new ISE("Cannot acquire lock on basePersistDir: %s", computeLockFile()); - } - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - } - - private void unlockBasePersistDirectory() - { - try { - if (basePersistDirLock != null) { - basePersistDirLock.release(); - basePersistDirLockChannel.close(); - basePersistDirLock = null; - } - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - private void initializeExecutors() - { - final int maxPendingPersists = tuningConfig.getMaxPendingPersists(); - - if (persistExecutor == null) { - // use a blocking single threaded executor to throttle the firehose when write to disk is slow - persistExecutor = MoreExecutors.listeningDecorator( - Execs.newBlockingThreaded( - "[" + StringUtils.encodeForFormat(myId) + "]-appenderator-persist", - tuningConfig.getNumPersistThreads(), maxPendingPersists - ) - ); - } - - if (pushExecutor == null) { - // use a blocking single threaded executor to throttle the firehose when write to disk is slow - pushExecutor = MoreExecutors.listeningDecorator( - Execs.newBlockingSingleThreaded("[" + StringUtils.encodeForFormat(myId) + "]-appenderator-merge", 1) - ); - } - - if (intermediateTempExecutor == null) { - // use single threaded executor with SynchronousQueue so that all abandon operations occur sequentially - intermediateTempExecutor = MoreExecutors.listeningDecorator( - Execs.newBlockingSingleThreaded("[" + StringUtils.encodeForFormat(myId) + "]-appenderator-abandon", 0) - ); - } - } - - private void shutdownExecutors() - { - if (persistExecutor != null) { - persistExecutor.shutdownNow(); - } - - if (pushExecutor != null) { - pushExecutor.shutdownNow(); - } - - if (intermediateTempExecutor != null) { - intermediateTempExecutor.shutdownNow(); - } - } - - private void resetNextFlush() - { - nextFlush = DateTimes.nowUtc().plus(tuningConfig.getIntermediatePersistPeriod()).getMillis(); - } - - /** - * Populate "sinks" and "sinkTimeline" with committed segments, and announce them with the segmentAnnouncer. - * - * @return persisted commit metadata - */ - private Object bootstrapSinksFromDisk() - { - Preconditions.checkState(sinks.isEmpty(), "Already bootstrapped?!"); - - final File baseDir = tuningConfig.getBasePersistDirectory(); - if (!baseDir.exists()) { - return null; - } - - final File[] files = baseDir.listFiles(); - if (files == null) { - return null; - } - - - final Committed committed; - File commitFile = null; - try { - commitLock.lock(); - commitFile = computeCommitFile(); - if (commitFile.exists()) { - committed = objectMapper.readValue(commitFile, Committed.class); - } else { - committed = Committed.nil(); - } - } - catch (Exception e) { - throw new ISE(e, "Failed to read commitFile: %s", commitFile); - } - finally { - commitLock.unlock(); - } - - int rowsSoFar = 0; - - if (committed.equals(Committed.nil())) { - log.debug("No previously committed metadata."); - } else { - log.info( - "Loading partially-persisted segments[%s] from[%s] with commit metadata: %s", - String.join(", ", committed.getHydrants().keySet()), - baseDir, - committed.getMetadata() - ); - } - - for (File sinkDir : files) { - final File identifierFile = new File(sinkDir, IDENTIFIER_FILE_NAME); - if (!identifierFile.isFile()) { - // No identifier in this sinkDir; it must not actually be a sink directory. Skip it. - continue; - } - - try { - final SegmentIdWithShardSpec identifier = objectMapper.readValue( - new File(sinkDir, "identifier.json"), - SegmentIdWithShardSpec.class - ); - - final int committedHydrants = committed.getCommittedHydrants(identifier.toString()); - - if (committedHydrants <= 0) { - log.info("Removing uncommitted segment at [%s].", sinkDir); - FileUtils.deleteDirectory(sinkDir); - continue; - } - - // To avoid reading and listing of "merged" dir and other special files - final File[] sinkFiles = sinkDir.listFiles( - (dir, fileName) -> !(Ints.tryParse(fileName) == null) - ); - - Arrays.sort( - sinkFiles, - (o1, o2) -> Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName())) - ); - - List hydrants = new ArrayList<>(); - for (File hydrantDir : sinkFiles) { - final int hydrantNumber = Integer.parseInt(hydrantDir.getName()); - - if (hydrantNumber >= committedHydrants) { - log.info("Removing uncommitted partial segment at [%s]", hydrantDir); - FileUtils.deleteDirectory(hydrantDir); - } else { - log.debug("Loading previously persisted partial segment at [%s]", hydrantDir); - if (hydrantNumber != hydrants.size()) { - throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir); - } - - hydrants.add( - new FireHydrant( - new QueryableIndexSegment(indexIO.loadIndex(hydrantDir), identifier.asSegmentId()), - hydrantNumber - ) - ); - } - } - - // Make sure we loaded enough hydrants. - if (committedHydrants != hydrants.size()) { - throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir); - } - - Sink currSink = new Sink( - identifier.getInterval(), - schema, - identifier.getShardSpec(), - identifier.getVersion(), - tuningConfig.getAppendableIndexSpec(), - tuningConfig.getMaxRowsInMemory(), - maxBytesTuningConfig, - useMaxMemoryEstimates, - hydrants - ); - rowsSoFar += currSink.getNumRows(); - sinks.put(identifier, currSink); - sinkTimeline.add( - currSink.getInterval(), - currSink.getVersion(), - identifier.getShardSpec().createChunk(currSink) - ); - - segmentAnnouncer.announceSegment(currSink.getSegment()); - } - catch (IOException e) { - log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource()) - .addData("sinkDir", sinkDir) - .emit(); - } - } - - // Make sure we loaded all committed sinks. - final Set loadedSinks = Sets.newHashSet( - Iterables.transform(sinks.keySet(), SegmentIdWithShardSpec::toString) - ); - final Set missingSinks = Sets.difference(committed.getHydrants().keySet(), loadedSinks); - if (!missingSinks.isEmpty()) { - throw new ISE("Missing committed sinks [%s]", Joiner.on(", ").join(missingSinks)); - } - - totalRows.set(rowsSoFar); - return committed.getMetadata(); - } - - private ListenableFuture abandonSegment( - final SegmentIdWithShardSpec identifier, - final Sink sink, - final boolean removeOnDiskData - ) - { - // Ensure no future writes will be made to this sink. - if (sink.finishWriting()) { - // Decrement this sink's rows from the counters. we only count active sinks so that we don't double decrement, - // i.e. those that haven't been persisted for *InMemory counters, or pushed to deep storage for the total counter. - rowsCurrentlyInMemory.addAndGet(-sink.getNumRowsInMemory()); - bytesCurrentlyInMemory.addAndGet(-sink.getBytesInMemory()); - bytesCurrentlyInMemory.addAndGet(-calculateSinkMemoryInUsed()); - for (FireHydrant hydrant : sink) { - // Decrement memory used by all Memory Mapped Hydrant - if (!hydrant.equals(sink.getCurrHydrant())) { - bytesCurrentlyInMemory.addAndGet(-calculateMMappedHydrantMemoryInUsed(hydrant)); - } - } - totalRows.addAndGet(-sink.getNumRows()); - } - - // Mark this identifier as dropping, so no future push tasks will pick it up. - droppingSinks.add(identifier); - - // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. - return Futures.transform( - pushBarrier(), - new Function() - { - @Nullable - @Override - public Void apply(@Nullable Object input) - { - if (!sinks.remove(identifier, sink)) { - log.error("Sink for segment[%s] no longer valid, not abandoning.", identifier); - return null; - } - - metrics.setSinkCount(sinks.size()); - - if (removeOnDiskData) { - // Remove this segment from the committed list. This must be done from the persist thread. - log.debug("Removing commit metadata for segment[%s].", identifier); - try { - commitLock.lock(); - final Committed oldCommit = readCommit(); - if (oldCommit != null) { - writeCommit(oldCommit.without(identifier.toString())); - } - } - catch (Exception e) { - log.makeAlert(e, "Failed to update committed segments[%s]", schema.getDataSource()) - .addData("identifier", identifier.toString()) - .emit(); - throw new RuntimeException(e); - } - finally { - commitLock.unlock(); - } - } - - // Unannounce the segment. - try { - segmentAnnouncer.unannounceSegment(sink.getSegment()); - } - catch (Exception e) { - log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) - .addData("identifier", identifier.toString()) - .emit(); - } - - droppingSinks.remove(identifier); - sinkTimeline.remove( - sink.getInterval(), - sink.getVersion(), - identifier.getShardSpec().createChunk(sink) - ); - for (FireHydrant hydrant : sink) { - if (cache != null) { - cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant)); - } - hydrant.swapSegment(null); - // remove hydrant from persisted metadata: - persistedHydrantMetadata.remove(hydrant); - } - - if (removeOnDiskData) { - removeDirectory(computePersistDir(identifier)); - } - - log.info("Dropped segment[%s].", identifier); - - return null; - } - }, - // use persistExecutor to make sure that all the pending persists completes before - // starting to abandon segments - persistExecutor - ); - } - - private Committed readCommit() throws IOException - { - final File commitFile = computeCommitFile(); - if (commitFile.exists()) { - // merge current hydrants with existing hydrants - return objectMapper.readValue(commitFile, Committed.class); - } else { - return null; - } - } - - private void writeCommit(Committed newCommit) throws IOException - { - final File commitFile = computeCommitFile(); - objectMapper.writeValue(commitFile, newCommit); - } - - private File computeCommitFile() - { - return new File(tuningConfig.getBasePersistDirectory(), "commit.json"); - } - - private File computeLockFile() - { - return new File(tuningConfig.getBasePersistDirectory(), ".lock"); - } - - private File computePersistDir(SegmentIdWithShardSpec identifier) - { - return new File(tuningConfig.getBasePersistDirectory(), identifier.toString()); - } - - private File computeIdentifierFile(SegmentIdWithShardSpec identifier) - { - return new File(computePersistDir(identifier), IDENTIFIER_FILE_NAME); - } - - private File computeDescriptorFile(SegmentIdWithShardSpec identifier) - { - return new File(computePersistDir(identifier), "descriptor.json"); - } - - private File createPersistDirIfNeeded(SegmentIdWithShardSpec identifier) throws IOException - { - final File persistDir = computePersistDir(identifier); - FileUtils.mkdirp(persistDir); - - objectMapper.writeValue(computeIdentifierFile(identifier), identifier); - - return persistDir; - } - - /** - * Persists the given hydrant and returns the number of rows persisted. Must only be called in the single-threaded - * persistExecutor. - * - * @param indexToPersist hydrant to persist - * @param identifier the segment this hydrant is going to be part of - * - * @return the number of rows persisted - */ - private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec identifier) - { - synchronized (indexToPersist) { - if (indexToPersist.hasSwapped()) { - log.info( - "Segment[%s] hydrant[%s] already swapped. Ignoring request to persist.", - identifier, - indexToPersist - ); - return 0; - } - - log.debug("Segment[%s], persisting Hydrant[%s]", identifier, indexToPersist); - - try { - final long startTime = System.nanoTime(); - int numRows = indexToPersist.getIndex().size(); - - final File persistedFile; - final File persistDir = createPersistDirIfNeeded(identifier); - persistedFile = indexMerger.persist( - indexToPersist.getIndex(), - identifier.getInterval(), - new File(persistDir, String.valueOf(indexToPersist.getCount())), - tuningConfig.getIndexSpecForIntermediatePersists(), - tuningConfig.getSegmentWriteOutMediumFactory() - ); - - log.info( - "Flushed in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).", - indexToPersist.getSegmentId(), - indexToPersist.getCount(), - (System.nanoTime() - startTime) / 1000000, - numRows - ); - - // Map only when this appenderator is being driven by a real time task: - Segment segmentToSwap = null; - if (isOpenSegments()) { - segmentToSwap = new QueryableIndexSegment(indexIO.loadIndex(persistedFile), indexToPersist.getSegmentId()); - } else { - // remember file path & segment id to rebuild the queryable index for merge: - persistedHydrantMetadata.put(indexToPersist, new Pair<>(persistedFile, indexToPersist.getSegmentId())); - } - indexToPersist.swapSegment(segmentToSwap); - - return numRows; - } - catch (IOException e) { - log.makeAlert("Incremental persist failed") - .addData("segment", identifier.toString()) - .addData("dataSource", schema.getDataSource()) - .addData("count", indexToPersist.getCount()) - .emit(); - - throw new RuntimeException(e); - } - } - } - - private void removeDirectory(final File target) - { - if (target.exists()) { - try { - FileUtils.deleteDirectory(target); - } - catch (Exception e) { - log.makeAlert(e, "Failed to remove directory[%s]", schema.getDataSource()) - .addData("file", target) - .emit(); - } - } - } - - private int calculateMMappedHydrantMemoryInUsed(FireHydrant hydrant) - { - if (skipBytesInMemoryOverheadCheck) { - return 0; - } - // These calculations are approximated from actual heap dumps. - // Memory footprint includes count integer in FireHydrant, shorts in ReferenceCountingSegment, - // Objects in SimpleQueryableIndex (such as SmooshedFileMapper, each ColumnHolder in column map, etc.) - int total; - total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_HYDRANT; - if (isOpenSegments()) { - // for real time add references to byte memory mapped references.. - total += (hydrant.getSegmentNumDimensionColumns() * ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER) + - (hydrant.getSegmentNumMetricColumns() * ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + - ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER; - } - return total; - } - - private int calculateSinkMemoryInUsed() - { - if (skipBytesInMemoryOverheadCheck) { - return 0; - } - // Rough estimate of memory footprint of empty Sink based on actual heap dumps - return ROUGH_OVERHEAD_PER_SINK; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 56af5f3f62c..28b4379f7b9 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -37,7 +37,6 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; import org.apache.druid.timeline.VersionedIntervalTimeline; public class Appenderators @@ -97,7 +96,7 @@ public class Appenderators ); } - public static Appenderator createOffline( + public static Appenderator createBatch( String id, DataSchema schema, AppenderatorConfig config, @@ -130,77 +129,4 @@ public class Appenderators centralizedDatasourceSchemaConfig ); } - - public static Appenderator createOpenSegmentsOffline( - String id, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - // fallback to original code known to be working, this is just a fallback option in case new - // batch appenderator has some early bugs but we will remove this fallback as soon as - // we determine that batch appenderator code is stable - return new AppenderatorImpl( - id, - schema, - config, - metrics, - dataSegmentPusher, - objectMapper, - new NoopDataSegmentAnnouncer(), - null, - indexIO, - indexMerger, - null, - rowIngestionMeters, - parseExceptionHandler, - true, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - } - - public static Appenderator createClosedSegmentsOffline( - String id, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - return new AppenderatorImpl( - id, - schema, - config, - metrics, - dataSegmentPusher, - objectMapper, - new NoopDataSegmentAnnouncer(), - null, - indexIO, - indexMerger, - null, - rowIngestionMeters, - parseExceptionHandler, - false, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java index 3bbf364656e..ec328c3b3cd 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java @@ -62,8 +62,9 @@ import org.joda.time.Interval; public interface AppenderatorsManager { /** - * Creates an Appenderator suited for realtime ingestion. Note that this method's parameters include objects - * used for query processing. + * Creates an {@link StreamAppenderator} suited for realtime ingestion. Note that this method's parameters include + * objects used for query processing. Intermediary segments are persisted to disk and memory mapped to be available + * for query processing. */ Appenderator createRealtimeAppenderatorForTask( SegmentLoaderConfig segmentLoaderConfig, @@ -90,39 +91,11 @@ public interface AppenderatorsManager ); /** - * Creates an Appenderator suited for batch ingestion. + * Creates a {@link BatchAppenderator} suitable for batch ingestion with no ability to process queries against + * the processed data. Intermediary segments are persisted to temporary disk and then merged into the final set of + * segments at publishing time. */ - Appenderator createOpenSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ); - - Appenderator createClosedSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ); - - Appenderator createOfflineAppenderatorForTask( + Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index b544d33705c..979d8cb92b3 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -236,7 +236,7 @@ public class BatchAppenderator implements Appenderator } if (pushExecutor == null) { - // use a blocking single threaded executor to throttle the firehose when write to disk is slow + // use a blocking single threaded executor to throttle the input source when write to disk is slow pushExecutor = MoreExecutors.listeningDecorator( Execs.newBlockingSingleThreaded( "[" + StringUtils.encodeForFormat(myId) + "]-batch-appenderator-push", diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java index f44fffe20e1..d613f3ff59c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java @@ -84,45 +84,7 @@ public class DummyForInjectionAppenderatorsManager implements AppenderatorsManag } @Override - public Appenderator createOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - throw new UOE(ERROR_MSG); - } - - @Override - public Appenderator createOpenSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - throw new UOE(ERROR_MSG); - } - - @Override - public Appenderator createClosedSegmentsOfflineAppenderatorForTask( + public Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java index 52f75f72e47..998f674daf7 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java @@ -118,7 +118,7 @@ public class PeonAppenderatorsManager implements AppenderatorsManager } @Override - public Appenderator createOfflineAppenderatorForTask( + public Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, @@ -137,7 +137,7 @@ public class PeonAppenderatorsManager implements AppenderatorsManager if (realtimeAppenderator != null) { throw new ISE("A realtime appenderator was already created for this peon's task."); } else { - batchAppenderator = Appenderators.createOffline( + batchAppenderator = Appenderators.createBatch( taskId, schema, config, @@ -155,81 +155,6 @@ public class PeonAppenderatorsManager implements AppenderatorsManager } } - @Override - public Appenderator createOpenSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators - if (realtimeAppenderator != null) { - throw new ISE("A realtime appenderator was already created for this peon's task."); - } else { - batchAppenderator = Appenderators.createOpenSegmentsOffline( - taskId, - schema, - config, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - return batchAppenderator; - } - } - - @Override - public Appenderator createClosedSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators - if (realtimeAppenderator != null) { - throw new ISE("A realtime appenderator was already created for this peon's task."); - } else { - batchAppenderator = Appenderators.createClosedSegmentsOffline( - taskId, - schema, - config, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - return batchAppenderator; - } - } @Override public void removeAppenderatorsForTask(String taskId, String dataSource) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index 4d1253591e0..a25d6b7acd7 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -1234,7 +1234,7 @@ public class StreamAppenderator implements Appenderator } if (pushExecutor == null) { - // use a blocking single threaded executor to throttle the firehose when write to disk is slow + // use a blocking single threaded executor to throttle the input source when write to disk is slow pushExecutor = MoreExecutors.listeningDecorator( Execs.newBlockingSingleThreaded("[" + StringUtils.encodeForFormat(myId) + "]-appenderator-merge", 1) ); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index ffdfb8d1eb0..0088e33ca7a 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -205,7 +205,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager } @Override - public Appenderator createOfflineAppenderatorForTask( + public Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, @@ -226,89 +226,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager DatasourceBundle::new ); - Appenderator appenderator = Appenderators.createOffline( - taskId, - schema, - rewriteAppenderatorConfigMemoryLimits(config), - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - wrapIndexMerger(indexMerger), - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - datasourceBundle.addAppenderator(taskId, appenderator); - return appenderator; - } - } - - @Override - public Appenderator createOpenSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - synchronized (this) { - DatasourceBundle datasourceBundle = datasourceBundles.computeIfAbsent( - schema.getDataSource(), - DatasourceBundle::new - ); - - Appenderator appenderator = Appenderators.createOpenSegmentsOffline( - taskId, - schema, - rewriteAppenderatorConfigMemoryLimits(config), - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - wrapIndexMerger(indexMerger), - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - datasourceBundle.addAppenderator(taskId, appenderator); - return appenderator; - } - } - - @Override - public Appenderator createClosedSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - synchronized (this) { - DatasourceBundle datasourceBundle = datasourceBundles.computeIfAbsent( - schema.getDataSource(), - DatasourceBundle::new - ); - - Appenderator appenderator = Appenderators.createClosedSegmentsOffline( + Appenderator appenderator = Appenderators.createBatch( taskId, schema, rewriteAppenderatorConfigMemoryLimits(config), @@ -595,7 +513,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager /** * This wrapper around IndexMerger limits concurrent calls to the merge/persist methods used by * {@link StreamAppenderator} with a shared executor service. Merge/persist methods that are not used by - * AppenderatorImpl will throw an exception if called. + * StreamAppenderator will throw an exception if called. */ public static class LimitedPoolIndexMerger implements IndexMerger { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ClippedFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/ClippedFirehoseFactory.java deleted file mode 100644 index 9ca595ce5c2..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ClippedFirehoseFactory.java +++ /dev/null @@ -1,81 +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.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Predicate; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.InputRowParser; -import org.joda.time.Interval; - -import java.io.File; -import java.io.IOException; - -/** - * Creates firehoses clipped to a particular time interval. Useful for enforcing min time, max time, and time windows. - */ -@Deprecated -public class ClippedFirehoseFactory implements FirehoseFactory -{ - private final FirehoseFactory delegate; - private final Interval interval; - - @JsonCreator - public ClippedFirehoseFactory( - @JsonProperty("delegate") FirehoseFactory delegate, - @JsonProperty("interval") Interval interval - ) - { - this.delegate = delegate; - this.interval = interval; - } - - @JsonProperty - public FirehoseFactory getDelegate() - { - return delegate; - } - - @JsonProperty - public Interval getInterval() - { - return interval; - } - - @Override - public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException - { - return new PredicateFirehose( - delegate.connect(parser, temporaryDirectory), - new Predicate() - { - @Override - public boolean apply(InputRow input) - { - return interval.contains(input.getTimestampFromEpoch()); - } - } - ); - } - -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java deleted file mode 100644 index 4fad62a6d46..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java +++ /dev/null @@ -1,659 +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.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; -import com.google.common.io.CountingInputStream; -import com.google.common.util.concurrent.Uninterruptibles; -import com.google.errorprone.annotations.concurrent.GuardedBy; -import org.apache.druid.concurrent.Threads; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.guice.annotations.Json; -import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.metrics.EventReceiverFirehoseMetric; -import org.apache.druid.server.metrics.EventReceiverFirehoseRegister; -import org.apache.druid.server.security.Access; -import org.apache.druid.server.security.Action; -import org.apache.druid.server.security.AuthorizationUtils; -import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.server.security.Resource; -import org.apache.druid.server.security.ResourceAction; -import org.joda.time.DateTime; - -import javax.annotation.Nullable; -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.Consumes; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.QueryParam; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; - -/** - * Builds firehoses that accept events through the {@link EventReceiver} interface. Can also register these - * firehoses with an {@link ServiceAnnouncingChatHandlerProvider}. - */ -@Deprecated -public class EventReceiverFirehoseFactory implements FirehoseFactory>> -{ - private static final EmittingLogger log = new EmittingLogger(EventReceiverFirehoseFactory.class); - - public static final int MAX_FIREHOSE_PRODUCERS = 10_000; - - private static final int DEFAULT_BUFFER_SIZE = 100_000; - - /** - * A "poison pill" object for {@link EventReceiverFirehose}'s internal buffer. - */ - private static final Object FIREHOSE_CLOSED = new Object(); - - private final String serviceName; - private final int bufferSize; - - /** - * Doesn't really support max idle times finer than 1 second due to how {@link - * EventReceiverFirehose#delayedCloseExecutor} is implemented, see a comment inside {@link - * EventReceiverFirehose#createDelayedCloseExecutor()}. This aspect is not reflected in docs because it's unlikely - * that anybody configures or cares about finer max idle times, and also because this is an implementation detail of - * {@link EventReceiverFirehose} that may change in the future. - */ - private final long maxIdleTimeMillis; - private final ChatHandlerProvider chatHandlerProvider; - private final ObjectMapper jsonMapper; - private final ObjectMapper smileMapper; - private final EventReceiverFirehoseRegister eventReceiverFirehoseRegister; - private final AuthorizerMapper authorizerMapper; - - @JsonCreator - public EventReceiverFirehoseFactory( - @JsonProperty("serviceName") String serviceName, - @JsonProperty("bufferSize") Integer bufferSize, - // Keeping the legacy 'maxIdleTime' property name for backward compatibility. When the project is updated to - // Jackson 2.9 it could be changed, see https://github.com/apache/druid/issues/7152 - @JsonProperty("maxIdleTime") @Nullable Long maxIdleTimeMillis, - @JacksonInject ChatHandlerProvider chatHandlerProvider, - @JacksonInject @Json ObjectMapper jsonMapper, - @JacksonInject @Smile ObjectMapper smileMapper, - @JacksonInject EventReceiverFirehoseRegister eventReceiverFirehoseRegister, - @JacksonInject AuthorizerMapper authorizerMapper - ) - { - Preconditions.checkNotNull(serviceName, "serviceName"); - - this.serviceName = serviceName; - this.bufferSize = bufferSize == null || bufferSize <= 0 ? DEFAULT_BUFFER_SIZE : bufferSize; - this.maxIdleTimeMillis = (maxIdleTimeMillis == null || maxIdleTimeMillis <= 0) ? Long.MAX_VALUE : maxIdleTimeMillis; - this.chatHandlerProvider = chatHandlerProvider; - this.jsonMapper = jsonMapper; - this.smileMapper = smileMapper; - this.eventReceiverFirehoseRegister = eventReceiverFirehoseRegister; - this.authorizerMapper = authorizerMapper; - } - - @Override - public Firehose connect( - InputRowParser> firehoseParser, - File temporaryDirectory - ) - { - log.info("Connecting firehose: %s", serviceName); - final EventReceiverFirehose firehose = new EventReceiverFirehose(firehoseParser); - - if (chatHandlerProvider != null) { - log.info("Found chathandler of class[%s]", chatHandlerProvider.getClass().getName()); - chatHandlerProvider.register(serviceName, firehose); - int lastIndexOfColon = serviceName.lastIndexOf(':'); - if (lastIndexOfColon > 0) { - chatHandlerProvider.register(serviceName.substring(lastIndexOfColon + 1), firehose); - } - } else { - log.warn("No chathandler detected"); - } - - eventReceiverFirehoseRegister.register(serviceName, firehose); - - return firehose; - } - - @JsonProperty - public String getServiceName() - { - return serviceName; - } - - @JsonProperty - public int getBufferSize() - { - return bufferSize; - } - - /** - * Keeping the legacy 'maxIdleTime' property name for backward compatibility. When the project is updated to Jackson - * 2.9 it could be changed, see https://github.com/apache/druid/issues/7152 - */ - @JsonProperty("maxIdleTime") - public long getMaxIdleTimeMillis() - { - return maxIdleTimeMillis; - } - - /** - * Apart from adhering to {@link Firehose} contract regarding concurrency, this class has two methods that might be - * called concurrently with any other methods and each other, from arbitrary number of threads: {@link #addAll} and - * {@link #shutdown}. - * - * Concurrent data flow: in {@link #addAll} (can be called concurrently with any other methods and other calls to - * {@link #addAll}) rows are pushed into {@link #buffer}. The single Firehose "consumer" thread calls {@link #hasMore} - * and {@link #nextRow()}, where rows are taken out from the other end of the {@link #buffer} queue. - * - * This class creates and manages one thread ({@link #delayedCloseExecutor}) for calling {@link #close()} - * asynchronously in response to a {@link #shutdown} request, or after this Firehose has been idle (no calls to {@link - * #addAll}) for {@link #maxIdleTimeMillis}. - */ - @VisibleForTesting - public class EventReceiverFirehose implements ChatHandler, Firehose, EventReceiverFirehoseMetric - { - /** - * How does this thread work (and its interruption policy) is described in the comment for {@link - * #createDelayedCloseExecutor}. - */ - @GuardedBy("this") - private @Nullable Thread delayedCloseExecutor; - - /** - * Contains {@link InputRow} objects, the last one is {@link #FIREHOSE_CLOSED} which is a "poison pill". Poison pill - * is used to notify the thread that calls {@link #hasMore()} and {@link #nextRow()} that the EventReceiverFirehose - * is closed without heuristic 500 ms timed blocking in a loop instead of a simple {@link BlockingQueue#take()} - * call (see {@link #hasMore} code). - */ - private final BlockingQueue buffer; - private final InputRowParser> parser; - - /** - * This field needs to be volatile to ensure progress in {@link #addRows} method where it is read in a loop, and - * also in testing code calling {@link #isClosed()}. - */ - private volatile boolean closed = false; - - /** - * This field and {@link #rowsRunOut} are not volatile because they are accessed only from {@link #hasMore()} and - * {@link #nextRow()} methods that are called from a single thread according to {@link Firehose} spec. - */ - @Nullable - private InputRow nextRow = null; - private boolean rowsRunOut = false; - - private final AtomicLong bytesReceived = new AtomicLong(0); - private final AtomicLong lastBufferAddFailLoggingTimeNs = new AtomicLong(System.nanoTime()); - private final ConcurrentHashMap producerSequences = new ConcurrentHashMap<>(); - - /** - * This field and {@link #requestedShutdownTimeNs} use nanoseconds instead of milliseconds not to deal with the fact - * that {@link System#currentTimeMillis()} can "go backward", e. g. due to time correction on the server. - * - * This field and {@link #requestedShutdownTimeNs} must be volatile because they are de facto lazily initialized - * fields that are used concurrently in {@link #delayedCloseExecutor} (see {@link #createDelayedCloseExecutor()}). - * If they were not volatile, NPE would be possible in {@link #delayedCloseExecutor}. See - * https://shipilev.net/blog/2016/close-encounters-of-jmm-kind/#wishful-hb-actual for explanations. - */ - @Nullable - private volatile Long idleCloseTimeNs = null; - @Nullable - private volatile Long requestedShutdownTimeNs = null; - - EventReceiverFirehose(InputRowParser> parser) - { - this.buffer = new ArrayBlockingQueue<>(bufferSize); - this.parser = parser; - - if (maxIdleTimeMillis != Long.MAX_VALUE) { - idleCloseTimeNs = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(maxIdleTimeMillis); - synchronized (this) { - createDelayedCloseExecutor(); - } - } - } - - @VisibleForTesting - synchronized @Nullable Thread getDelayedCloseExecutor() - { - return delayedCloseExecutor; - } - - /** - * Creates and starts a {@link #delayedCloseExecutor} thread, either right from the EventReceiverFirehose's - * constructor if {@link #maxIdleTimeMillis} is specified, or otherwise lazily from {@link #shutdown}. - * - * The thread waits until the time when the Firehose should be closed because either {@link #addAll} was not called - * for the specified max idle time (see {@link #idleCloseTimeNs}), or until the shutoff time requested last - * via {@link #shutdown} (see {@link #requestedShutdownTimeNs}), whatever is sooner. Then the thread does - * two things: - * 1. if the Firehose is already closed (or in the process of closing, but {@link #closed} flag is already set), it - * silently exits. - * 2. It checks both deadlines again: - * a) if either of them has arrived, it calls {@link #close()} and exits. - * b) otherwise, it waits until the nearest deadline again, and so on in a loop. - * - * This way the thread works predictably and robustly regardless of how both deadlines change (for example, shutoff - * time specified via {@link #shutdown} may jump in both directions). - * - * Other methods notify {@link #delayedCloseExecutor} that the Firehose state in some way that is important for this - * thread (that is, when {@link #close()} is called, {@link #delayedCloseExecutor} is no longer needed and should - * exit as soon as possible to release system resources; when {@link #shutdown} is called, the thread may need to - * wake up sooner if the shutoff time has been moved sooner) by simply interrupting it. The thread wakes up and - * continues its loop. - */ - @GuardedBy("this") - private Thread createDelayedCloseExecutor() - { - Thread delayedCloseExecutor = new Thread( - () -> { - // The closed = true is visible after close() because there is a happens-before edge between - // delayedCloseExecutor.interrupt() call in close() and catching InterruptedException below in this loop. - while (!closed) { - if (idleCloseTimeNs == null && requestedShutdownTimeNs == null) { - // This is not possible unless there are bugs in the code of EventReceiverFirehose. AssertionError could - // have been thrown instead, but it doesn't seem to make a lot of sense in a background thread. Instead, - // we long the error and continue a loop after some pause. - log.error( - "Either idleCloseTimeNs or requestedShutdownTimeNs must be non-null. " - + "Please file a bug at https://github.com/apache/druid/issues" - ); - } - if (idleCloseTimeNs != null && idleCloseTimeNs - System.nanoTime() <= 0) { // overflow-aware comparison - log.info("Firehose has been idle for %d ms, closing.", maxIdleTimeMillis); - close(); - } else if (requestedShutdownTimeNs != null && - requestedShutdownTimeNs - System.nanoTime() <= 0) { // overflow-aware comparison - log.info("Closing Firehose after a shutdown request"); - close(); - } - try { - // It is possible to write code that sleeps until the next the next idleCloseTimeNs or - // requestedShutdownTimeNs, whatever is non-null and sooner, but that's fairly complicated code. That - // complexity perhaps overweighs the minor inefficiency of simply waking up every second. - Threads.sleepFor(1, TimeUnit.SECONDS); - } - catch (InterruptedException ignore) { - // Interruption is a wakeup, continue the loop - } - } - }, - "event-receiver-firehose-closer" - ); - delayedCloseExecutor.setDaemon(true); - this.delayedCloseExecutor = delayedCloseExecutor; - delayedCloseExecutor.start(); - return delayedCloseExecutor; - } - - /** - * This method might be called concurrently from multiple threads, if multiple requests arrive to the server at the - * same time (possibly exact duplicates). Concurrency is controlled in {@link #checkProducerSequence}, where only - * requests with "X-Firehose-Producer-Seq" number greater than the max "X-Firehose-Producer-Seq" in previously - * arrived requests are allowed to proceed. After that check requests don't synchronize with each other and - * therefore if two large batches are sent with little interval, the events from the batches might be mixed up in - * {@link #buffer} (if two {@link #addRows(Iterable)} are executed concurrently). - */ - @POST - @Path("/push-events") - @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - public Response addAll(InputStream in, @Context final HttpServletRequest req) throws JsonProcessingException - { - idleCloseTimeNs = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(maxIdleTimeMillis); - Access accessResult = AuthorizationUtils.authorizeResourceAction( - req, - new ResourceAction( - Resource.STATE_RESOURCE, - Action.WRITE - ), - authorizerMapper - ); - if (!accessResult.isAllowed()) { - return Response.status(403).build(); - } - - final String reqContentType = req.getContentType(); - final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(reqContentType); - final String contentType = isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON; - - ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; - - Response producerSequenceResponse = checkProducerSequence(req, reqContentType, objectMapper); - if (producerSequenceResponse != null) { - return producerSequenceResponse; - } - - CountingInputStream countingInputStream = new CountingInputStream(in); - Collection> events; - try { - events = objectMapper.readValue( - countingInputStream, - new TypeReference>>() - { - } - ); - } - catch (IOException e) { - return Response.serverError().entity(ImmutableMap.of("error", e.getMessage())).build(); - } - finally { - bytesReceived.addAndGet(countingInputStream.getCount()); - } - log.debug("Adding %,d events to firehose: %s", events.size(), serviceName); - - final List rows = new ArrayList<>(); - for (final Map event : events) { - // Might throw an exception. We'd like that to happen now, instead of while adding to the row buffer. - rows.addAll(parser.parseBatch(event)); - } - - try { - addRows(rows); - return Response.ok( - objectMapper.writeValueAsString(ImmutableMap.of("eventCount", events.size())), - contentType - ).build(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - - @Override - public boolean hasMore() - { - if (rowsRunOut) { - return false; - } - if (nextRow != null) { - return true; - } - Object next; - try { - next = buffer.take(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - //noinspection ObjectEquality - if (next == FIREHOSE_CLOSED) { - rowsRunOut = true; - return false; - } - nextRow = (InputRow) next; - return true; - } - - @Nullable - @Override - public InputRow nextRow() - { - final InputRow row = nextRow; - - if (row == null) { - throw new NoSuchElementException(); - } else { - nextRow = null; - return row; - } - } - - @Override - public int getCurrentBufferSize() - { - return buffer.size(); - } - - @Override - public int getCapacity() - { - return bufferSize; - } - - @Override - public long getBytesReceived() - { - return bytesReceived.get(); - } - - /** - * This method is synchronized because it might be called concurrently from multiple threads: from {@link - * #delayedCloseExecutor}, and from the thread that creates and uses the Firehose object. - */ - @Override - public synchronized void close() - { - if (closed) { - return; - } - closed = true; - log.info("Firehose closing."); - - // Critical to add the poison pill to the queue, don't allow interruption. - Uninterruptibles.putUninterruptibly(buffer, FIREHOSE_CLOSED); - - eventReceiverFirehoseRegister.unregister(serviceName); - if (chatHandlerProvider != null) { - chatHandlerProvider.unregister(serviceName); - } - if (delayedCloseExecutor != null && !delayedCloseExecutor.equals(Thread.currentThread())) { - // Interrupt delayedCloseExecutor to let it discover that closed flag is already set and exit. - delayedCloseExecutor.interrupt(); - } - } - - @VisibleForTesting - void addRows(Iterable rows) throws InterruptedException - { - for (final InputRow row : rows) { - boolean added = false; - while (!closed && !added) { - added = buffer.offer(row, 500, TimeUnit.MILLISECONDS); - if (!added) { - long currTimeNs = System.nanoTime(); - long lastTimeNs = lastBufferAddFailLoggingTimeNs.get(); - if (currTimeNs - lastTimeNs > TimeUnit.SECONDS.toNanos(10) && - lastBufferAddFailLoggingTimeNs.compareAndSet(lastTimeNs, currTimeNs)) { - log.warn("Failed to add event to buffer with current size [%s] . Retrying...", buffer.size()); - } - } - } - - if (!added) { - throw new IllegalStateException("Cannot add events to closed firehose!"); - } - } - } - - /** - * This method might be called concurrently from multiple threads, if multiple shutdown requests arrive at the same - * time. No attempts are made to synchronize such requests, or prioritize them a-la "latest shutdown time wins" or - * "soonest shutdown time wins". {@link #delayedCloseExecutor}'s logic (see {@link #createDelayedCloseExecutor()}) - * is indifferent to shutdown times jumping in arbitrary directions. But once a shutdown request is made, it can't - * be cancelled entirely, the shutdown time could only be rescheduled with a new request. - */ - @POST - @Path("/shutdown") - @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - public Response shutdown( - @QueryParam("shutoffTime") final String shutoffTimeMillis, - @Context final HttpServletRequest req - ) - { - Access accessResult = AuthorizationUtils.authorizeResourceAction( - req, - new ResourceAction( - Resource.STATE_RESOURCE, - Action.WRITE - ), - authorizerMapper - ); - if (!accessResult.isAllowed()) { - return Response.status(403).build(); - } - - try { - DateTime shutoffAt = shutoffTimeMillis == null ? DateTimes.nowUtc() : DateTimes.of(shutoffTimeMillis); - log.info("Setting Firehose shutoffTime to %s", shutoffTimeMillis); - long shutoffTimeoutMillis = Math.max(shutoffAt.getMillis() - System.currentTimeMillis(), 0); - - requestedShutdownTimeNs = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(shutoffTimeoutMillis); - Thread delayedCloseExecutor; - // Need to interrupt delayedCloseExecutor because a newly specified shutdown time might be closer than idle - // timeout or previously specified shutdown. Interruption of delayedCloseExecutor lets it adjust the sleep time - // (see the logic of this thread in createDelayedCloseExecutor()). - boolean needToInterruptDelayedCloseExecutor = true; - synchronized (this) { - delayedCloseExecutor = this.delayedCloseExecutor; - if (delayedCloseExecutor == null) { - delayedCloseExecutor = createDelayedCloseExecutor(); - // Don't need to interrupt a freshly created thread - needToInterruptDelayedCloseExecutor = false; - } - } - if (needToInterruptDelayedCloseExecutor) { - delayedCloseExecutor.interrupt(); - } - return Response.ok().build(); - } - catch (IllegalArgumentException e) { - return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", e.getMessage())) - .build(); - - } - } - - @VisibleForTesting - boolean isClosed() - { - return closed; - } - - /** - * Checks the request for a producer ID and sequence value. If the producer ID is specified, a corresponding - * sequence value must be specified as well. If the incoming sequence is less than or equal to the last seen - * sequence for that producer ID, the request is ignored. - * - * This method might be called concurrently from multiple threads. - * - * @param req Http request - * @param responseContentType Response content type - * @param responseMapper Response object mapper - * @return an error response to return or null if the request can proceed - */ - @Nullable - private Response checkProducerSequence( - final HttpServletRequest req, - final String responseContentType, - final ObjectMapper responseMapper - ) - { - final String producerId = req.getHeader("X-Firehose-Producer-Id"); - - if (producerId == null) { - return null; - } - - final String sequenceValue = req.getHeader("X-Firehose-Producer-Seq"); - - if (sequenceValue == null) { - return Response - .status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", "Producer sequence value is missing")) - .build(); - } - - Long producerSequence = producerSequences.computeIfAbsent(producerId, key -> Long.MIN_VALUE); - - if (producerSequences.size() >= MAX_FIREHOSE_PRODUCERS) { - return Response - .status(Response.Status.FORBIDDEN) - .entity( - ImmutableMap.of( - "error", - "Too many individual producer IDs for this firehose. Max is " + MAX_FIREHOSE_PRODUCERS - ) - ) - .build(); - } - - try { - Long newSequence = Long.parseLong(sequenceValue); - - while (true) { - if (newSequence <= producerSequence) { - return Response.ok( - responseMapper.writeValueAsString(ImmutableMap.of("eventCount", 0, "skipped", true)), - responseContentType - ).build(); - } - if (producerSequences.replace(producerId, producerSequence, newSequence)) { - return null; - } - producerSequence = producerSequences.get(producerId); - } - } - catch (JsonProcessingException ex) { - throw new RuntimeException(ex); - } - catch (NumberFormatException ex) { - return Response - .status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", "Producer sequence must be a number")) - .build(); - } - } - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java deleted file mode 100644 index 72a48b33a45..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java +++ /dev/null @@ -1,93 +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.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.InputRowParser; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; - -/** - * Firehose to give out only first n events from the delegate firehose. - */ -public class FixedCountFirehoseFactory implements FirehoseFactory -{ - private final FirehoseFactory delegate; - private final int count; - - @JsonCreator - public FixedCountFirehoseFactory( - @JsonProperty("delegate") FirehoseFactory delegate, - @JsonProperty("count") int count - ) - { - this.delegate = delegate; - this.count = count; - } - - @JsonProperty - public FirehoseFactory getDelegate() - { - return delegate; - } - - @JsonProperty - public int getCount() - { - return count; - } - - @Override - public Firehose connect(final InputRowParser parser, File temporaryDirectory) throws IOException - { - return new Firehose() - { - private int i = 0; - private final Firehose delegateFirehose = delegate.connect(parser, temporaryDirectory); - - @Override - public boolean hasMore() throws IOException - { - return i < count && delegateFirehose.hasMore(); - } - - @Nullable - @Override - public InputRow nextRow() throws IOException - { - Preconditions.checkArgument(i++ < count, "Max events limit reached."); - return delegateFirehose.nextRow(); - } - - @Override - public void close() throws IOException - { - delegateFirehose.close(); - } - }; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java deleted file mode 100644 index c0064a25f64..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java +++ /dev/null @@ -1,210 +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.segment.realtime.firehose; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.guava.Yielders; -import org.apache.druid.query.dimension.DefaultDimensionSpec; -import org.apache.druid.query.filter.DimFilter; -import org.apache.druid.segment.BaseLongColumnValueSelector; -import org.apache.druid.segment.BaseObjectColumnValueSelector; -import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.data.IndexedInts; -import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.segment.transform.Transformer; - -import javax.annotation.Nullable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -public class IngestSegmentFirehose implements Firehose -{ - private final Transformer transformer; - private Yielder rowYielder; - - public IngestSegmentFirehose( - final List adapters, - final TransformSpec transformSpec, - final List dims, - final List metrics, - final DimFilter dimFilter - ) - { - this.transformer = transformSpec.toTransformer(); - - Sequence rows = Sequences.concat( - Iterables.transform( - adapters, - new Function>() - { - @Nullable - @Override - public Sequence apply(WindowedStorageAdapter adapter) - { - return Sequences.concat( - Sequences.map( - adapter.getAdapter().makeCursors( - Filters.toFilter(dimFilter), - adapter.getInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), new Function>() - { - @Nullable - @Override - public Sequence apply(final Cursor cursor) - { - final BaseLongColumnValueSelector timestampColumnSelector = - cursor.getColumnSelectorFactory().makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME); - - final Map dimSelectors = new HashMap<>(); - for (String dim : dims) { - final DimensionSelector dimSelector = cursor - .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec(dim, dim)); - // dimSelector is null if the dimension is not present - if (dimSelector != null) { - dimSelectors.put(dim, dimSelector); - } - } - - final Map metSelectors = new HashMap<>(); - for (String metric : metrics) { - final BaseObjectColumnValueSelector metricSelector = - cursor.getColumnSelectorFactory().makeColumnValueSelector(metric); - metSelectors.put(metric, metricSelector); - } - - return Sequences.simple( - new Iterable() - { - @Override - public Iterator iterator() - { - return new Iterator() - { - @Override - public boolean hasNext() - { - return !cursor.isDone(); - } - - @Override - public InputRow next() - { - final Map theEvent = Maps.newLinkedHashMap(); - final long timestamp = timestampColumnSelector.getLong(); - theEvent.put(TimestampSpec.DEFAULT_COLUMN, DateTimes.utc(timestamp)); - - for (Map.Entry dimSelector : - dimSelectors.entrySet()) { - final String dim = dimSelector.getKey(); - final DimensionSelector selector = dimSelector.getValue(); - final IndexedInts vals = selector.getRow(); - - int valsSize = vals.size(); - if (valsSize == 1) { - final String dimVal = selector.lookupName(vals.get(0)); - theEvent.put(dim, dimVal); - } else if (valsSize > 1) { - List dimVals = new ArrayList<>(valsSize); - for (int i = 0; i < valsSize; ++i) { - dimVals.add(selector.lookupName(vals.get(i))); - } - theEvent.put(dim, dimVals); - } - } - - for (Map.Entry metSelector : - metSelectors.entrySet()) { - final String metric = metSelector.getKey(); - final BaseObjectColumnValueSelector selector = metSelector.getValue(); - Object value = selector.getObject(); - if (value != null) { - theEvent.put(metric, value); - } - } - cursor.advance(); - return new MapBasedInputRow(timestamp, dims, theEvent); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException("Remove Not Supported"); - } - }; - } - } - ); - } - } - ) - ); - } - } - ) - ); - rowYielder = Yielders.each(rows); - } - - @Override - public boolean hasMore() - { - return !rowYielder.isDone(); - } - - @Nullable - @Override - public InputRow nextRow() - { - final InputRow inputRow = rowYielder.get(); - rowYielder = rowYielder.next(null); - return transformer.transform(inputRow); - } - - @Override - public void close() throws IOException - { - rowYielder.close(); - } - -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/PredicateFirehose.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/PredicateFirehose.java deleted file mode 100644 index d6aadf07adc..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/PredicateFirehose.java +++ /dev/null @@ -1,89 +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.segment.realtime.firehose; - -import com.google.common.base.Predicate; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.logger.Logger; - -import javax.annotation.Nullable; -import java.io.IOException; - -/** - * Provides a view on a firehose that only returns rows that match a certain predicate. - * Not thread-safe. - */ -public class PredicateFirehose implements Firehose -{ - private static final Logger log = new Logger(PredicateFirehose.class); - private static final int IGNORE_THRESHOLD = 5000; - private long ignored = 0; - - private final Firehose firehose; - private final Predicate predicate; - - @Nullable - private InputRow savedInputRow = null; - - public PredicateFirehose(Firehose firehose, Predicate predicate) - { - this.firehose = firehose; - this.predicate = predicate; - } - - @Override - public boolean hasMore() throws IOException - { - if (savedInputRow != null) { - return true; - } - - while (firehose.hasMore()) { - final InputRow row = firehose.nextRow(); - if (predicate.apply(row)) { - savedInputRow = row; - return true; - } - // Do not silently discard the rows - if (ignored % IGNORE_THRESHOLD == 0) { - log.warn("[%,d] InputRow(s) ignored as they do not satisfy the predicate", ignored); - } - ignored++; - } - - return false; - } - - @Nullable - @Override - public InputRow nextRow() - { - final InputRow row = savedInputRow; - savedInputRow = null; - return row; - } - - @Override - public void close() throws IOException - { - firehose.close(); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java deleted file mode 100644 index 9bfda42c8aa..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java +++ /dev/null @@ -1,139 +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.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.errorprone.annotations.concurrent.GuardedBy; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.utils.CloseableUtils; -import org.joda.time.DateTime; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -/** - * Creates firehoses that shut off at a particular time. Useful for limiting the lifespan of a realtime job. - * - * Each {@link Firehose} created by this factory spins up and manages one thread for calling {@link Firehose#close()} - * asynchronously at the specified {@link #shutoffTime}. - */ -@Deprecated -public class TimedShutoffFirehoseFactory implements FirehoseFactory -{ - private static final EmittingLogger log = new EmittingLogger(FirehoseFactory.class); - - private final FirehoseFactory delegateFactory; - private final DateTime shutoffTime; - - @JsonCreator - public TimedShutoffFirehoseFactory( - @JsonProperty("delegate") FirehoseFactory delegateFactory, - @JsonProperty("shutoffTime") DateTime shutoffTime - ) - { - this.delegateFactory = delegateFactory; - this.shutoffTime = shutoffTime; - } - - @Override - public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException - { - return new TimedShutoffFirehose(parser, temporaryDirectory); - } - - class TimedShutoffFirehose implements Firehose - { - private final Firehose firehose; - private final ScheduledExecutorService shutdownExec; - @GuardedBy("this") - private boolean closed = false; - - TimedShutoffFirehose(InputRowParser parser, File temporaryDirectory) throws IOException - { - firehose = delegateFactory.connect(parser, temporaryDirectory); - - shutdownExec = Execs.scheduledSingleThreaded("timed-shutoff-firehose-%d"); - - shutdownExec.schedule( - () -> { - log.info("Closing delegate firehose."); - - try { - TimedShutoffFirehose.this.close(); - } - catch (IOException e) { - log.warn(e, "Failed to close delegate firehose, ignoring."); - } - }, - shutoffTime.getMillis() - System.currentTimeMillis(), - TimeUnit.MILLISECONDS - ); - - log.info("Firehose created, will shut down at: %s", shutoffTime); - } - - @Override - public boolean hasMore() throws IOException - { - return firehose.hasMore(); - } - - @Nullable - @Override - public InputRow nextRow() throws IOException - { - return firehose.nextRow(); - } - - /** - * This method is synchronized because it might be called concurrently from multiple threads: from {@link - * #shutdownExec}, and explicitly on this Firehose object. - */ - @Override - public synchronized void close() throws IOException - { - if (!closed) { - closed = true; - CloseableUtils.closeAll(firehose, shutdownExec::shutdownNow); - } - } - } - - @JsonProperty("delegate") - public FirehoseFactory getDelegateFactory() - { - return delegateFactory; - } - - @JsonProperty("shutoffTime") - public DateTime getShutoffTime() - { - return shutoffTime; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/package-info.java b/server/src/main/java/org/apache/druid/segment/realtime/package-info.java similarity index 94% rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/package-info.java rename to server/src/main/java/org/apache/druid/segment/realtime/package-info.java index e4a06999f36..2f80f8a1a06 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/package-info.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/package-info.java @@ -18,6 +18,6 @@ */ @EverythingIsNonnullByDefault -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; import org.apache.druid.annotations.EverythingIsNonnullByDefault; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java new file mode 100644 index 00000000000..6009dc12cf4 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexer.CompactionEngine; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Cluster-level compaction configs. + * All fields of this class are nullable. A non-null value denotes that the + * corresponding field has been explicitly specified. + */ +public class ClusterCompactionConfig +{ + private final Double compactionTaskSlotRatio; + private final Integer maxCompactionTaskSlots; + private final Boolean useAutoScaleSlots; + private final CompactionEngine engine; + + @JsonCreator + public ClusterCompactionConfig( + @JsonProperty("compactionTaskSlotRatio") @Nullable Double compactionTaskSlotRatio, + @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, + @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots, + @JsonProperty("engine") @Nullable CompactionEngine engine + ) + { + this.compactionTaskSlotRatio = compactionTaskSlotRatio; + this.maxCompactionTaskSlots = maxCompactionTaskSlots; + this.useAutoScaleSlots = useAutoScaleSlots; + this.engine = engine; + } + + @Nullable + @JsonProperty + public Double getCompactionTaskSlotRatio() + { + return compactionTaskSlotRatio; + } + + @Nullable + @JsonProperty + public Integer getMaxCompactionTaskSlots() + { + return maxCompactionTaskSlots; + } + + @Nullable + @JsonProperty + public Boolean getUseAutoScaleSlots() + { + return useAutoScaleSlots; + } + + @Nullable + @JsonProperty + public CompactionEngine getEngine() + { + return engine; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ClusterCompactionConfig that = (ClusterCompactionConfig) o; + return Objects.equals(compactionTaskSlotRatio, that.compactionTaskSlotRatio) + && Objects.equals(maxCompactionTaskSlots, that.maxCompactionTaskSlots) + && Objects.equals(useAutoScaleSlots, that.useAutoScaleSlots) + && engine == that.engine; + } + + @Override + public int hashCode() + { + return Objects.hash(compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots, engine); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java b/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java index 88eaa3e923a..d482903e0d6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CompactionConfigValidationResult.java @@ -23,10 +23,23 @@ import org.apache.druid.java.util.common.StringUtils; public class CompactionConfigValidationResult { + private static final CompactionConfigValidationResult SUCCESS + = new CompactionConfigValidationResult(true, null); + private final boolean valid; private final String reason; - public CompactionConfigValidationResult(boolean valid, String format, Object... args) + public static CompactionConfigValidationResult success() + { + return SUCCESS; + } + + public static CompactionConfigValidationResult failure(String msgFormat, Object... args) + { + return new CompactionConfigValidationResult(false, msgFormat, args); + } + + private CompactionConfigValidationResult(boolean valid, String format, Object... args) { this.valid = valid; this.reason = format == null ? null : StringUtils.format(format, args); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java index c82adfef1e8..e8265bcdd1a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java @@ -31,7 +31,7 @@ import java.util.function.UnaryOperator; /** * Manager to fetch and update dynamic configs {@link CoordinatorDynamicConfig} - * and {@link CoordinatorCompactionConfig}. + * and {@link DruidCompactionConfig}. */ public class CoordinatorConfigManager { @@ -71,12 +71,12 @@ public class CoordinatorConfigManager ); } - public CoordinatorCompactionConfig getCurrentCompactionConfig() + public DruidCompactionConfig getCurrentCompactionConfig() { - CoordinatorCompactionConfig config = jacksonConfigManager.watch( - CoordinatorCompactionConfig.CONFIG_KEY, - CoordinatorCompactionConfig.class, - CoordinatorCompactionConfig.empty() + DruidCompactionConfig config = jacksonConfigManager.watch( + DruidCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.class, + DruidCompactionConfig.empty() ).get(); return Preconditions.checkNotNull(config, "Got null config from watcher?!"); @@ -91,7 +91,7 @@ public class CoordinatorConfigManager * or if the update was successful. */ public ConfigManager.SetResult getAndUpdateCompactionConfig( - UnaryOperator operator, + UnaryOperator operator, AuditInfo auditInfo ) { @@ -102,16 +102,16 @@ public class CoordinatorConfigManager tablesConfig.getConfigTable(), MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, - CoordinatorCompactionConfig.CONFIG_KEY + DruidCompactionConfig.CONFIG_KEY ); - CoordinatorCompactionConfig current = convertBytesToCompactionConfig(currentBytes); - CoordinatorCompactionConfig updated = operator.apply(current); + DruidCompactionConfig current = convertBytesToCompactionConfig(currentBytes); + DruidCompactionConfig updated = operator.apply(current); if (current.equals(updated)) { return ConfigManager.SetResult.ok(); } else { return jacksonConfigManager.set( - CoordinatorCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, currentBytes, updated, auditInfo @@ -119,12 +119,12 @@ public class CoordinatorConfigManager } } - public CoordinatorCompactionConfig convertBytesToCompactionConfig(byte[] bytes) + public DruidCompactionConfig convertBytesToCompactionConfig(byte[] bytes) { return jacksonConfigManager.convertByteToConfig( bytes, - CoordinatorCompactionConfig.class, - CoordinatorCompactionConfig.empty() + DruidCompactionConfig.class, + DruidCompactionConfig.empty() ); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 767e8218f31..193b53a5d05 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -42,6 +42,12 @@ public class DataSourceCompactionConfig private final String dataSource; private final int taskPriority; private final long inputSegmentSizeBytes; + + public static Builder builder() + { + return new Builder(); + } + /** * The number of input segments is limited because the byte size of a serialized task spec is limited by * org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig.maxZnodeBytes. @@ -227,4 +233,127 @@ public class DataSourceCompactionConfig result = 31 * result + Arrays.hashCode(metricsSpec); return result; } + + public static class Builder + { + private String dataSource; + private Integer taskPriority; + private Long inputSegmentSizeBytes; + private Integer maxRowsPerSegment; + private Period skipOffsetFromLatest; + private UserCompactionTaskQueryTuningConfig tuningConfig; + private UserCompactionTaskGranularityConfig granularitySpec; + private UserCompactionTaskDimensionsConfig dimensionsSpec; + private AggregatorFactory[] metricsSpec; + private UserCompactionTaskTransformConfig transformSpec; + private UserCompactionTaskIOConfig ioConfig; + private CompactionEngine engine; + private Map taskContext; + + public DataSourceCompactionConfig build() + { + return new DataSourceCompactionConfig( + dataSource, + taskPriority, + inputSegmentSizeBytes, + maxRowsPerSegment, + skipOffsetFromLatest, + tuningConfig, + granularitySpec, + dimensionsSpec, + metricsSpec, + transformSpec, + ioConfig, + engine, + taskContext + ); + } + + public Builder forDataSource(String dataSource) + { + this.dataSource = dataSource; + return this; + } + + public Builder withTaskPriority(Integer taskPriority) + { + this.taskPriority = taskPriority; + return this; + } + + public Builder withInputSegmentSizeBytes(Long inputSegmentSizeBytes) + { + this.inputSegmentSizeBytes = inputSegmentSizeBytes; + return this; + } + + @Deprecated + public Builder withMaxRowsPerSegment(Integer maxRowsPerSegment) + { + this.maxRowsPerSegment = maxRowsPerSegment; + return this; + } + + public Builder withSkipOffsetFromLatest(Period skipOffsetFromLatest) + { + this.skipOffsetFromLatest = skipOffsetFromLatest; + return this; + } + + public Builder withTuningConfig( + UserCompactionTaskQueryTuningConfig tuningConfig + ) + { + this.tuningConfig = tuningConfig; + return this; + } + + public Builder withGranularitySpec( + UserCompactionTaskGranularityConfig granularitySpec + ) + { + this.granularitySpec = granularitySpec; + return this; + } + + public Builder withDimensionsSpec( + UserCompactionTaskDimensionsConfig dimensionsSpec + ) + { + this.dimensionsSpec = dimensionsSpec; + return this; + } + + public Builder withMetricsSpec(AggregatorFactory[] metricsSpec) + { + this.metricsSpec = metricsSpec; + return this; + } + + public Builder withTransformSpec( + UserCompactionTaskTransformConfig transformSpec + ) + { + this.transformSpec = transformSpec; + return this; + } + + public Builder withIoConfig(UserCompactionTaskIOConfig ioConfig) + { + this.ioConfig = ioConfig; + return this; + } + + public Builder withEngine(CompactionEngine engine) + { + this.engine = engine; + return this; + } + + public Builder withTaskContext(Map taskContext) + { + this.taskContext = taskContext; + return this; + } + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java index 3424212446f..fdbbb57e53e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java @@ -20,24 +20,25 @@ package org.apache.druid.server.coordinator; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.audit.AuditInfo; import org.joda.time.DateTime; +import java.util.Objects; + /** * A DTO containing audit information for compaction config for a datasource. */ public class DataSourceCompactionConfigAuditEntry { - private final GlobalCompactionConfig globalConfig; + private final ClusterCompactionConfig globalConfig; private final DataSourceCompactionConfig compactionConfig; private final AuditInfo auditInfo; private final DateTime auditTime; @JsonCreator public DataSourceCompactionConfigAuditEntry( - @JsonProperty("globalConfig") GlobalCompactionConfig globalConfig, + @JsonProperty("globalConfig") ClusterCompactionConfig globalConfig, @JsonProperty("compactionConfig") DataSourceCompactionConfig compactionConfig, @JsonProperty("auditInfo") AuditInfo auditInfo, @JsonProperty("auditTime") DateTime auditTime @@ -50,7 +51,7 @@ public class DataSourceCompactionConfigAuditEntry } @JsonProperty - public GlobalCompactionConfig getGlobalConfig() + public ClusterCompactionConfig getGlobalConfig() { return globalConfig; } @@ -73,52 +74,9 @@ public class DataSourceCompactionConfigAuditEntry return auditTime; } - /** - * A DTO containing compaction config for that affects the entire cluster. - */ - public static class GlobalCompactionConfig + public boolean hasSameConfig(DataSourceCompactionConfigAuditEntry other) { - private final double compactionTaskSlotRatio; - private final int maxCompactionTaskSlots; - private final boolean useAutoScaleSlots; - - @JsonCreator - public GlobalCompactionConfig( - @JsonProperty("compactionTaskSlotRatio") - double compactionTaskSlotRatio, - @JsonProperty("maxCompactionTaskSlots") int maxCompactionTaskSlots, - @JsonProperty("useAutoScaleSlots") boolean useAutoScaleSlots - ) - { - this.compactionTaskSlotRatio = compactionTaskSlotRatio; - this.maxCompactionTaskSlots = maxCompactionTaskSlots; - this.useAutoScaleSlots = useAutoScaleSlots; - } - - @JsonProperty - public double getCompactionTaskSlotRatio() - { - return compactionTaskSlotRatio; - } - - @JsonProperty - public int getMaxCompactionTaskSlots() - { - return maxCompactionTaskSlots; - } - - @JsonProperty - public boolean isUseAutoScaleSlots() - { - return useAutoScaleSlots; - } - - @JsonIgnore - public boolean hasSameConfig(CoordinatorCompactionConfig coordinatorCompactionConfig) - { - return useAutoScaleSlots == coordinatorCompactionConfig.isUseAutoScaleSlots() && - compactionTaskSlotRatio == coordinatorCompactionConfig.getCompactionTaskSlotRatio() && - coordinatorCompactionConfig.getMaxCompactionTaskSlots() == maxCompactionTaskSlots; - } + return Objects.equals(this.compactionConfig, other.compactionConfig) + && Objects.equals(this.globalConfig, other.globalConfig); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistory.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistory.java index ceb4be0d8a6..4ff9d57465b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistory.java @@ -27,7 +27,7 @@ import java.util.Stack; /** * A utility class to build the config history for a datasource from audit entries for - * {@link CoordinatorCompactionConfig}. The {@link CoordinatorCompactionConfig} contains the entire config for the + * {@link DruidCompactionConfig}. The {@link DruidCompactionConfig} contains the entire config for the * cluster, so this class creates adds audit entires to the history only when a setting for this datasource or a global * setting has changed. */ @@ -41,54 +41,29 @@ public class DataSourceCompactionConfigHistory this.dataSource = dataSource; } - public void add(CoordinatorCompactionConfig coordinatorCompactionConfig, AuditInfo auditInfo, DateTime auditTime) + public void add(DruidCompactionConfig compactionConfig, AuditInfo auditInfo, DateTime auditTime) { - DataSourceCompactionConfigAuditEntry current = auditEntries.isEmpty() ? null : auditEntries.peek(); - DataSourceCompactionConfigAuditEntry newEntry = null; - boolean hasDataSourceCompactionConfig = false; - for (DataSourceCompactionConfig dataSourceCompactionConfig : coordinatorCompactionConfig.getCompactionConfigs()) { - if (dataSource.equals(dataSourceCompactionConfig.getDataSource())) { - hasDataSourceCompactionConfig = true; - if ( - current == null || - ( - !dataSourceCompactionConfig.equals(current.getCompactionConfig()) || - !current.getGlobalConfig().hasSameConfig(coordinatorCompactionConfig) - ) - ) { - current = new DataSourceCompactionConfigAuditEntry( - new DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig( - coordinatorCompactionConfig.getCompactionTaskSlotRatio(), - coordinatorCompactionConfig.getMaxCompactionTaskSlots(), - coordinatorCompactionConfig.isUseAutoScaleSlots() - ), - dataSourceCompactionConfig, - auditInfo, - auditTime - ); - newEntry = current; - } - break; - } + final DataSourceCompactionConfigAuditEntry previousEntry = auditEntries.isEmpty() ? null : auditEntries.peek(); + final DataSourceCompactionConfigAuditEntry newEntry = new DataSourceCompactionConfigAuditEntry( + compactionConfig.clusterConfig(), + compactionConfig.findConfigForDatasource(dataSource).orNull(), + auditInfo, + auditTime + ); + + final boolean shouldAddEntry; + if (previousEntry == null) { + shouldAddEntry = newEntry.getCompactionConfig() != null; + } else { + shouldAddEntry = !newEntry.hasSameConfig(previousEntry); } - if (newEntry != null) { - auditEntries.push(newEntry); - } else if (current != null && !hasDataSourceCompactionConfig) { - newEntry = new DataSourceCompactionConfigAuditEntry( - new DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig( - coordinatorCompactionConfig.getCompactionTaskSlotRatio(), - coordinatorCompactionConfig.getMaxCompactionTaskSlots(), - coordinatorCompactionConfig.isUseAutoScaleSlots() - ), - null, - auditInfo, - auditTime - ); + + if (shouldAddEntry) { auditEntries.push(newEntry); } } - public List getHistory() + public List getEntries() { return auditEntries; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java similarity index 55% rename from server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java rename to server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java index 036c53121e9..7793b55c4b9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java @@ -21,83 +21,84 @@ package org.apache.druid.server.coordinator; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableList; +import com.google.common.base.Optional; import org.apache.druid.common.config.Configs; import org.apache.druid.indexer.CompactionEngine; import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; -public class CoordinatorCompactionConfig +public class DruidCompactionConfig { public static final String CONFIG_KEY = "coordinator.compaction.config"; - private static final double DEFAULT_COMPACTION_TASK_RATIO = 0.1; - private static final int DEFAULT_MAX_COMPACTION_TASK_SLOTS = Integer.MAX_VALUE; - private static final boolean DEFAULT_USE_AUTO_SCALE_SLOTS = false; - private static final CompactionEngine DEFAULT_COMPACTION_ENGINE = CompactionEngine.NATIVE; + private static final DruidCompactionConfig EMPTY_INSTANCE + = new DruidCompactionConfig(Collections.emptyList(), null, null, null, null); private final List compactionConfigs; private final double compactionTaskSlotRatio; private final int maxCompactionTaskSlots; private final boolean useAutoScaleSlots; - private final CompactionEngine compactionEngine; + private final CompactionEngine engine; - public static CoordinatorCompactionConfig from( - CoordinatorCompactionConfig baseConfig, + public DruidCompactionConfig withDatasourceConfigs( List compactionConfigs ) { - return new CoordinatorCompactionConfig( + return new DruidCompactionConfig( compactionConfigs, - baseConfig.compactionTaskSlotRatio, - baseConfig.maxCompactionTaskSlots, - baseConfig.useAutoScaleSlots, - null + compactionTaskSlotRatio, + maxCompactionTaskSlots, + useAutoScaleSlots, + engine ); } - public static CoordinatorCompactionConfig from( - CoordinatorCompactionConfig baseConfig, - @Nullable Double compactionTaskSlotRatio, - @Nullable Integer maxCompactionTaskSlots, - @Nullable Boolean useAutoScaleSlots + public DruidCompactionConfig withClusterConfig( + ClusterCompactionConfig update ) { - return new CoordinatorCompactionConfig( - baseConfig.compactionConfigs, - compactionTaskSlotRatio == null ? baseConfig.compactionTaskSlotRatio : compactionTaskSlotRatio, - maxCompactionTaskSlots == null ? baseConfig.maxCompactionTaskSlots : maxCompactionTaskSlots, - useAutoScaleSlots == null ? baseConfig.useAutoScaleSlots : useAutoScaleSlots, - null + return new DruidCompactionConfig( + this.compactionConfigs, + Configs.valueOrDefault(update.getCompactionTaskSlotRatio(), compactionTaskSlotRatio), + Configs.valueOrDefault(update.getMaxCompactionTaskSlots(), maxCompactionTaskSlots), + Configs.valueOrDefault(update.getUseAutoScaleSlots(), useAutoScaleSlots), + Configs.valueOrDefault(update.getEngine(), engine) ); } - public static CoordinatorCompactionConfig from(List compactionConfigs) + public DruidCompactionConfig withDatasourceConfig(DataSourceCompactionConfig dataSourceConfig) { - return new CoordinatorCompactionConfig(compactionConfigs, null, null, null, null); + final Map configs = dataSourceToCompactionConfigMap(); + configs.put(dataSourceConfig.getDataSource(), dataSourceConfig); + return withDatasourceConfigs(new ArrayList<>(configs.values())); } - public static CoordinatorCompactionConfig empty() + public static DruidCompactionConfig empty() { - return new CoordinatorCompactionConfig(ImmutableList.of(), null, null, null, null); + return EMPTY_INSTANCE; } @JsonCreator - public CoordinatorCompactionConfig( + public DruidCompactionConfig( @JsonProperty("compactionConfigs") List compactionConfigs, @JsonProperty("compactionTaskSlotRatio") @Nullable Double compactionTaskSlotRatio, @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots, - @JsonProperty("compactionEngine") @Nullable CompactionEngine compactionEngine + @JsonProperty("engine") @Nullable CompactionEngine compactionEngine ) { - this.compactionConfigs = compactionConfigs; - this.compactionTaskSlotRatio = Configs.valueOrDefault(compactionTaskSlotRatio, DEFAULT_COMPACTION_TASK_RATIO); - this.maxCompactionTaskSlots = Configs.valueOrDefault(maxCompactionTaskSlots, DEFAULT_MAX_COMPACTION_TASK_SLOTS); - this.useAutoScaleSlots = Configs.valueOrDefault(useAutoScaleSlots, DEFAULT_USE_AUTO_SCALE_SLOTS); - this.compactionEngine = Configs.valueOrDefault(compactionEngine, DEFAULT_COMPACTION_ENGINE); + this.compactionConfigs = Configs.valueOrDefault(compactionConfigs, Collections.emptyList()); + this.compactionTaskSlotRatio = Configs.valueOrDefault(compactionTaskSlotRatio, 0.1); + this.maxCompactionTaskSlots = Configs.valueOrDefault(maxCompactionTaskSlots, Integer.MAX_VALUE); + this.useAutoScaleSlots = Configs.valueOrDefault(useAutoScaleSlots, false); + this.engine = Configs.valueOrDefault(compactionEngine, CompactionEngine.NATIVE); } @JsonProperty @@ -127,7 +128,36 @@ public class CoordinatorCompactionConfig @JsonProperty public CompactionEngine getEngine() { - return compactionEngine; + return engine; + } + + + // Null-safe getters not used for serialization + public ClusterCompactionConfig clusterConfig() + { + return new ClusterCompactionConfig( + compactionTaskSlotRatio, + maxCompactionTaskSlots, + useAutoScaleSlots, + engine + ); + } + + public Map dataSourceToCompactionConfigMap() + { + return getCompactionConfigs().stream().collect( + Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity()) + ); + } + + public Optional findConfigForDatasource(String dataSource) + { + for (DataSourceCompactionConfig dataSourceConfig : getCompactionConfigs()) { + if (dataSource.equals(dataSourceConfig.getDataSource())) { + return Optional.of(dataSourceConfig); + } + } + return Optional.absent(); } @Override @@ -139,11 +169,11 @@ public class CoordinatorCompactionConfig if (o == null || getClass() != o.getClass()) { return false; } - CoordinatorCompactionConfig that = (CoordinatorCompactionConfig) o; + DruidCompactionConfig that = (DruidCompactionConfig) o; return Double.compare(that.compactionTaskSlotRatio, compactionTaskSlotRatio) == 0 && maxCompactionTaskSlots == that.maxCompactionTaskSlots && useAutoScaleSlots == that.useAutoScaleSlots && - compactionEngine == that.compactionEngine && + engine == that.engine && Objects.equals(compactionConfigs, that.compactionConfigs); } @@ -155,7 +185,7 @@ public class CoordinatorCompactionConfig compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots, - compactionEngine + engine ); } @@ -167,7 +197,7 @@ public class CoordinatorCompactionConfig ", compactionTaskSlotRatio=" + compactionTaskSlotRatio + ", maxCompactionTaskSlots=" + maxCompactionTaskSlots + ", useAutoScaleSlots=" + useAutoScaleSlots + - ", compactionEngine=" + compactionEngine + + ", engine=" + engine + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 9710bda79b4..5e468182fa7 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -706,7 +706,7 @@ public class DruidCoordinator = metadataManager.segments().getSnapshotOfDataSourcesWithAllUsedSegments(); final CoordinatorDynamicConfig dynamicConfig = metadataManager.configs().getCurrentDynamicConfig(); - final CoordinatorCompactionConfig compactionConfig = metadataManager.configs().getCurrentCompactionConfig(); + final DruidCompactionConfig compactionConfig = metadataManager.configs().getCurrentCompactionConfig(); DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams .newBuilder(coordinatorStartTime) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index ebdbd4f500e..45f3993caa5 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -67,7 +67,7 @@ public class DruidCoordinatorRuntimeParams private final @Nullable TreeSet usedSegments; private final @Nullable DataSourcesSnapshot dataSourcesSnapshot; private final CoordinatorDynamicConfig coordinatorDynamicConfig; - private final CoordinatorCompactionConfig coordinatorCompactionConfig; + private final DruidCompactionConfig compactionConfig; private final SegmentLoadingConfig segmentLoadingConfig; private final CoordinatorRunStats stats; private final BalancerStrategy balancerStrategy; @@ -81,7 +81,7 @@ public class DruidCoordinatorRuntimeParams @Nullable TreeSet usedSegments, @Nullable DataSourcesSnapshot dataSourcesSnapshot, CoordinatorDynamicConfig coordinatorDynamicConfig, - CoordinatorCompactionConfig coordinatorCompactionConfig, + DruidCompactionConfig compactionConfig, SegmentLoadingConfig segmentLoadingConfig, CoordinatorRunStats stats, BalancerStrategy balancerStrategy, @@ -95,7 +95,7 @@ public class DruidCoordinatorRuntimeParams this.usedSegments = usedSegments; this.dataSourcesSnapshot = dataSourcesSnapshot; this.coordinatorDynamicConfig = coordinatorDynamicConfig; - this.coordinatorCompactionConfig = coordinatorCompactionConfig; + this.compactionConfig = compactionConfig; this.segmentLoadingConfig = segmentLoadingConfig; this.stats = stats; this.balancerStrategy = balancerStrategy; @@ -151,9 +151,9 @@ public class DruidCoordinatorRuntimeParams return coordinatorDynamicConfig; } - public CoordinatorCompactionConfig getCoordinatorCompactionConfig() + public DruidCompactionConfig getCompactionConfig() { - return coordinatorCompactionConfig; + return compactionConfig; } public SegmentLoadingConfig getSegmentLoadingConfig() @@ -197,7 +197,7 @@ public class DruidCoordinatorRuntimeParams usedSegments, dataSourcesSnapshot, coordinatorDynamicConfig, - coordinatorCompactionConfig, + compactionConfig, segmentLoadingConfig, stats, balancerStrategy, @@ -215,7 +215,7 @@ public class DruidCoordinatorRuntimeParams private @Nullable TreeSet usedSegments; private @Nullable DataSourcesSnapshot dataSourcesSnapshot; private CoordinatorDynamicConfig coordinatorDynamicConfig; - private CoordinatorCompactionConfig coordinatorCompactionConfig; + private DruidCompactionConfig compactionConfig; private SegmentLoadingConfig segmentLoadingConfig; private CoordinatorRunStats stats; private BalancerStrategy balancerStrategy; @@ -225,7 +225,7 @@ public class DruidCoordinatorRuntimeParams { this.coordinatorStartTime = coordinatorStartTime; this.coordinatorDynamicConfig = CoordinatorDynamicConfig.builder().build(); - this.coordinatorCompactionConfig = CoordinatorCompactionConfig.empty(); + this.compactionConfig = DruidCompactionConfig.empty(); this.broadcastDatasources = Collections.emptySet(); } @@ -237,7 +237,7 @@ public class DruidCoordinatorRuntimeParams @Nullable TreeSet usedSegments, @Nullable DataSourcesSnapshot dataSourcesSnapshot, CoordinatorDynamicConfig coordinatorDynamicConfig, - CoordinatorCompactionConfig coordinatorCompactionConfig, + DruidCompactionConfig compactionConfig, SegmentLoadingConfig segmentLoadingConfig, CoordinatorRunStats stats, BalancerStrategy balancerStrategy, @@ -251,7 +251,7 @@ public class DruidCoordinatorRuntimeParams this.usedSegments = usedSegments; this.dataSourcesSnapshot = dataSourcesSnapshot; this.coordinatorDynamicConfig = coordinatorDynamicConfig; - this.coordinatorCompactionConfig = coordinatorCompactionConfig; + this.compactionConfig = compactionConfig; this.segmentLoadingConfig = segmentLoadingConfig; this.stats = stats; this.balancerStrategy = balancerStrategy; @@ -271,7 +271,7 @@ public class DruidCoordinatorRuntimeParams usedSegments, dataSourcesSnapshot, coordinatorDynamicConfig, - coordinatorCompactionConfig, + compactionConfig, segmentLoadingConfig, stats, balancerStrategy, @@ -367,9 +367,9 @@ public class DruidCoordinatorRuntimeParams return this; } - public Builder withCompactionConfig(CoordinatorCompactionConfig config) + public Builder withCompactionConfig(DruidCompactionConfig config) { - this.coordinatorCompactionConfig = config; + this.compactionConfig = config; return this; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 01f3bc77e9e..7b2392b8c66 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -49,8 +49,8 @@ import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator; import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; @@ -120,7 +120,7 @@ public class CompactSegments implements CoordinatorCustomDuty { LOG.info("Running CompactSegments duty"); - final CoordinatorCompactionConfig dynamicConfig = params.getCoordinatorCompactionConfig(); + final DruidCompactionConfig dynamicConfig = params.getCompactionConfig(); final int maxCompactionTaskSlots = dynamicConfig.getMaxCompactionTaskSlots(); if (maxCompactionTaskSlots <= 0) { LOG.info("Skipping compaction as maxCompactionTaskSlots is [%d].", maxCompactionTaskSlots); @@ -278,7 +278,8 @@ public class CompactSegments implements CoordinatorCustomDuty { final List lockFilterPolicies = compactionConfigs .stream() - .map(config -> new LockFilterPolicy(config.getDataSource(), config.getTaskPriority(), config.getTaskContext())) + .map(config -> + new LockFilterPolicy(config.getDataSource(), config.getTaskPriority(), null, config.getTaskContext())) .collect(Collectors.toList()); final Map> datasourceToLockedIntervals = new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervals(lockFilterPolicies), true)); @@ -343,7 +344,7 @@ public class CompactSegments implements CoordinatorCustomDuty return tuningConfig.getPartitionsSpec() instanceof DimensionRangePartitionsSpec; } - private int getCompactionTaskCapacity(CoordinatorCompactionConfig dynamicConfig) + private int getCompactionTaskCapacity(DruidCompactionConfig dynamicConfig) { int totalWorkerCapacity = CoordinatorDutyUtils.getTotalWorkerCapacity(overlordClient); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillCompactionConfig.java index c231c31c543..7395be6d843 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillCompactionConfig.java @@ -26,9 +26,9 @@ import org.apache.druid.java.util.RetryableException; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.SegmentsMetadataManager; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; import org.apache.druid.server.coordinator.stats.Stats; import org.joda.time.DateTime; @@ -82,12 +82,12 @@ public class KillCompactionConfig extends MetadataCleanupDuty /** * Creates a new compaction config by deleting entries for inactive datasources. */ - private CoordinatorCompactionConfig deleteConfigsForInactiveDatasources( - CoordinatorCompactionConfig current + private DruidCompactionConfig deleteConfigsForInactiveDatasources( + DruidCompactionConfig current ) { // If current compaction config is empty then there is nothing to do - if (CoordinatorCompactionConfig.empty().equals(current)) { + if (DruidCompactionConfig.empty().equals(current)) { log.info("Nothing to do as compaction config is already empty."); return current; } @@ -102,7 +102,7 @@ public class KillCompactionConfig extends MetadataCleanupDuty .filter(dataSourceCompactionConfig -> activeDatasources.contains(dataSourceCompactionConfig.getDataSource())) .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - return CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(updated.values())); + return current.withDatasourceConfigs(ImmutableList.copyOf(updated.values())); } /** @@ -116,7 +116,7 @@ public class KillCompactionConfig extends MetadataCleanupDuty ConfigManager.SetResult result = configManager.getAndUpdateCompactionConfig( current -> { - final CoordinatorCompactionConfig updated = deleteConfigsForInactiveDatasources(current); + final DruidCompactionConfig updated = deleteConfigsForInactiveDatasources(current); int numCurrentConfigs = current.getCompactionConfigs() == null ? 0 : current.getCompactionConfigs().size(); int numUpdatedConfigs = updated.getCompactionConfigs() == null ? 0 : updated.getCompactionConfigs().size(); compactionConfigRemoved.set(Math.max(0, numCurrentConfigs - numUpdatedConfigs)); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java index 575c320b9e0..64b61df5e53 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java @@ -20,6 +20,8 @@ package org.apache.druid.server.coordinator.duty; import com.google.common.base.Predicate; +import com.google.common.collect.Sets; +import org.apache.druid.collections.CircularList; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.java.util.common.DateTimes; @@ -40,10 +42,11 @@ import org.joda.time.Duration; import org.joda.time.Interval; import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.Collection; +import java.util.Comparator; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; /** @@ -58,6 +61,11 @@ import java.util.concurrent.ConcurrentHashMap; * as there can be multiple unused segments with different {@code used_status_last_updated} time. *

    *

    + * The datasources to be killed during each cycle are selected from {@link #datasourceCircularKillList}. This state is + * refreshed in a run if the set of datasources to be killed changes. Consecutive duplicate datasources are avoided + * across runs, provided there are other datasources to be killed. + *

    + *

    * See {@link org.apache.druid.indexing.common.task.KillUnusedSegmentsTask}. *

    */ @@ -75,18 +83,22 @@ public class KillUnusedSegments implements CoordinatorDuty private final Duration durationToRetain; private final boolean ignoreDurationToRetain; private final int maxSegmentsToKill; + private final Duration bufferPeriod; /** * Used to keep track of the last interval end time that was killed for each * datasource. */ private final Map datasourceToLastKillIntervalEnd; + private DateTime lastKillTime; - private final Duration bufferPeriod; private final SegmentsMetadataManager segmentsMetadataManager; private final OverlordClient overlordClient; + private String prevDatasourceKilled; + private CircularList datasourceCircularKillList; + public KillUnusedSegments( SegmentsMetadataManager segmentsMetadataManager, OverlordClient overlordClient, @@ -94,7 +106,6 @@ public class KillUnusedSegments implements CoordinatorDuty ) { this.period = killConfig.getCleanupPeriod(); - this.maxSegmentsToKill = killConfig.getMaxSegments(); this.ignoreDurationToRetain = killConfig.isIgnoreDurationToRetain(); this.durationToRetain = killConfig.getDurationToRetain(); @@ -107,8 +118,6 @@ public class KillUnusedSegments implements CoordinatorDuty } this.bufferPeriod = killConfig.getBufferPeriod(); - datasourceToLastKillIntervalEnd = new ConcurrentHashMap<>(); - log.info( "Kill task scheduling enabled with period[%s], durationToRetain[%s], bufferPeriod[%s], maxSegmentsToKill[%s]", this.period, @@ -119,6 +128,7 @@ public class KillUnusedSegments implements CoordinatorDuty this.segmentsMetadataManager = segmentsMetadataManager; this.overlordClient = overlordClient; + this.datasourceToLastKillIntervalEnd = new ConcurrentHashMap<>(); } @Override @@ -141,18 +151,27 @@ public class KillUnusedSegments implements CoordinatorDuty final CoordinatorRunStats stats = params.getCoordinatorStats(); final int availableKillTaskSlots = getAvailableKillTaskSlots(dynamicConfig, stats); - Collection dataSourcesToKill = dynamicConfig.getSpecificDataSourcesToKillUnusedSegmentsIn(); - - if (availableKillTaskSlots > 0) { - // If no datasource has been specified, all are eligible for killing unused segments - if (CollectionUtils.isNullOrEmpty(dataSourcesToKill)) { - dataSourcesToKill = segmentsMetadataManager.retrieveAllDataSourceNames(); - } - - lastKillTime = DateTimes.nowUtc(); - killUnusedSegments(dataSourcesToKill, availableKillTaskSlots, stats); + if (availableKillTaskSlots <= 0) { + log.debug("Skipping KillUnusedSegments because there are no available kill task slots."); + return params; } + final Set dataSourcesToKill; + if (CollectionUtils.isNullOrEmpty(dynamicConfig.getSpecificDataSourcesToKillUnusedSegmentsIn())) { + dataSourcesToKill = segmentsMetadataManager.retrieveAllDataSourceNames(); + } else { + dataSourcesToKill = dynamicConfig.getSpecificDataSourcesToKillUnusedSegmentsIn(); + } + + if (datasourceCircularKillList == null || + !datasourceCircularKillList.equalsSet(dataSourcesToKill)) { + datasourceCircularKillList = new CircularList<>(dataSourcesToKill, Comparator.naturalOrder()); + } + + lastKillTime = DateTimes.nowUtc(); + + killUnusedSegments(dataSourcesToKill, availableKillTaskSlots, stats); + // any datasources that are no longer being considered for kill should have their // last kill interval removed from map. datasourceToLastKillIntervalEnd.keySet().retainAll(dataSourcesToKill); @@ -163,30 +182,37 @@ public class KillUnusedSegments implements CoordinatorDuty * Spawn kill tasks for each datasource in {@code dataSourcesToKill} upto {@code availableKillTaskSlots}. */ private void killUnusedSegments( - @Nullable final Collection dataSourcesToKill, + final Set dataSourcesToKill, final int availableKillTaskSlots, final CoordinatorRunStats stats ) { - if (CollectionUtils.isNullOrEmpty(dataSourcesToKill) || availableKillTaskSlots <= 0) { + if (CollectionUtils.isNullOrEmpty(dataSourcesToKill)) { + log.debug("Skipping KillUnusedSegments because there are no datasources to kill."); stats.add(Stats.Kill.SUBMITTED_TASKS, 0); return; } - final Collection remainingDatasourcesToKill = new ArrayList<>(dataSourcesToKill); + final Set remainingDatasourcesToKill = new HashSet<>(dataSourcesToKill); + int submittedTasks = 0; - for (String dataSource : dataSourcesToKill) { - if (submittedTasks >= availableKillTaskSlots) { - log.info( - "Submitted [%d] kill tasks and reached kill task slot limit [%d].", - submittedTasks, availableKillTaskSlots - ); - break; + for (String dataSource : datasourceCircularKillList) { + if (dataSource.equals(prevDatasourceKilled) && remainingDatasourcesToKill.size() > 1) { + // Skip this dataSource if it's the same as the previous one and there are remaining datasources to kill. + continue; + } else { + prevDatasourceKilled = dataSource; + remainingDatasourcesToKill.remove(dataSource); } + final DateTime maxUsedStatusLastUpdatedTime = DateTimes.nowUtc().minus(bufferPeriod); final Interval intervalToKill = findIntervalForKill(dataSource, maxUsedStatusLastUpdatedTime, stats); if (intervalToKill == null) { datasourceToLastKillIntervalEnd.remove(dataSource); + // If no interval is found for this datasource, either terminate or continue based on remaining datasources to kill. + if (remainingDatasourcesToKill.isEmpty()) { + break; + } continue; } @@ -204,7 +230,11 @@ public class KillUnusedSegments implements CoordinatorDuty ); ++submittedTasks; datasourceToLastKillIntervalEnd.put(dataSource, intervalToKill.getEnd()); - remainingDatasourcesToKill.remove(dataSource); + + // Termination conditions. + if (remainingDatasourcesToKill.isEmpty() || submittedTasks >= availableKillTaskSlots) { + break; + } } catch (Exception ex) { log.error(ex, "Failed to submit kill task for dataSource[%s] in interval[%s]", dataSource, intervalToKill); @@ -216,8 +246,12 @@ public class KillUnusedSegments implements CoordinatorDuty } log.info( - "Submitted [%d] kill tasks for [%d] datasources. Remaining datasources to kill: %s", - submittedTasks, dataSourcesToKill.size() - remainingDatasourcesToKill.size(), remainingDatasourcesToKill + "Submitted [%d] kill tasks for [%d] datasources: [%s]. Remaining [%d] datasources to kill: [%s].", + submittedTasks, + dataSourcesToKill.size() - remainingDatasourcesToKill.size(), + Sets.difference(dataSourcesToKill, remainingDatasourcesToKill), + remainingDatasourcesToKill.size(), + remainingDatasourcesToKill ); stats.add(Stats.Kill.SUBMITTED_TASKS, submittedTasks); @@ -230,13 +264,14 @@ public class KillUnusedSegments implements CoordinatorDuty final CoordinatorRunStats stats ) { + final DateTime minStartTime = datasourceToLastKillIntervalEnd.get(dataSource); final DateTime maxEndTime = ignoreDurationToRetain ? DateTimes.COMPARE_DATE_AS_STRING_MAX : DateTimes.nowUtc().minus(durationToRetain); final List unusedSegmentIntervals = segmentsMetadataManager.getUnusedSegmentIntervals( dataSource, - datasourceToLastKillIntervalEnd.get(dataSource), + minStartTime, maxEndTime, maxSegmentsToKill, maxUsedStatusLastUpdatedTime diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index 0bba5cf63fa..7bd4ee85124 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -19,6 +19,7 @@ package org.apache.druid.server.http; +import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; @@ -33,13 +34,15 @@ import org.apache.druid.error.InvalidInput; import org.apache.druid.error.NotFound; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfigHistory; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.servlet.http.HttpServletRequest; @@ -59,9 +62,7 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; import java.util.concurrent.ThreadLocalRandom; -import java.util.function.Function; import java.util.function.UnaryOperator; -import java.util.stream.Collectors; @Path("/druid/coordinator/v1/config/compaction") @ResourceFilters(ConfigResourceFilter.class) @@ -69,7 +70,7 @@ public class CoordinatorCompactionConfigsResource { private static final Logger LOG = new Logger(CoordinatorCompactionConfigsResource.class); private static final long UPDATE_RETRY_DELAY = 1000; - static final int UPDATE_NUM_RETRY = 5; + static final int MAX_UPDATE_RETRIES = 5; private final CoordinatorConfigManager configManager; private final AuditManager auditManager; @@ -92,6 +93,45 @@ public class CoordinatorCompactionConfigsResource } @POST + @Path("/cluster") + @Consumes(MediaType.APPLICATION_JSON) + public Response updateClusterCompactionConfig( + ClusterCompactionConfig updatePayload, + @Context HttpServletRequest req + ) + { + UnaryOperator operator = current -> { + final DruidCompactionConfig newConfig = current.withClusterConfig(updatePayload); + + final List datasourceConfigs = newConfig.getCompactionConfigs(); + if (CollectionUtils.isNullOrEmpty(datasourceConfigs) + || current.getEngine() == newConfig.getEngine()) { + return newConfig; + } + + // Validate all the datasource configs against the new engine + for (DataSourceCompactionConfig datasourceConfig : datasourceConfigs) { + CompactionConfigValidationResult validationResult = + ClientCompactionRunnerInfo.validateCompactionConfig(datasourceConfig, newConfig.getEngine()); + if (!validationResult.isValid()) { + throw InvalidInput.exception( + "Cannot update engine to [%s] as it does not support" + + " compaction config of DataSource[%s]. Reason[%s].", + newConfig.getEngine(), datasourceConfig.getDataSource(), validationResult.getReason() + ); + } + } + + return newConfig; + }; + return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); + } + + /** + * @deprecated in favor of {@link #updateClusterCompactionConfig}. + */ + @POST + @Deprecated @Path("/taskslots") @Consumes(MediaType.APPLICATION_JSON) public Response setCompactionTaskLimit( @@ -101,39 +141,32 @@ public class CoordinatorCompactionConfigsResource @Context HttpServletRequest req ) { - UnaryOperator operator = - current -> CoordinatorCompactionConfig.from( - current, + return updateClusterCompactionConfig( + new ClusterCompactionConfig( compactionTaskSlotRatio, maxCompactionTaskSlots, - useAutoScaleSlots - ); - return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); + useAutoScaleSlots, + null + ), + req + ); } @POST @Consumes(MediaType.APPLICATION_JSON) - public Response addOrUpdateCompactionConfig( + public Response addOrUpdateDatasourceCompactionConfig( final DataSourceCompactionConfig newConfig, @Context HttpServletRequest req ) { - UnaryOperator callable = current -> { - final CoordinatorCompactionConfig newCompactionConfig; - final Map newConfigs = current - .getCompactionConfigs() - .stream() - .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); + UnaryOperator callable = current -> { CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig(newConfig, current.getEngine()); - if (!validationResult.isValid()) { + if (validationResult.isValid()) { + return current.withDatasourceConfig(newConfig); + } else { throw InvalidInput.exception("Compaction config not supported. Reason[%s].", validationResult.getReason()); } - // Don't persist config with the default engine if engine not specified, to enable update of the default. - newConfigs.put(newConfig.getDataSource(), newConfig); - newCompactionConfig = CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(newConfigs.values())); - - return newCompactionConfig; }; return updateConfigHelper( callable, @@ -144,20 +177,15 @@ public class CoordinatorCompactionConfigsResource @GET @Path("/{dataSource}") @Produces(MediaType.APPLICATION_JSON) - public Response getCompactionConfig(@PathParam("dataSource") String dataSource) + public Response getDatasourceCompactionConfig(@PathParam("dataSource") String dataSource) { - final CoordinatorCompactionConfig current = configManager.getCurrentCompactionConfig(); - final Map configs = current - .getCompactionConfigs() - .stream() - .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - - final DataSourceCompactionConfig config = configs.get(dataSource); - if (config == null) { + final DruidCompactionConfig current = configManager.getCurrentCompactionConfig(); + final Optional config = current.findConfigForDatasource(dataSource); + if (config.isPresent()) { + return Response.ok().entity(config.get()).build(); + } else { return Response.status(Response.Status.NOT_FOUND).build(); } - - return Response.ok().entity(config).build(); } @GET @@ -174,25 +202,25 @@ public class CoordinatorCompactionConfigsResource List auditEntries; if (theInterval == null && count != null) { auditEntries = auditManager.fetchAuditHistory( - CoordinatorCompactionConfig.CONFIG_KEY, - CoordinatorCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, count ); } else { auditEntries = auditManager.fetchAuditHistory( - CoordinatorCompactionConfig.CONFIG_KEY, - CoordinatorCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, theInterval ); } DataSourceCompactionConfigHistory history = new DataSourceCompactionConfigHistory(dataSource); for (AuditEntry audit : auditEntries) { - CoordinatorCompactionConfig coordinatorCompactionConfig = configManager.convertBytesToCompactionConfig( + DruidCompactionConfig compactionConfig = configManager.convertBytesToCompactionConfig( audit.getPayload().serialized().getBytes(StandardCharsets.UTF_8) ); - history.add(coordinatorCompactionConfig, audit.getAuditInfo(), audit.getAuditTime()); + history.add(compactionConfig, audit.getAuditInfo(), audit.getAuditTime()); } - return Response.ok(history.getHistory()).build(); + return Response.ok(history.getEntries()).build(); } catch (IllegalArgumentException e) { return Response.status(Response.Status.BAD_REQUEST) @@ -209,31 +237,27 @@ public class CoordinatorCompactionConfigsResource @Context HttpServletRequest req ) { - UnaryOperator callable = current -> { - final Map configs = current - .getCompactionConfigs() - .stream() - .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - + UnaryOperator callable = current -> { + final Map configs = current.dataSourceToCompactionConfigMap(); final DataSourceCompactionConfig config = configs.remove(dataSource); if (config == null) { throw NotFound.exception("datasource not found"); } - return CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(configs.values())); + return current.withDatasourceConfigs(ImmutableList.copyOf(configs.values())); }; return updateConfigHelper(callable, AuthorizationUtils.buildAuditInfo(req)); } private Response updateConfigHelper( - UnaryOperator configOperator, + UnaryOperator configOperator, AuditInfo auditInfo ) { int attemps = 0; SetResult setResult = null; try { - while (attemps < UPDATE_NUM_RETRY) { + while (attemps < MAX_UPDATE_RETRIES) { setResult = configManager.getAndUpdateCompactionConfig(configOperator, auditInfo); if (setResult.isOk() || !setResult.isRetryable()) { break; diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java index aead7b86c71..8ee3d5ba081 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java @@ -31,7 +31,7 @@ import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.annotations.RemoteChatHandler; import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.lifecycle.Lifecycle; -import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; +import org.apache.druid.segment.realtime.ChatHandlerResource; import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.initialization.TLSServerConfig; diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java index 687ca2ef548..feb61965daa 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java @@ -32,7 +32,7 @@ import org.apache.druid.guice.annotations.RemoteChatHandler; import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.query.lookup.LookupModule; -import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; +import org.apache.druid.segment.realtime.ChatHandlerResource; import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.initialization.TLSServerConfig; diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/TaskIdResponseHeaderFilterHolder.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/TaskIdResponseHeaderFilterHolder.java index 7dcd3b2c237..c4c0cb8ce4b 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/TaskIdResponseHeaderFilterHolder.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/TaskIdResponseHeaderFilterHolder.java @@ -21,7 +21,7 @@ package org.apache.druid.server.initialization.jetty; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; +import org.apache.druid.segment.realtime.ChatHandlerResource; public class TaskIdResponseHeaderFilterHolder extends ResponseHeaderFilterHolder { diff --git a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMetric.java b/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMetric.java deleted file mode 100644 index 13502e18493..00000000000 --- a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMetric.java +++ /dev/null @@ -1,48 +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.server.metrics; - -/** - * An EventReceiverFirehoseMetric is an object with metrics about EventReceiverFirehose objects. - * It is not likely that anything other than an EventReceiverFirehose actually implements this. - * This interface is not part of the public API and backwards incompatible changes can occur without - * requiring a major (or even minor) version change. - * The interface's primary purpose is to be able to share metrics via the EventReceiverFirehoseRegister - * without exposing the entire EventReceiverFirehose - */ -public interface EventReceiverFirehoseMetric -{ - /** - * Return the current number of {@link org.apache.druid.data.input.InputRow} that are stored in the buffer. - */ - int getCurrentBufferSize(); - - /** - * Return the capacity of the buffer. - */ - int getCapacity(); - - /** - * Return the number of bytes received by the firehose. - */ - long getBytesReceived(); - - -} diff --git a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMonitor.java b/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMonitor.java deleted file mode 100644 index 29c1808ad51..00000000000 --- a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMonitor.java +++ /dev/null @@ -1,90 +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.server.metrics; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.inject.Inject; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.java.util.metrics.AbstractMonitor; -import org.apache.druid.java.util.metrics.KeyedDiff; -import org.apache.druid.java.util.metrics.MonitorUtils; -import org.apache.druid.query.DruidMetrics; - -import java.util.Map; -import java.util.Properties; - -public class EventReceiverFirehoseMonitor extends AbstractMonitor -{ - - private final EventReceiverFirehoseRegister register; - private final KeyedDiff keyedDiff = new KeyedDiff(); - private final Map dimensions; - - @Inject - public EventReceiverFirehoseMonitor( - EventReceiverFirehoseRegister eventReceiverFirehoseRegister, - Properties props - ) - { - this.register = eventReceiverFirehoseRegister; - this.dimensions = MonitorsConfig.extractDimensions( - props, - Lists.newArrayList(DruidMetrics.DATASOURCE, DruidMetrics.TASK_ID, DruidMetrics.TASK_TYPE) - ); - } - - @Override - public boolean doMonitor(ServiceEmitter emitter) - { - for (Map.Entry entry : register.getMetrics()) { - final String serviceName = entry.getKey(); - final EventReceiverFirehoseMetric metric = entry.getValue(); - - final ServiceMetricEvent.Builder builder = createEventBuilder(serviceName) - .setDimension( - "bufferCapacity", - String.valueOf(metric.getCapacity()) - ); - emitter.emit(builder.setMetric("ingest/events/buffered", metric.getCurrentBufferSize())); - Map diff = keyedDiff.to( - serviceName, - ImmutableMap.of("ingest/bytes/received", metric.getBytesReceived()) - ); - if (diff != null) { - final ServiceMetricEvent.Builder eventBuilder = createEventBuilder(serviceName); - for (Map.Entry diffEntry : diff.entrySet()) { - emitter.emit(eventBuilder.setMetric(diffEntry.getKey(), diffEntry.getValue())); - } - } - } - - return true; - } - - private ServiceMetricEvent.Builder createEventBuilder(String serviceName) - { - ServiceMetricEvent.Builder builder = ServiceMetricEvent.builder() - .setDimension("serviceName", serviceName); - MonitorUtils.addDimensionsToBuilder(builder, dimensions); - return builder; - } -} diff --git a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseRegister.java b/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseRegister.java deleted file mode 100644 index 66a022992e7..00000000000 --- a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseRegister.java +++ /dev/null @@ -1,56 +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.server.metrics; - -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.logger.Logger; - -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -public class EventReceiverFirehoseRegister -{ - - private static final Logger log = new Logger(EventReceiverFirehoseRegister.class); - - private final ConcurrentMap metrics = new ConcurrentHashMap<>(); - - public void register(String serviceName, EventReceiverFirehoseMetric metric) - { - log.info("Registering EventReceiverFirehoseMetric for service [%s]", serviceName); - if (metrics.putIfAbsent(serviceName, metric) != null) { - throw new ISE("Service [%s] is already registered!", serviceName); - } - } - - public Iterable> getMetrics() - { - return metrics.entrySet(); - } - - public void unregister(String serviceName) - { - log.info("Unregistering EventReceiverFirehoseMetric for service [%s]", serviceName); - if (metrics.remove(serviceName) == null) { - log.warn("Unregistering a non-exist service. Service [%s] never exists.", serviceName); - } - } -} diff --git a/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java b/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java index 278a170910a..ab2cdb3811f 100644 --- a/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java +++ b/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java @@ -91,7 +91,6 @@ public class MetricsModule implements Module binder.bind(DataSourceTaskIdHolder.class).in(LazySingleton.class); - binder.bind(EventReceiverFirehoseRegister.class).in(LazySingleton.class); binder.bind(ExecutorServiceMonitor.class).in(LazySingleton.class); // Instantiate eagerly so that we get everything registered and put into the Lifecycle diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java index f6d4a2b6e58..7742eaaf138 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java @@ -64,8 +64,7 @@ public class ClientCompactionRunnerInfoTest ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( - "Invalid partitionsSpec type[HashedPartitionsSpec] for MSQ engine." - + " Type must be either 'dynamic' or 'range'.", + "MSQ: Invalid partitioning type[HashedPartitionsSpec]. Must be either 'dynamic' or 'range'", validationResult.getReason() ); } @@ -85,7 +84,7 @@ public class ClientCompactionRunnerInfoTest ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( - "maxTotalRows[100] in DynamicPartitionsSpec not supported for MSQ engine.", + "MSQ: 'maxTotalRows' not supported with 'dynamic' partitioning", validationResult.getReason() ); } @@ -130,7 +129,7 @@ public class ClientCompactionRunnerInfoTest } @Test - public void testMSQEngineWithRollupFalseWithMetricsSpecIsInValid() + public void testMSQEngineWithRollupFalseWithMetricsSpecIsInvalid() { DataSourceCompactionConfig compactionConfig = createCompactionConfig( new DynamicPartitionsSpec(3, null), @@ -144,30 +143,7 @@ public class ClientCompactionRunnerInfoTest ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( - "rollup in granularitySpec must be set to True if metricsSpec is specifed for MSQ engine.", - validationResult.getReason() - ); - } - - @Test - public void testMSQEngineWithUnsupportedMetricsSpecIsInValid() - { - // Aggregators having different input and ouput column names are unsupported. - final String inputColName = "added"; - final String outputColName = "sum_added"; - DataSourceCompactionConfig compactionConfig = createCompactionConfig( - new DynamicPartitionsSpec(3, null), - Collections.emptyMap(), - new UserCompactionTaskGranularityConfig(null, null, null), - new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)} - ); - CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( - compactionConfig, - CompactionEngine.NATIVE - ); - Assert.assertFalse(validationResult.isValid()); - Assert.assertEquals( - "Different name[sum_added] and fieldName(s)[[added]] for aggregator unsupported for MSQ engine.", + "MSQ: 'granularitySpec.rollup' must be true if 'metricsSpec' is specified", validationResult.getReason() ); } diff --git a/server/src/test/java/org/apache/druid/curator/discovery/ServiceAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/discovery/ServiceAnnouncerTest.java index 57f7517239d..f80f35bdca8 100644 --- a/server/src/test/java/org/apache/druid/curator/discovery/ServiceAnnouncerTest.java +++ b/server/src/test/java/org/apache/druid/curator/discovery/ServiceAnnouncerTest.java @@ -19,7 +19,6 @@ package org.apache.druid.curator.discovery; -import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; import org.apache.curator.x.discovery.ServiceDiscovery; @@ -49,27 +48,21 @@ public class ServiceAnnouncerTest extends CuratorTestBase curator.blockUntilConnected(); List serviceNames = ImmutableList.of( "druid/overlord", - "druid/coordinator", - "druid/firehose/tranquility_test-50-0000-0000" + "druid/coordinator" ); final ServiceDiscovery serviceDiscovery = createAndAnnounceServices(serviceNames); Assert.assertTrue( Iterators.all( serviceNames.iterator(), - new Predicate() - { - @Override - public boolean apply(String input) - { - try { - return serviceDiscovery.queryForInstances(input.replace('/', ':')).size() == 1; - } - catch (Exception e) { - throw new ISE( - "Something went wrong while finding instance with name [%s] in Service Discovery", - input - ); - } + input -> { + try { + return serviceDiscovery.queryForInstances(input.replace('/', ':')).size() == 1; + } + catch (Exception e) { + throw new ISE( + "Something went wrong while finding instance with name [%s] in Service Discovery", + input + ); } } ) diff --git a/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java b/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java deleted file mode 100644 index 8ecc93dece2..00000000000 --- a/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java +++ /dev/null @@ -1,93 +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.guice; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.cfg.MapperConfig; -import com.fasterxml.jackson.databind.introspect.AnnotatedClass; -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.google.common.reflect.ClassPath; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.segment.realtime.firehose.ClippedFirehoseFactory; -import org.apache.druid.utils.JvmUtils; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.lang.reflect.Modifier; -import java.net.URL; -import java.net.URLClassLoader; -import java.util.Collection; -import java.util.Set; -import java.util.function.Predicate; -import java.util.stream.Collectors; - -public class FirehoseModuleTest -{ - private static final Predicate IS_FIREHOSE_FACTORY = - c -> FirehoseFactory.class.isAssignableFrom(c) && !Modifier.isAbstract(c.getModifiers()); - - @Test - public void testAllFirehoseFactorySubtypesRegistered() throws IOException - { - ObjectMapper objectMapper = createObjectMapper(); - Set registeredSubtypeClasses = getFirehoseFactorySubtypeClasses(objectMapper); - String packageName = ClippedFirehoseFactory.class.getPackage().getName(); - Set expectedSubtypeClasses = getFirehoseFactoryClassesInPackage(packageName); - Assert.assertEquals(expectedSubtypeClasses, registeredSubtypeClasses); - } - - private static ObjectMapper createObjectMapper() - { - ObjectMapper objectMapper = new ObjectMapper(); - for (Module jacksonModule : new FirehoseModule().getJacksonModules()) { - objectMapper.registerModule(jacksonModule); - } - return objectMapper; - } - - private static Set getFirehoseFactorySubtypeClasses(ObjectMapper objectMapper) - { - Class parentClass = FirehoseFactory.class; - MapperConfig config = objectMapper.getDeserializationConfig(); - AnnotatedClass ac = AnnotatedClass.constructWithoutSuperTypes(parentClass, config); - Collection subtypes = objectMapper.getSubtypeResolver().collectAndResolveSubtypesByClass(config, ac); - Assert.assertNotNull(subtypes); - return subtypes.stream() - .map(NamedType::getType) - .filter(c -> !c.equals(parentClass)) - .collect(Collectors.toSet()); - } - - @SuppressWarnings("UnstableApiUsage") // for ClassPath - private static Set getFirehoseFactoryClassesInPackage(String packageName) throws IOException - { - // workaround for Guava 16, which can only parse the classpath from URLClassLoaders - // requires Guava 28 or later to work properly with the system class loader in Java 9 and above - URLClassLoader classloader = new URLClassLoader(JvmUtils.systemClassPath().toArray(new URL[0])); - ClassPath classPath = ClassPath.from(classloader); - return classPath.getTopLevelClasses(packageName).stream() - .map(ClassPath.ClassInfo::load) - .filter(IS_FIREHOSE_FACTORY) - .collect(Collectors.toSet()); - } -} - diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java b/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java index ccd71cfaff5..4053ec9ecf2 100644 --- a/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java @@ -68,7 +68,7 @@ public class SqlEntityTest File tmpFile = File.createTempFile("testQueryResults", ""); InputEntity.CleanableFile queryResult = SqlEntity.openCleanableFile( VALID_SQL, - testUtils.getDerbyFirehoseConnector(), + testUtils.getDerbyInputSourceConnector(), mapper, true, tmpFile @@ -95,7 +95,7 @@ public class SqlEntityTest IOException.class, () -> SqlEntity.openCleanableFile( INVALID_SQL, - testUtils.getDerbyFirehoseConnector(), + testUtils.getDerbyInputSourceConnector(), mapper, true, tmpFile diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java b/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java index 7a5ea7b2149..c48dfd83946 100644 --- a/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java @@ -41,7 +41,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.metadata.MetadataStorageConnectorConfig; -import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.SQLInputSourceDatabaseConnector; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.initialization.JdbcAccessSecurityConfig; @@ -66,7 +66,7 @@ import java.util.stream.Stream; public class SqlInputSourceTest { - private static final List FIREHOSE_TMP_DIRS = new ArrayList<>(); + private static final List INPUT_SOURCE_TMP_DIRS = new ArrayList<>(); private final String TABLE_1 = "FOOS_TABLE_1"; private final String TABLE_2 = "FOOS_TABLE_2"; @@ -94,31 +94,31 @@ public class SqlInputSourceTest @AfterClass public static void teardown() throws IOException { - for (File dir : FIREHOSE_TMP_DIRS) { + for (File dir : INPUT_SOURCE_TMP_DIRS) { org.apache.commons.io.FileUtils.forceDelete(dir); } } - private File createFirehoseTmpDir(String dirSuffix) throws IOException + private File createInputSourceTmpDir(String dirSuffix) throws IOException { - final File firehoseTempDir = File.createTempFile( + final File inputSourceTempDir = File.createTempFile( SqlInputSourceTest.class.getSimpleName(), dirSuffix ); - org.apache.commons.io.FileUtils.forceDelete(firehoseTempDir); - FileUtils.mkdirp(firehoseTempDir); - FIREHOSE_TMP_DIRS.add(firehoseTempDir); - return firehoseTempDir; + org.apache.commons.io.FileUtils.forceDelete(inputSourceTempDir); + FileUtils.mkdirp(inputSourceTempDir); + INPUT_SOURCE_TMP_DIRS.add(inputSourceTempDir); + return inputSourceTempDir; } @Test public void testSerde() throws IOException { - mapper.registerSubtypes(TestSerdeFirehoseConnector.class); - final SqlInputSourceTest.TestSerdeFirehoseConnector testSerdeFirehoseConnector = new SqlInputSourceTest.TestSerdeFirehoseConnector( + mapper.registerSubtypes(TestSerdeInputSourceConnector.class); + final TestSerdeInputSourceConnector serdeInputSourceConnector = new TestSerdeInputSourceConnector( new MetadataStorageConnectorConfig()); final SqlInputSource sqlInputSource = - new SqlInputSource(SqlTestUtils.selectFrom(TABLE_1), true, testSerdeFirehoseConnector, mapper); + new SqlInputSource(SqlTestUtils.selectFrom(TABLE_1), true, serdeInputSourceConnector, mapper); final String valueString = mapper.writeValueAsString(sqlInputSource); final SqlInputSource inputSourceFromJson = mapper.readValue(valueString, SqlInputSource.class); Assert.assertEquals(sqlInputSource, inputSourceFromJson); @@ -127,11 +127,11 @@ public class SqlInputSourceTest @Test public void testGetTypes() { - mapper.registerSubtypes(TestSerdeFirehoseConnector.class); - final SqlInputSourceTest.TestSerdeFirehoseConnector testSerdeFirehoseConnector = new SqlInputSourceTest.TestSerdeFirehoseConnector( + mapper.registerSubtypes(TestSerdeInputSourceConnector.class); + final TestSerdeInputSourceConnector serdeInputSourceConnector = new TestSerdeInputSourceConnector( new MetadataStorageConnectorConfig()); final SqlInputSource sqlInputSource = - new SqlInputSource(SqlTestUtils.selectFrom(TABLE_1), true, testSerdeFirehoseConnector, mapper); + new SqlInputSource(SqlTestUtils.selectFrom(TABLE_1), true, serdeInputSourceConnector, mapper); Assert.assertEquals(Collections.singleton(SqlInputSource.TYPE_KEY), sqlInputSource.getTypes()); } @@ -141,13 +141,13 @@ public class SqlInputSourceTest derbyConnector = derbyConnectorRule.getConnector(); SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); final List expectedRows = testUtils.createTableWithRows(TABLE_1, 10); - final File tempDir = createFirehoseTmpDir("testSingleSplit"); + final File tempDir = createInputSourceTmpDir("testSingleSplit"); final InputStats inputStats = new InputStatsImpl(); SqlInputSource sqlInputSource = new SqlInputSource( SqlTestUtils.selectFrom(TABLE_1), true, - testUtils.getDerbyFirehoseConnector(), + testUtils.getDerbyInputSourceConnector(), mapper ); InputSourceReader sqlReader = sqlInputSource.fixedFormatReader(INPUT_ROW_SCHEMA, tempDir); @@ -170,11 +170,11 @@ public class SqlInputSourceTest SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); final List expectedRowsTable1 = testUtils.createTableWithRows(TABLE_1, 10); final List expectedRowsTable2 = testUtils.createTableWithRows(TABLE_2, 10); - final File tempDir = createFirehoseTmpDir("testMultipleSplit"); + final File tempDir = createInputSourceTmpDir("testMultipleSplit"); SqlInputSource sqlInputSource = new SqlInputSource( SqlTestUtils.selectFrom(TABLE_1, TABLE_2), true, - testUtils.getDerbyFirehoseConnector(), + testUtils.getDerbyInputSourceConnector(), mapper ); @@ -198,7 +198,7 @@ public class SqlInputSourceTest SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); final List sqls = SqlTestUtils.selectFrom(TABLE_1, TABLE_2); SqlInputSource sqlInputSource = - new SqlInputSource(sqls, true, testUtils.getDerbyFirehoseConnector(), mapper); + new SqlInputSource(sqls, true, testUtils.getDerbyInputSourceConnector(), mapper); InputFormat inputFormat = EasyMock.createMock(InputFormat.class); Stream> sqlSplits = sqlInputSource.createSplits(inputFormat, null); Assert.assertEquals(sqls, sqlSplits.map(InputSplit::get).collect(Collectors.toList())); @@ -212,9 +212,9 @@ public class SqlInputSourceTest SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); final List expectedRows = testUtils.createTableWithRows(TABLE_1, 10); try { - final File tempDir = createFirehoseTmpDir("testSingleSplit"); + final File tempDir = createInputSourceTmpDir("testSingleSplit"); SqlInputSource sqlInputSource = - new SqlInputSource(SqlTestUtils.selectFrom(TABLE_1), true, testUtils.getDerbyFirehoseConnector(), mapper); + new SqlInputSource(SqlTestUtils.selectFrom(TABLE_1), true, testUtils.getDerbyInputSourceConnector(), mapper); InputSourceReader sqlReader = sqlInputSource.fixedFormatReader(INPUT_ROW_SCHEMA, tempDir); CloseableIterator resultIterator = sqlReader.sample(); final List rows = new ArrayList<>(); @@ -230,6 +230,64 @@ public class SqlInputSourceTest } } + @Test + public void testConnectorValidationInvalidUri() + { + derbyConnector = derbyConnectorRule.getConnector(); + Throwable t = Assert.assertThrows( + IllegalArgumentException.class, + () -> new SqlTestUtils( + derbyConnector, + new MetadataStorageConnectorConfig() + { + @Override + public String getConnectURI() + { + return ""; + } + } + ) + ); + Assert.assertEquals("connectURI cannot be null or empty", t.getMessage()); + } + + @Test + public void testConnectorValidationAllowedProperties() + { + derbyConnector = derbyConnectorRule.getConnector(); + Throwable t = Assert.assertThrows( + IllegalArgumentException.class, + () -> new SqlTestUtils( + derbyConnector, + new MetadataStorageConnectorConfig(), + new JdbcAccessSecurityConfig() + ) + ); + Assert.assertEquals( + "The property [user] is not in the allowed list [useSSL, requireSSL, ssl, sslmode]", + t.getMessage() + ); + } + + @Test + public void testConnectorValidationSkipAllowedProperties() + { + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils( + derbyConnector, + new MetadataStorageConnectorConfig(), + new JdbcAccessSecurityConfig() + { + @Override + public boolean isEnforceAllowedProperties() + { + return false; + } + } + ); + Assert.assertNotNull(testUtils); + } + @Test public void testEquals() { @@ -240,18 +298,19 @@ public class SqlInputSourceTest new ObjectMapper() ) .withIgnoredFields("objectMapper") - .withNonnullFields("sqls", "sqlFirehoseDatabaseConnector") + .withNonnullFields("sqls", "sqlInputSourceDatabaseConnector") .usingGetClass() .verify(); } + @JsonTypeName("test") - private static class TestSerdeFirehoseConnector extends SQLFirehoseDatabaseConnector + private static class TestSerdeInputSourceConnector extends SQLInputSourceDatabaseConnector { private final DBI dbi; private final MetadataStorageConnectorConfig metadataStorageConnectorConfig; - private TestSerdeFirehoseConnector( + private TestSerdeInputSourceConnector( @JsonProperty("connectorConfig") MetadataStorageConnectorConfig metadataStorageConnectorConfig ) { @@ -287,7 +346,7 @@ public class SqlInputSourceTest if (o == null || getClass() != o.getClass()) { return false; } - TestSerdeFirehoseConnector that = (TestSerdeFirehoseConnector) o; + TestSerdeInputSourceConnector that = (TestSerdeInputSourceConnector) o; return metadataStorageConnectorConfig.equals(that.metadataStorageConnectorConfig); } diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java b/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java index 2e99bfbb301..f999e6dd41c 100644 --- a/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java @@ -28,7 +28,7 @@ import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.metadata.MetadataStorageConnectorConfig; -import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.SQLInputSourceDatabaseConnector; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.server.initialization.JdbcAccessSecurityConfig; import org.junit.Rule; @@ -49,24 +49,48 @@ public class SqlTestUtils { @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private final TestDerbyFirehoseConnector derbyFirehoseConnector; + private final TestDerbyInputSourceConnector derbyInputSourceConnector; private final TestDerbyConnector derbyConnector; public SqlTestUtils(TestDerbyConnector derbyConnector) { this.derbyConnector = derbyConnector; - this.derbyFirehoseConnector = new SqlTestUtils.TestDerbyFirehoseConnector( + this.derbyInputSourceConnector = new TestDerbyInputSourceConnector( new MetadataStorageConnectorConfig(), derbyConnector.getDBI() ); } - private static class TestDerbyFirehoseConnector extends SQLFirehoseDatabaseConnector + public SqlTestUtils(TestDerbyConnector derbyConnector, MetadataStorageConnectorConfig config) + { + this.derbyConnector = derbyConnector; + this.derbyInputSourceConnector = new TestDerbyInputSourceConnector( + config, + derbyConnector.getDBI() + ); + } + + public SqlTestUtils( + TestDerbyConnector derbyConnector, + MetadataStorageConnectorConfig config, + JdbcAccessSecurityConfig securityConfig + ) + { + this.derbyConnector = derbyConnector; + this.derbyInputSourceConnector = new TestDerbyInputSourceConnector( + config, + securityConfig, + derbyConnector.getDBI() + ); + } + + private static class TestDerbyInputSourceConnector extends SQLInputSourceDatabaseConnector { private final DBI dbi; - private TestDerbyFirehoseConnector( - @JsonProperty("connectorConfig") MetadataStorageConnectorConfig metadataStorageConnectorConfig, DBI dbi + private TestDerbyInputSourceConnector( + @JsonProperty("connectorConfig") MetadataStorageConnectorConfig metadataStorageConnectorConfig, + DBI dbi ) { final BasicDataSource datasource = getDatasource( @@ -85,6 +109,21 @@ public class SqlTestUtils this.dbi = dbi; } + private TestDerbyInputSourceConnector( + MetadataStorageConnectorConfig metadataStorageConnectorConfig, + JdbcAccessSecurityConfig securityConfig, + DBI dbi + ) + { + final BasicDataSource datasource = getDatasource( + metadataStorageConnectorConfig, + securityConfig + ); + datasource.setDriverClassLoader(getClass().getClassLoader()); + datasource.setDriverClassName("org.apache.derby.jdbc.ClientDriver"); + this.dbi = dbi; + } + @Override public DBI getDBI() { @@ -151,9 +190,9 @@ public class SqlTestUtils ); } - public TestDerbyFirehoseConnector getDerbyFirehoseConnector() + public TestDerbyInputSourceConnector getDerbyInputSourceConnector() { - return derbyFirehoseConnector; + return derbyInputSourceConnector; } /** diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java index 8c3b867e368..5f583746266 100644 --- a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java @@ -225,7 +225,7 @@ public class OverlordClientImplTest final Map> lockMap = ImmutableMap.of("foo", Collections.singletonList(Intervals.of("2000/2001"))); final List requests = ImmutableList.of( - new LockFilterPolicy("foo", 3, null) + new LockFilterPolicy("foo", 3, null, null) ); serviceClient.expectAndRespond( @@ -246,7 +246,7 @@ public class OverlordClientImplTest public void test_findLockedIntervals_nullReturn() throws Exception { final List requests = ImmutableList.of( - new LockFilterPolicy("foo", 3, null) + new LockFilterPolicy("foo", 3, null, null) ); serviceClient.expectAndRespond( diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 78294fca0c4..87ddac50f01 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -684,4 +684,24 @@ public class DataSchemaTest extends InitializedNullHandlingTest Assert.assertSame(oldSchema.getParserMap(), newSchema.getParserMap()); } + + @Test + public void testCombinedDataSchemaSetsHasRolledUpSegments() + { + CombinedDataSchema schema = new CombinedDataSchema( + IdUtilsTest.VALID_ID_CHARS, + new TimestampSpec("time", "auto", null), + DimensionsSpec.builder() + .setDimensions( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dimA", "dimB", "metric1")) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .build(), + null, + new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + null, + true + ); + Assert.assertTrue(schema.hasRolledUpSegments()); + } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResourceTest.java b/server/src/test/java/org/apache/druid/segment/realtime/ChatHandlerResourceTest.java similarity index 97% rename from server/src/test/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResourceTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/ChatHandlerResourceTest.java index 870636fb416..21d99c62225 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResourceTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/ChatHandlerResourceTest.java @@ -18,7 +18,7 @@ */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; import com.google.common.base.Optional; import org.apache.druid.server.initialization.jetty.ServiceUnavailableException; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java b/server/src/test/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProviderTest.java similarity index 99% rename from server/src/test/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProviderTest.java index 71a3fe308f4..05fb11e4b62 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProviderTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; import org.apache.druid.curator.discovery.ServiceAnnouncer; import org.apache.druid.server.DruidNode; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java deleted file mode 100644 index 66b2281d33c..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java +++ /dev/null @@ -1,245 +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.segment.realtime.appenderator; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.core.NoopEmitter; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMerger; -import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.incremental.ParseExceptionHandler; -import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.loading.NoopDataSegmentPusher; -import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.SegmentGenerationMetrics; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.apache.druid.timeline.partition.LinearShardSpec; -import org.junit.Assert; -import org.junit.Test; - -import javax.annotation.Nullable; -import java.io.File; -import java.util.Map; - - -public class AppenderatorsTest -{ - @Test - public void testOpenSegmentsOfflineAppenderator() throws Exception - { - try (final AppenderatorTester tester = new AppenderatorTester("OPEN_SEGMENTS")) { - Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); - AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; - Assert.assertTrue(appenderator.isOpenSegments()); - } - } - - @Test - public void testClosedSegmentsOfflineAppenderator() throws Exception - { - try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS")) { - Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); - AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; - Assert.assertFalse(appenderator.isOpenSegments()); - } - } - - @Test - public void testClosedSegmentsSinksOfflineAppenderator() throws Exception - { - try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS_SINKS")) { - Assert.assertTrue(tester.appenderator instanceof BatchAppenderator); - } - } - - private static class AppenderatorTester implements AutoCloseable - { - public static final String DATASOURCE = "foo"; - - private final AppenderatorConfig tuningConfig; - private final Appenderator appenderator; - private final ServiceEmitter emitter; - - public AppenderatorTester(final String batchMode) - { - this(100, 100, null, new SimpleRowIngestionMeters(), false, batchMode); - } - - public AppenderatorTester( - final int maxRowsInMemory, - final long maxSizeInBytes, - @Nullable final File basePersistDirectory, - final RowIngestionMeters rowIngestionMeters, - final boolean skipBytesInMemoryOverheadCheck, - String batchMode - ) - { - ObjectMapper objectMapper = new DefaultObjectMapper(); - objectMapper.registerSubtypes(LinearShardSpec.class); - - final Map parserMap = objectMapper.convertValue( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec("ts", "auto", null), - DimensionsSpec.EMPTY, - null, - null, - null - ) - ), - Map.class - ); - - DataSchema schema = new DataSchema( - DATASOURCE, - null, - null, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null, - parserMap, - objectMapper - ); - - tuningConfig = new TestAppenderatorConfig( - TuningConfig.DEFAULT_APPENDABLE_INDEX, - maxRowsInMemory, - maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, - skipBytesInMemoryOverheadCheck, - IndexSpec.DEFAULT, - 0, - false, - 0L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, - basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory - ); - SegmentGenerationMetrics metrics = new SegmentGenerationMetrics(); - - IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); - IndexMergerV9 indexMerger = new IndexMergerV9( - objectMapper, - indexIO, - OffHeapMemorySegmentWriteOutMediumFactory.instance() - ); - - emitter = new ServiceEmitter( - "test", - "test", - new NoopEmitter() - ); - emitter.start(); - EmittingLogger.registerEmitter(emitter); - - switch (batchMode) { - case "OPEN_SEGMENTS": - appenderator = Appenderators.createOpenSegmentsOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - new NoopDataSegmentPusher(), - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false, - CentralizedDatasourceSchemaConfig.create() - ); - break; - case "CLOSED_SEGMENTS": - appenderator = Appenderators.createClosedSegmentsOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - new NoopDataSegmentPusher(), - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false, - CentralizedDatasourceSchemaConfig.create() - ); - - break; - case "CLOSED_SEGMENTS_SINKS": - appenderator = Appenderators.createOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - new NoopDataSegmentPusher(), - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false, - CentralizedDatasourceSchemaConfig.create() - ); - break; - default: - throw new IllegalArgumentException("Unrecognized batchMode: " + batchMode); - } - } - - private long getDefaultMaxBytesInMemory() - { - return (Runtime.getRuntime().totalMemory()) / 3; - } - - @Override - public void close() throws Exception - { - appenderator.close(); - emitter.close(); - FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory()); - } - - private static File createNewBasePersistDirectory() - { - return FileUtils.createTempDir("druid-batch-persist"); - } - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java similarity index 97% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java index 269aeaca7c4..ed63bca3195 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java @@ -52,7 +52,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; -public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupport +public class BatchAppenderatorDriverTest extends EasyMockSupport { private static final String DATA_SOURCE = "foo"; private static final String VERSION = "abc123"; @@ -78,7 +78,7 @@ public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupp ); private SegmentAllocator allocator; - private ClosedSegmensSinksBatchAppenderatorTester appenderatorTester; + private BatchAppenderatorTester appenderatorTester; private BatchAppenderatorDriver driver; private DataSegmentKiller dataSegmentKiller; @@ -89,7 +89,7 @@ public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupp @Before public void setup() { - appenderatorTester = new ClosedSegmensSinksBatchAppenderatorTester(MAX_ROWS_IN_MEMORY); + appenderatorTester = new BatchAppenderatorTester(MAX_ROWS_IN_MEMORY); allocator = new TestSegmentAllocator(DATA_SOURCE, Granularities.HOUR); dataSegmentKiller = createStrictMock(DataSegmentKiller.class); driver = new BatchAppenderatorDriver( diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java similarity index 91% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java index 55b5f235506..8b9117705f5 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java @@ -49,7 +49,7 @@ import java.util.List; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; -public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHandlingTest +public class BatchAppenderatorTest extends InitializedNullHandlingTest { private static final List IDENTIFIERS = ImmutableList.of( createSegmentId("2000/2001", "A", 0), // should be in seg_0 @@ -60,14 +60,14 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testSimpleIngestion() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); // startJob Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // add #1 Assert.assertEquals( @@ -136,14 +136,14 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testPushFailure() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, true)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); // startJob Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // add #1 Assert.assertEquals( @@ -219,14 +219,14 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testPeriodGranularityNonUTCIngestion() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(1, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(1, false)) { final Appenderator appenderator = tester.getAppenderator(); // startJob Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // Create a segment identifier with a non-utc interval SegmentIdWithShardSpec segmentIdWithNonUTCTime = @@ -281,7 +281,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( 100, 1024, null, @@ -316,7 +316,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( 100, 1024, null, @@ -346,7 +346,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testMaxBytesInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(100, 15000, true)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(100, 15000, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -425,8 +425,8 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test(expected = RuntimeException.class, timeout = 5000L) public void testTaskFailAsPersistCannotFreeAnyMoreMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(100, 5180, true)) { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(100, 5180, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); @@ -437,7 +437,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( 100, 10, null, @@ -468,8 +468,8 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(100, 10000, true)) { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(100, 10000, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -495,8 +495,8 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testMaxBytesInMemoryInMultipleSinks() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(1000, 28748, true)) { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(1000, 28748, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -616,8 +616,8 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testIgnoreMaxBytesInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(100, -1, true)) { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(100, -1, true)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -649,7 +649,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testMaxRowsInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -682,7 +682,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testAllHydrantsAreRecovered() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(1, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(1, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -715,7 +715,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testTotalRowsPerSegment() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -775,7 +775,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testRestoreFromDisk() throws Exception { - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(2, false); + final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, false); final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -814,7 +814,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testCleanupFromDiskAfterClose() throws Exception { - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(2, false); + final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, false); final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -854,7 +854,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test(timeout = 5000L) public void testTotalRowCount() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, appenderator.getTotalRowCount()); @@ -895,10 +895,10 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testVerifyRowIngestionMetrics() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(5, - 10000L, - null, false, rowIngestionMeters + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(5, + 10000L, + null, false, rowIngestionMeters )) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -918,10 +918,10 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testPushContract() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(1, - 50000L, - null, false, rowIngestionMeters + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(1, + 50000L, + null, false, rowIngestionMeters )) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -967,10 +967,10 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testCloseContract() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(1, - 50000L, - null, false, rowIngestionMeters + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(1, + 50000L, + null, false, rowIngestionMeters )) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -1018,7 +1018,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan private static SegmentIdWithShardSpec createNonUTCSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( - ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, + BatchAppenderatorTester.DATASOURCE, new Interval(interval, ISOChronology.getInstance(DateTimes.inferTzFromString("Asia/Seoul"))), version, new LinearShardSpec(partitionNum) @@ -1029,7 +1029,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan private static SegmentIdWithShardSpec createSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( - ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, + BatchAppenderatorTester.DATASOURCE, Intervals.of(interval), version, new LinearShardSpec(partitionNum) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java similarity index 95% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java index cf2d7f79898..22034aa33aa 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -59,7 +59,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; -public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable +public class BatchAppenderatorTester implements AutoCloseable { public static final String DATASOURCE = "foo"; @@ -72,14 +72,14 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable private final List pushedSegments = new CopyOnWriteArrayList<>(); - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory ) { this(maxRowsInMemory, -1, null, false); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final boolean enablePushFailure ) @@ -87,7 +87,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable this(maxRowsInMemory, -1, null, enablePushFailure); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final boolean enablePushFailure @@ -96,7 +96,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable this(maxRowsInMemory, maxSizeInBytes, null, enablePushFailure); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final File basePersistDirectory, @@ -113,7 +113,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable ); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, @Nullable final File basePersistDirectory, @@ -126,7 +126,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable ); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, @Nullable final File basePersistDirectory, @@ -230,7 +230,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable throw new UnsupportedOperationException(); } }; - appenderator = Appenderators.createOffline( + appenderator = Appenderators.createBatch( schema.getDataSource(), schema, tuningConfig, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java deleted file mode 100644 index 33a0ed2f8a4..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java +++ /dev/null @@ -1,289 +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.segment.realtime.appenderator; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.core.NoopEmitter; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMerger; -import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.incremental.ParseExceptionHandler; -import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.SegmentGenerationMetrics; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.LinearShardSpec; - -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CopyOnWriteArrayList; - -public class OpenAndClosedSegmentsAppenderatorTester implements AutoCloseable -{ - public static final String DATASOURCE = "foo"; - - private final DataSchema schema; - private final SegmentGenerationMetrics metrics; - private final DataSegmentPusher dataSegmentPusher; - private final ObjectMapper objectMapper; - private final Appenderator appenderator; - private final IndexIO indexIO; - private final IndexMergerV9 indexMerger; - private final ServiceEmitter emitter; - private final AppenderatorConfig tuningConfig; - - - private final List pushedSegments = new CopyOnWriteArrayList<>(); - - public OpenAndClosedSegmentsAppenderatorTester( - final int maxRowsInMemory, - final boolean enablePushFailure, - boolean batchMemoryMappedIndex - ) - { - this(maxRowsInMemory, -1, null, enablePushFailure, batchMemoryMappedIndex); - } - - public OpenAndClosedSegmentsAppenderatorTester( - final int maxRowsInMemory, - final long maxSizeInBytes, - final File basePersistDirectory, - final boolean enablePushFailure, - boolean batchMemoryMappedIndex - ) - { - this( - maxRowsInMemory, - maxSizeInBytes, - basePersistDirectory, - enablePushFailure, - new SimpleRowIngestionMeters(), - false, - batchMemoryMappedIndex - ); - } - - public OpenAndClosedSegmentsAppenderatorTester( - final int maxRowsInMemory, - final long maxSizeInBytes, - final File basePersistDirectory, - final boolean enablePushFailure, - final RowIngestionMeters rowIngestionMeters, - final boolean skipBytesInMemoryOverheadCheck, - boolean batchMemoryMappedIndex - ) - { - objectMapper = new DefaultObjectMapper(); - objectMapper.registerSubtypes(LinearShardSpec.class); - - final Map parserMap = objectMapper.convertValue( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec("ts", "auto", null), - DimensionsSpec.EMPTY, - null, - null, - null - ) - ), - Map.class - ); - schema = new DataSchema( - DATASOURCE, - parserMap, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null, - objectMapper - ); - tuningConfig = new TestAppenderatorConfig( - TuningConfig.DEFAULT_APPENDABLE_INDEX, - maxRowsInMemory, - maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, - skipBytesInMemoryOverheadCheck, - IndexSpec.DEFAULT, - 0, - false, - 0L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, - basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory - ); - - metrics = new SegmentGenerationMetrics(); - - indexIO = new IndexIO( - objectMapper, - new ColumnConfig() - { - } - ); - indexMerger = new IndexMergerV9(objectMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); - - emitter = new ServiceEmitter( - "test", - "test", - new NoopEmitter() - ); - emitter.start(); - EmittingLogger.registerEmitter(emitter); - dataSegmentPusher = new DataSegmentPusher() - { - @Deprecated - @Override - public String getPathForHadoop(String dataSource) - { - return getPathForHadoop(); - } - - @Override - public String getPathForHadoop() - { - throw new UnsupportedOperationException(); - } - - @Override - public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException - { - if (enablePushFailure) { - throw new IOException("Push failure test"); - } - pushedSegments.add(segment); - return segment; - } - - @Override - public Map makeLoadSpec(URI uri) - { - throw new UnsupportedOperationException(); - } - }; - if (batchMemoryMappedIndex) { - appenderator = Appenderators.createOpenSegmentsOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - true, - CentralizedDatasourceSchemaConfig.create() - ); - } else { - appenderator = Appenderators.createClosedSegmentsOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - true, - CentralizedDatasourceSchemaConfig.create() - ); - } - } - - private long getDefaultMaxBytesInMemory() - { - return (Runtime.getRuntime().totalMemory()) / 3; - } - - public DataSchema getSchema() - { - return schema; - } - - public AppenderatorConfig getTuningConfig() - { - return tuningConfig; - } - - public SegmentGenerationMetrics getMetrics() - { - return metrics; - } - - public DataSegmentPusher getDataSegmentPusher() - { - return dataSegmentPusher; - } - - public ObjectMapper getObjectMapper() - { - return objectMapper; - } - - public Appenderator getAppenderator() - { - return appenderator; - } - - public List getPushedSegments() - { - return pushedSegments; - } - - @Override - public void close() throws Exception - { - appenderator.close(); - emitter.close(); - FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory()); - } - - private static File createNewBasePersistDirectory() - { - return FileUtils.createTempDir("druid-batch-persist"); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java deleted file mode 100644 index 0c6fb552a4d..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java +++ /dev/null @@ -1,205 +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.segment.realtime.appenderator; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.segment.loading.DataSegmentKiller; -import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver.SegmentsForSequence; -import org.apache.druid.segment.realtime.appenderator.SegmentWithState.SegmentState; -import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriverTest.TestSegmentAllocator; -import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.easymock.EasyMock; -import org.easymock.EasyMockSupport; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; -import java.util.stream.Collectors; - -public class OpenAndClosedSegmentsBatchAppenderatorDriverTest extends EasyMockSupport -{ - private static final String DATA_SOURCE = "foo"; - private static final String VERSION = "abc123"; - private static final int MAX_ROWS_IN_MEMORY = 100; - private static final long TIMEOUT = 1000; - - private static final List ROWS = Arrays.asList( - new MapBasedInputRow( - DateTimes.of("2000"), - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo", "met1", "1") - ), - new MapBasedInputRow( - DateTimes.of("2000T01"), - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo", "met1", 2.0) - ), - new MapBasedInputRow( - DateTimes.of("2000T01"), - ImmutableList.of("dim2"), - ImmutableMap.of("dim2", "bar", "met1", 2.0) - ) - ); - - private SegmentAllocator allocator; - private OpenAndClosedSegmentsAppenderatorTester openAndClosedSegmentsAppenderatorTester; - private BatchAppenderatorDriver driver; - private DataSegmentKiller dataSegmentKiller; - - static { - NullHandling.initializeForTests(); - } - - @Before - public void setup() - { - openAndClosedSegmentsAppenderatorTester = - new OpenAndClosedSegmentsAppenderatorTester(MAX_ROWS_IN_MEMORY, false, - false - ); - allocator = new TestSegmentAllocator(DATA_SOURCE, Granularities.HOUR); - dataSegmentKiller = createStrictMock(DataSegmentKiller.class); - driver = new BatchAppenderatorDriver( - openAndClosedSegmentsAppenderatorTester.getAppenderator(), - allocator, - new TestPublishedSegmentRetriever(openAndClosedSegmentsAppenderatorTester.getPushedSegments()), - dataSegmentKiller - ); - - EasyMock.replay(dataSegmentKiller); - } - - @After - public void tearDown() throws Exception - { - EasyMock.verify(dataSegmentKiller); - - driver.clear(); - driver.close(); - } - - @Test (timeout = 2000L) - public void testSimple() throws Exception - { - Assert.assertNull(driver.startJob(null)); - - for (InputRow row : ROWS) { - Assert.assertTrue(driver.add(row, "dummy").isOk()); - } - - checkSegmentStates(2, SegmentState.APPENDING); - - driver.pushAllAndClear(TIMEOUT); - - checkSegmentStates(2, SegmentState.PUSHED_AND_DROPPED); - - final SegmentsAndCommitMetadata published = - driver.publishAll(null, null, makeOkPublisher(), Function.identity(), null).get(TIMEOUT, TimeUnit.MILLISECONDS); - - Assert.assertEquals( - ImmutableSet.of( - new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000/PT1H"), VERSION, new NumberedShardSpec(0, 0)), - new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000T01/PT1H"), VERSION, new NumberedShardSpec(0, 0)) - ), - published.getSegments() - .stream() - .map(SegmentIdWithShardSpec::fromDataSegment) - .collect(Collectors.toSet()) - ); - - Assert.assertNull(published.getCommitMetadata()); - } - - @Test(timeout = 5000L) - public void testIncrementalPush() throws Exception - { - Assert.assertNull(driver.startJob(null)); - - int i = 0; - for (InputRow row : ROWS) { - Assert.assertTrue(driver.add(row, "dummy").isOk()); - - checkSegmentStates(1, SegmentState.APPENDING); - checkSegmentStates(i, SegmentState.PUSHED_AND_DROPPED); - - driver.pushAllAndClear(TIMEOUT); - checkSegmentStates(0, SegmentState.APPENDING); - checkSegmentStates(++i, SegmentState.PUSHED_AND_DROPPED); - } - - final SegmentsAndCommitMetadata published = - driver.publishAll(null, null, makeOkPublisher(), Function.identity(), null).get(TIMEOUT, TimeUnit.MILLISECONDS); - - Assert.assertEquals( - ImmutableSet.of( - new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000/PT1H"), VERSION, new NumberedShardSpec(0, 0)), - new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000T01/PT1H"), VERSION, new NumberedShardSpec(0, 0)), - new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000T01/PT1H"), VERSION, new NumberedShardSpec(1, 0)) - ), - published.getSegments() - .stream() - .map(SegmentIdWithShardSpec::fromDataSegment) - .collect(Collectors.toSet()) - ); - - Assert.assertNull(published.getCommitMetadata()); - } - - @Test - public void testRestart() - { - Assert.assertNull(driver.startJob(null)); - driver.close(); - openAndClosedSegmentsAppenderatorTester.getAppenderator().close(); - - Assert.assertNull(driver.startJob(null)); - } - - private void checkSegmentStates(int expectedNumSegmentsInState, SegmentState expectedState) - { - final SegmentsForSequence segmentsForSequence = driver.getSegments().get("dummy"); - Assert.assertNotNull(segmentsForSequence); - final List segmentWithStates = segmentsForSequence - .allSegmentStateStream() - .filter(segmentWithState -> segmentWithState.getState() == expectedState) - .collect(Collectors.toList()); - - Assert.assertEquals(expectedNumSegmentsInState, segmentWithStates.size()); - } - - static TransactionalSegmentPublisher makeOkPublisher() - { - return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata, schema) -> SegmentPublishResult.ok(ImmutableSet.of()); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java deleted file mode 100644 index 2f5e5cde7ed..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java +++ /dev/null @@ -1,228 +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.segment.realtime.appenderator; - -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.testing.InitializedNullHandlingTest; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.LinearShardSpec; -import org.junit.Assert; -import org.junit.Test; - -import java.util.List; -import java.util.stream.Collectors; - -public class OpenAndClosedSegmentsBatchAppenderatorTest extends InitializedNullHandlingTest -{ - private static final List IDENTIFIERS = ImmutableList.of( - createSegmentId("2000/2001", "A", 0), - createSegmentId("2000/2001", "A", 1), - createSegmentId("2001/2002", "A", 0) - ); - - @Test - public void testSimpleIngestionWithIndexesNotMapped() throws Exception - { - try (final OpenAndClosedSegmentsAppenderatorTester tester = - new OpenAndClosedSegmentsAppenderatorTester(2, - false, - false)) { - final Appenderator appenderator = tester.getAppenderator(); - boolean thrown; - - // startJob - Assert.assertEquals(null, appenderator.startJob()); - - // getDataSource - Assert.assertEquals(OpenAndClosedSegmentsAppenderatorTester.DATASOURCE, appenderator.getDataSource()); - - // add - Assert.assertEquals( - 1, - appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null) - .getNumRowsInSegment() - ); - - Assert.assertEquals( - 2, - appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 2), null) - .getNumRowsInSegment() - ); - - Assert.assertEquals( - 1, - appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 4), null) - .getNumRowsInSegment() - ); - - // getSegments - Assert.assertEquals(IDENTIFIERS.subList(0, 2), - appenderator.getSegments().stream().sorted().collect(Collectors.toList())); - - // getRowCount - Assert.assertEquals(2, appenderator.getRowCount(IDENTIFIERS.get(0))); - Assert.assertEquals(1, appenderator.getRowCount(IDENTIFIERS.get(1))); - thrown = false; - try { - appenderator.getRowCount(IDENTIFIERS.get(2)); - } - catch (IllegalStateException e) { - thrown = true; - } - Assert.assertTrue(thrown); - - // push all - final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( - appenderator.getSegments(), - null, - false - ).get(); - Assert.assertEquals( - IDENTIFIERS.subList(0, 2), - Lists.transform( - segmentsAndCommitMetadata.getSegments(), - new Function() - { - @Override - public SegmentIdWithShardSpec apply(DataSegment input) - { - return SegmentIdWithShardSpec.fromDataSegment(input); - } - } - ).stream().sorted().collect(Collectors.toList()) - ); - Assert.assertEquals(tester.getPushedSegments().stream().sorted().collect(Collectors.toList()), - segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList())); - - appenderator.clear(); - Assert.assertTrue(appenderator.getSegments().isEmpty()); - } - } - - @Test - public void testSimpleIngestionWithIndexesMapped() throws Exception - { - try (final OpenAndClosedSegmentsAppenderatorTester tester = new OpenAndClosedSegmentsAppenderatorTester(2, - false, - true)) { - final Appenderator appenderator = tester.getAppenderator(); - boolean thrown; - - // startJob - Assert.assertEquals(null, appenderator.startJob()); - - // getDataSource - Assert.assertEquals(OpenAndClosedSegmentsAppenderatorTester.DATASOURCE, appenderator.getDataSource()); - - // add - Assert.assertEquals( - 1, - appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null) - .getNumRowsInSegment() - ); - - Assert.assertEquals( - 2, - appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 2), null) - .getNumRowsInSegment() - ); - - Assert.assertEquals( - 1, - appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 4), null) - .getNumRowsInSegment() - ); - - // getSegments - Assert.assertEquals(IDENTIFIERS.subList(0, 2), - appenderator.getSegments().stream().sorted().collect(Collectors.toList())); - - // getRowCount - Assert.assertEquals(2, appenderator.getRowCount(IDENTIFIERS.get(0))); - Assert.assertEquals(1, appenderator.getRowCount(IDENTIFIERS.get(1))); - thrown = false; - try { - appenderator.getRowCount(IDENTIFIERS.get(2)); - } - catch (IllegalStateException e) { - thrown = true; - } - Assert.assertTrue(thrown); - - // push all - final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( - appenderator.getSegments(), - null, - false - ).get(); - Assert.assertEquals( - IDENTIFIERS.subList(0, 2), - Lists.transform( - segmentsAndCommitMetadata.getSegments(), - new Function() - { - @Override - public SegmentIdWithShardSpec apply(DataSegment input) - { - return SegmentIdWithShardSpec.fromDataSegment(input); - } - } - ).stream().sorted().collect(Collectors.toList()) - ); - Assert.assertEquals(tester.getPushedSegments().stream().sorted().collect(Collectors.toList()), - segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList())); - - appenderator.clear(); - Assert.assertTrue(appenderator.getSegments().isEmpty()); - } - } - private static SegmentIdWithShardSpec createSegmentId(String interval, String version, int partitionNum) - { - return new SegmentIdWithShardSpec( - OpenAndClosedSegmentsAppenderatorTester.DATASOURCE, - Intervals.of(interval), - version, - new LinearShardSpec(partitionNum) - - ); - } - - static InputRow createInputRow(String ts, String dim, Object met) - { - return new MapBasedInputRow( - DateTimes.of(ts).getMillis(), - ImmutableList.of("dim"), - ImmutableMap.of( - "dim", - dim, - "met", - met - ) - ); - } - -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java index 2a39718667c..cc0dc1fad1b 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java @@ -222,10 +222,7 @@ public class StreamAppenderatorTester implements AutoCloseable QueryRunnerTestHelper.NOOP_QUERYWATCHER ), ScanQuery.class, new ScanQueryRunnerFactory( - new ScanQueryQueryToolChest( - new ScanQueryConfig(), - new DefaultGenericQueryMetricsFactory() - ), + new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()), new ScanQueryEngine(), new ScanQueryConfig() ) @@ -269,10 +266,7 @@ public class StreamAppenderatorTester implements AutoCloseable QueryRunnerTestHelper.NOOP_QUERYWATCHER ), ScanQuery.class, new ScanQueryRunnerFactory( - new ScanQueryQueryToolChest( - new ScanQueryConfig(), - new DefaultGenericQueryMetricsFactory() - ), + new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()), new ScanQueryEngine(), new ScanQueryConfig() ) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java index 9bf629931b5..23ac93db009 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java @@ -96,7 +96,7 @@ public class UnifiedIndexerAppenderatorsManagerTest extends InitializedNullHandl EasyMock.expect(appenderatorConfig.getMaxPendingPersists()).andReturn(0); EasyMock.expect(appenderatorConfig.isSkipBytesInMemoryOverheadCheck()).andReturn(false); EasyMock.replay(appenderatorConfig); - appenderator = manager.createClosedSegmentsOfflineAppenderatorForTask( + appenderator = manager.createBatchAppenderatorForTask( "taskId", new DataSchema( "myDataSource", diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java deleted file mode 100644 index 419b29ace7e..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java +++ /dev/null @@ -1,136 +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.segment.realtime.firehose; - -import com.google.common.collect.ImmutableList; -import org.apache.commons.io.IOUtils; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.server.metrics.EventReceiverFirehoseRegister; -import org.apache.druid.server.security.AllowAllAuthenticator; -import org.apache.druid.server.security.AuthConfig; -import org.apache.druid.server.security.AuthTestUtils; -import org.easymock.EasyMock; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import javax.servlet.http.HttpServletRequest; -import java.util.Locale; - -public class EventReceiverFirehoseIdleTest -{ - private static final int CAPACITY = 300; - private static final long MAX_IDLE_TIME = 5_000L; - private static final String SERVICE_NAME = "test_firehose"; - - private final String inputRow = "[{\n" - + " \"timestamp\":123,\n" - + " \"d1\":\"v1\"\n" - + "}]"; - - private EventReceiverFirehoseFactory eventReceiverFirehoseFactory; - private EventReceiverFirehoseFactory.EventReceiverFirehose firehose; - private EventReceiverFirehoseRegister register = new EventReceiverFirehoseRegister(); - private HttpServletRequest req; - - @Before - public void setUp() - { - req = EasyMock.createMock(HttpServletRequest.class); - eventReceiverFirehoseFactory = new EventReceiverFirehoseFactory( - SERVICE_NAME, - CAPACITY, - MAX_IDLE_TIME, - null, - new DefaultObjectMapper(), - new DefaultObjectMapper(), - register, - AuthTestUtils.TEST_AUTHORIZER_MAPPER - ); - firehose = (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory.connect( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec( - "timestamp", - "auto", - null - ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1"))), - null, - null, - null - ) - ), - null - ); - } - - @Test(timeout = 40_000L) - public void testIdle() throws Exception - { - awaitFirehoseClosed(); - awaitDelayedExecutorThreadTerminated(); - } - - private void awaitFirehoseClosed() throws InterruptedException - { - while (!firehose.isClosed()) { - Thread.sleep(50); - } - } - - private void awaitDelayedExecutorThreadTerminated() throws InterruptedException - { - firehose.getDelayedCloseExecutor().join(); - } - - @Test(timeout = 40_000L) - public void testNotIdle() throws Exception - { - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) - .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) - .anyTimes(); - EasyMock.expect(req.getHeader("X-Firehose-Producer-Id")).andReturn(null).anyTimes(); - EasyMock.expect(req.getContentType()).andReturn("application/json").anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(req); - - final int checks = 5; - for (int i = 0; i < checks; i++) { - Assert.assertFalse(firehose.isClosed()); - System.out.printf(Locale.ENGLISH, "Check %d/%d passed\n", i + 1, checks); - firehose.addAll(IOUtils.toInputStream(inputRow), req); - Thread.sleep(3_000L); - } - - awaitFirehoseClosed(); - awaitDelayedExecutorThreadTerminated(); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java deleted file mode 100644 index 38b16c79cab..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java +++ /dev/null @@ -1,442 +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.segment.realtime.firehose; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import org.apache.commons.io.IOUtils; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.server.metrics.EventReceiverFirehoseMetric; -import org.apache.druid.server.metrics.EventReceiverFirehoseRegister; -import org.apache.druid.server.security.AllowAllAuthenticator; -import org.apache.druid.server.security.AuthConfig; -import org.apache.druid.server.security.AuthTestUtils; -import org.easymock.EasyMock; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.core.Response; -import java.io.IOException; -import java.io.InputStream; -import java.nio.charset.StandardCharsets; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -public class EventReceiverFirehoseTest -{ - private static final int CAPACITY = 300; - private static final int NUM_EVENTS = 100; - private static final long MAX_IDLE_TIME_MILLIS = TimeUnit.SECONDS.toMillis(20); - private static final String SERVICE_NAME = "test_firehose"; - - private final String inputRow = "[{\n" - + " \"timestamp\":123,\n" - + " \"d1\":\"v1\"\n" - + "}]"; - - private EventReceiverFirehoseFactory eventReceiverFirehoseFactory; - private EventReceiverFirehoseFactory.EventReceiverFirehose firehose; - private EventReceiverFirehoseRegister register = new EventReceiverFirehoseRegister(); - private HttpServletRequest req; - - @Before - public void setUp() - { - req = EasyMock.createMock(HttpServletRequest.class); - eventReceiverFirehoseFactory = new EventReceiverFirehoseFactory( - SERVICE_NAME, - CAPACITY, - MAX_IDLE_TIME_MILLIS, - null, - new DefaultObjectMapper(), - new DefaultObjectMapper(), - register, - AuthTestUtils.TEST_AUTHORIZER_MAPPER - ); - firehose = (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory.connect( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec( - "timestamp", - "auto", - null - ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1"))), - null, - null, - null - ) - ), - null - ); - } - - @Test(timeout = 60_000L) - public void testSingleThread() throws IOException, InterruptedException - { - for (int i = 0; i < NUM_EVENTS; ++i) { - setUpRequestExpectations(null, null); - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - firehose.addAll(inputStream, req); - Assert.assertEquals(i + 1, firehose.getCurrentBufferSize()); - inputStream.close(); - } - - EasyMock.verify(req); - - final Iterable> metrics = register.getMetrics(); - Assert.assertEquals(1, Iterables.size(metrics)); - - final Map.Entry entry = Iterables.getLast(metrics); - Assert.assertEquals(SERVICE_NAME, entry.getKey()); - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(NUM_EVENTS, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(NUM_EVENTS, firehose.getCurrentBufferSize()); - - for (int i = NUM_EVENTS - 1; i >= 0; --i) { - Assert.assertTrue(firehose.hasMore()); - Assert.assertNotNull(firehose.nextRow()); - Assert.assertEquals(i, firehose.getCurrentBufferSize()); - } - - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(0, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(0, firehose.getCurrentBufferSize()); - - firehose.close(); - Assert.assertFalse(firehose.hasMore()); - Assert.assertEquals(0, Iterables.size(register.getMetrics())); - - awaitDelayedExecutorThreadTerminated(); - } - - @Test(timeout = 60_000L) - public void testMultipleThreads() throws InterruptedException, IOException, TimeoutException, ExecutionException - { - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) - .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) - .anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - EasyMock.expectLastCall().anyTimes(); - - EasyMock.expect(req.getContentType()).andReturn("application/json").times(2 * NUM_EVENTS); - EasyMock.expect(req.getHeader("X-Firehose-Producer-Id")).andReturn(null).times(2 * NUM_EVENTS); - EasyMock.replay(req); - - final ExecutorService executorService = Execs.singleThreaded("single_thread"); - final Future future = executorService.submit( - new Callable() - { - @Override - public Boolean call() throws Exception - { - for (int i = 0; i < NUM_EVENTS; ++i) { - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - firehose.addAll(inputStream, req); - inputStream.close(); - } - return true; - } - } - ); - - for (int i = 0; i < NUM_EVENTS; ++i) { - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - firehose.addAll(inputStream, req); - inputStream.close(); - } - - future.get(10, TimeUnit.SECONDS); - - EasyMock.verify(req); - - final Iterable> metrics = register.getMetrics(); - Assert.assertEquals(1, Iterables.size(metrics)); - - final Map.Entry entry = Iterables.getLast(metrics); - - Assert.assertEquals(SERVICE_NAME, entry.getKey()); - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(2 * NUM_EVENTS, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(2 * NUM_EVENTS, firehose.getCurrentBufferSize()); - - for (int i = 2 * NUM_EVENTS - 1; i >= 0; --i) { - Assert.assertTrue(firehose.hasMore()); - Assert.assertNotNull(firehose.nextRow()); - Assert.assertEquals(i, firehose.getCurrentBufferSize()); - } - - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(0, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(0, firehose.getCurrentBufferSize()); - - firehose.close(); - Assert.assertFalse(firehose.hasMore()); - Assert.assertEquals(0, Iterables.size(register.getMetrics())); - - awaitDelayedExecutorThreadTerminated(); - - executorService.shutdownNow(); - } - - @Test(expected = ISE.class) - public void testDuplicateRegistering() - { - EventReceiverFirehoseFactory eventReceiverFirehoseFactory2 = new EventReceiverFirehoseFactory( - SERVICE_NAME, - CAPACITY, - MAX_IDLE_TIME_MILLIS, - null, - new DefaultObjectMapper(), - new DefaultObjectMapper(), - register, - AuthTestUtils.TEST_AUTHORIZER_MAPPER - ); - EventReceiverFirehoseFactory.EventReceiverFirehose firehose2 = - (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory2 - .connect( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec( - "timestamp", - "auto", - null - ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1"))), - null, - null, - null - ) - ), - null - ); - } - - @Test(timeout = 60_000L) - public void testShutdownWithPrevTime() throws Exception - { - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) - .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) - .anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(req); - - firehose.shutdown(DateTimes.nowUtc().minusMinutes(2).toString(), req); - awaitFirehoseClosed(); - awaitDelayedExecutorThreadTerminated(); - } - - private void awaitFirehoseClosed() throws InterruptedException - { - while (!firehose.isClosed()) { - Thread.sleep(50); - } - } - - private void awaitDelayedExecutorThreadTerminated() throws InterruptedException - { - firehose.getDelayedCloseExecutor().join(); - } - - @Test(timeout = 60_000L) - public void testShutdown() throws Exception - { - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) - .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) - .anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(req); - - firehose.shutdown(DateTimes.nowUtc().plusMillis(100).toString(), req); - awaitFirehoseClosed(); - awaitDelayedExecutorThreadTerminated(); - } - - @Test - public void testProducerSequence() throws IOException - { - for (int i = 0; i < NUM_EVENTS; ++i) { - setUpRequestExpectations("producer", String.valueOf(i)); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - firehose.addAll(inputStream, req); - Assert.assertEquals(i + 1, firehose.getCurrentBufferSize()); - inputStream.close(); - } - - EasyMock.verify(req); - - final Iterable> metrics = register.getMetrics(); - Assert.assertEquals(1, Iterables.size(metrics)); - - final Map.Entry entry = Iterables.getLast(metrics); - Assert.assertEquals(SERVICE_NAME, entry.getKey()); - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(NUM_EVENTS, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(NUM_EVENTS, firehose.getCurrentBufferSize()); - - for (int i = NUM_EVENTS - 1; i >= 0; --i) { - Assert.assertTrue(firehose.hasMore()); - Assert.assertNotNull(firehose.nextRow()); - Assert.assertEquals(i, firehose.getCurrentBufferSize()); - } - - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(0, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(0, firehose.getCurrentBufferSize()); - - firehose.close(); - Assert.assertFalse(firehose.hasMore()); - Assert.assertEquals(0, Iterables.size(register.getMetrics())); - } - - @Test - public void testLowProducerSequence() throws IOException - { - for (int i = 0; i < NUM_EVENTS; ++i) { - setUpRequestExpectations("producer", "1"); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - final Response response = firehose.addAll(inputStream, req); - Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals(1, firehose.getCurrentBufferSize()); - inputStream.close(); - } - - EasyMock.verify(req); - - firehose.close(); - } - - @Test - public void testMissingProducerSequence() throws IOException - { - setUpRequestExpectations("producer", null); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - final Response response = firehose.addAll(inputStream, req); - - Assert.assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), response.getStatus()); - - inputStream.close(); - - EasyMock.verify(req); - - firehose.close(); - } - - @Test - public void testTooManyProducerIds() throws IOException - { - for (int i = 0; i < EventReceiverFirehoseFactory.MAX_FIREHOSE_PRODUCERS - 1; i++) { - setUpRequestExpectations("producer-" + i, "0"); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - final Response response = firehose.addAll(inputStream, req); - Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - inputStream.close(); - Assert.assertTrue(firehose.hasMore()); - Assert.assertNotNull(firehose.nextRow()); - } - - setUpRequestExpectations("toomany", "0"); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - final Response response = firehose.addAll(inputStream, req); - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), response.getStatus()); - inputStream.close(); - - EasyMock.verify(req); - - firehose.close(); - } - - @Test - public void testNaNProducerSequence() throws IOException - { - setUpRequestExpectations("producer", "foo"); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - final Response response = firehose.addAll(inputStream, req); - - Assert.assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), response.getStatus()); - - inputStream.close(); - - EasyMock.verify(req); - - firehose.close(); - } - - private void setUpRequestExpectations(String producerId, String producerSequenceValue) - { - EasyMock.reset(req); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) - .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) - .anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - EasyMock.expectLastCall().anyTimes(); - - EasyMock.expect(req.getContentType()).andReturn("application/json"); - EasyMock.expect(req.getHeader("X-Firehose-Producer-Id")).andReturn(producerId); - - if (producerId != null) { - EasyMock.expect(req.getHeader("X-Firehose-Producer-Seq")).andReturn(producerSequenceValue); - } - - EasyMock.replay(req); - } -} diff --git a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java index c3863708a66..038fbce7d45 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -685,7 +685,6 @@ public class ClientQuerySegmentWalkerTest ScanQuery subquery = new Druids.ScanQueryBuilder().dataSource(MULTI) .columns("s", "n") .eternityInterval() - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build(); final GroupByQuery query = @@ -736,7 +735,6 @@ public class ClientQuerySegmentWalkerTest ScanQuery subquery = new Druids.ScanQueryBuilder().dataSource(MULTI) .columns("s", "n") .eternityInterval() - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build(); final TopNQuery query = @@ -858,7 +856,6 @@ public class ClientQuerySegmentWalkerTest .dataSource(FOO) .intervals(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)) .columns("s") - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() .withId(DUMMY_QUERY_ID); @@ -876,7 +873,6 @@ public class ClientQuerySegmentWalkerTest ) ) .columns("v") - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() .withId(DUMMY_QUERY_ID); diff --git a/server/src/test/java/org/apache/druid/server/QueryStackTests.java b/server/src/test/java/org/apache/druid/server/QueryStackTests.java index 3df1d95b33a..f6bfebcf344 100644 --- a/server/src/test/java/org/apache/druid/server/QueryStackTests.java +++ b/server/src/test/java/org/apache/druid/server/QueryStackTests.java @@ -343,10 +343,7 @@ public class QueryStackTests .put( ScanQuery.class, new ScanQueryRunnerFactory( - new ScanQueryQueryToolChest( - new ScanQueryConfig(), - new DefaultGenericQueryMetricsFactory() - ), + new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()), new ScanQueryEngine(), new ScanQueryConfig() ) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java index 365c872165e..27d80e3b568 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java @@ -19,81 +19,85 @@ package org.apache.druid.server.coordinator; +import org.apache.druid.audit.AuditInfo; +import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.java.util.common.DateTimes; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.Mockito; import org.mockito.junit.MockitoJUnitRunner; @RunWith(MockitoJUnitRunner.class) public class DataSourceCompactionConfigAuditEntryTest { - private static final double COMPACTION_TASK_SLOT_RATIO = 0.1; - private static final int MAX_COMPACTION_SLOTS = 9; - private static final boolean USE_AUTO_SCALE_SLOTS = true; + private static final String DS_WIKI = "wiki"; + private final AuditInfo auditInfo = new AuditInfo("author", "identity", "comment", "ip"); + + private final DataSourceCompactionConfigAuditEntry firstEntry = new DataSourceCompactionConfigAuditEntry( + new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.MSQ), + DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + auditInfo, + DateTimes.nowUtc() + ); - @Mock - private CoordinatorCompactionConfig coordinatorCompactionConfig; - - @Before - public void setUp() + @Test + public void testhasSameConfigWithSameBaseConfigIsTrue() { - Mockito.when(coordinatorCompactionConfig.getCompactionTaskSlotRatio()).thenReturn(COMPACTION_TASK_SLOT_RATIO); - Mockito.when(coordinatorCompactionConfig.getMaxCompactionTaskSlots()).thenReturn(MAX_COMPACTION_SLOTS); - Mockito.when(coordinatorCompactionConfig.isUseAutoScaleSlots()).thenReturn(USE_AUTO_SCALE_SLOTS); + final DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( + new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.MSQ), + DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + auditInfo, + DateTimes.nowUtc() + ); + Assert.assertTrue(firstEntry.hasSameConfig(secondEntry)); + Assert.assertTrue(secondEntry.hasSameConfig(firstEntry)); } @Test - public void testhasSameConfigWithSameBaseConfigShouldReturnTrue() + public void testhasSameConfigWithDifferentClusterConfigIsFalse() { - DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig config = - new DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig( - COMPACTION_TASK_SLOT_RATIO, - MAX_COMPACTION_SLOTS, - USE_AUTO_SCALE_SLOTS - ); + DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( + new ClusterCompactionConfig(0.1, 9, false, CompactionEngine.MSQ), + DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + auditInfo, + DateTimes.nowUtc() + ); + Assert.assertFalse(firstEntry.hasSameConfig(secondEntry)); + Assert.assertFalse(secondEntry.hasSameConfig(firstEntry)); - Assert.assertTrue(config.hasSameConfig(coordinatorCompactionConfig)); + secondEntry = new DataSourceCompactionConfigAuditEntry( + new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.NATIVE), + DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + auditInfo, + DateTimes.nowUtc() + ); + Assert.assertFalse(firstEntry.hasSameConfig(secondEntry)); + Assert.assertFalse(secondEntry.hasSameConfig(firstEntry)); } @Test - public void testhasSameConfigWithDifferentUseAutoScaleSlotsShouldReturnFalse() + public void testhasSameConfigWithDifferentDatasourceConfigIsFalse() { - DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig config = - new DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig( - COMPACTION_TASK_SLOT_RATIO, - MAX_COMPACTION_SLOTS, - !USE_AUTO_SCALE_SLOTS - ); - - Assert.assertFalse(config.hasSameConfig(coordinatorCompactionConfig)); + DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( + new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.NATIVE), + DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + auditInfo, + DateTimes.nowUtc() + ); + Assert.assertFalse(firstEntry.hasSameConfig(secondEntry)); + Assert.assertFalse(secondEntry.hasSameConfig(firstEntry)); } @Test - public void testhasSameConfigWithDifferentMaxCompactionSlotsShouldReturnFalse() + public void testhasSameConfigWithNullDatasourceConfigIsFalse() { - DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig config = - new DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig( - COMPACTION_TASK_SLOT_RATIO, - MAX_COMPACTION_SLOTS + 1, - USE_AUTO_SCALE_SLOTS - ); - - Assert.assertFalse(config.hasSameConfig(coordinatorCompactionConfig)); - } - - @Test - public void testhasSameConfigWithDifferentCompactionSlotRatioShouldReturnFalse() - { - DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig config = - new DataSourceCompactionConfigAuditEntry.GlobalCompactionConfig( - COMPACTION_TASK_SLOT_RATIO - 0.03, - MAX_COMPACTION_SLOTS, - USE_AUTO_SCALE_SLOTS - ); - - Assert.assertFalse(config.hasSameConfig(coordinatorCompactionConfig)); + final DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( + new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.NATIVE), + null, + auditInfo, + DateTimes.nowUtc() + ); + Assert.assertFalse(firstEntry.hasSameConfig(secondEntry)); + Assert.assertFalse(secondEntry.hasSameConfig(firstEntry)); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java index bf4badb44db..60a638974f4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java @@ -19,167 +19,183 @@ package org.apache.druid.server.coordinator; -import com.google.common.collect.ImmutableList; import org.apache.druid.audit.AuditInfo; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.java.util.common.DateTimes; import org.joda.time.DateTime; +import org.joda.time.Period; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.Answers; -import org.mockito.Mock; -import org.mockito.Mockito; import org.mockito.junit.MockitoJUnitRunner; +import java.util.List; + @RunWith(MockitoJUnitRunner.class) public class DataSourceCompactionConfigHistoryTest { - private static final String DATASOURCE = "DATASOURCE"; - private static final String DATASOURCE_2 = "DATASOURCE_2"; - private static final String DATASOURCE_NOT_EXISTS = "DATASOURCE_NOT_EXISTS"; - private static final double COMPACTION_TASK_SLOT_RATIO = 0.1; - private static final int MAX_COMPACTION_TASK_SLOTS = 9; - private static final boolean USE_AUTO_SCALE_SLOTS = false; - private static final DateTime AUDIT_TIME = DateTimes.of(2023, 1, 13, 9, 0); - private static final DateTime AUDIT_TIME_2 = DateTimes.of(2023, 1, 13, 9, 30); - private static final DateTime AUDIT_TIME_3 = DateTimes.of(2023, 1, 13, 10, 0); + private final AuditInfo auditInfo = new AuditInfo("author", "identity", "comment", "ip"); + private final DataSourceCompactionConfig wikiCompactionConfig + = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(); - @Mock - private CoordinatorCompactionConfig compactionConfig; - @Mock(answer = Answers.RETURNS_MOCKS) - private DataSourceCompactionConfig configForDataSource; - @Mock(answer = Answers.RETURNS_MOCKS) - private DataSourceCompactionConfig configForDataSourceWithChange; - @Mock(answer = Answers.RETURNS_MOCKS) - private DataSourceCompactionConfig configForDataSource2; - @Mock(answer = Answers.RETURNS_DEEP_STUBS) - private AuditInfo auditInfo; - @Mock(answer = Answers.RETURNS_DEEP_STUBS) - private AuditInfo auditInfo2; - @Mock(answer = Answers.RETURNS_DEEP_STUBS) - private AuditInfo auditInfo3; - - private DataSourceCompactionConfigHistory target; + private DataSourceCompactionConfigHistory wikiAuditHistory; @Before - public void setUp() + public void setup() { - Mockito.when(compactionConfig.getCompactionTaskSlotRatio()).thenReturn(COMPACTION_TASK_SLOT_RATIO); - Mockito.when(compactionConfig.getMaxCompactionTaskSlots()).thenReturn(MAX_COMPACTION_TASK_SLOTS); - Mockito.when(compactionConfig.isUseAutoScaleSlots()).thenReturn(USE_AUTO_SCALE_SLOTS); - Mockito.when(configForDataSource.getDataSource()).thenReturn(DATASOURCE); - Mockito.when(configForDataSourceWithChange.getDataSource()).thenReturn(DATASOURCE); - Mockito.when(configForDataSource2.getDataSource()).thenReturn(DATASOURCE_2); - Mockito.when(compactionConfig.getCompactionConfigs()) - .thenReturn(ImmutableList.of(configForDataSource, configForDataSource2)); - target = new DataSourceCompactionConfigHistory(DATASOURCE); + wikiAuditHistory = new DataSourceCompactionConfigHistory(DS.WIKI); } @Test - public void testAddCompactionConfigShouldAddToHistory() + public void testAddDatasourceConfigShouldAddToHistory() { - target.add(compactionConfig, auditInfo, AUDIT_TIME); - Assert.assertEquals(1, target.getHistory().size()); - DataSourceCompactionConfigAuditEntry auditEntry = target.getHistory().get(0); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); + final DateTime auditTime = DateTimes.nowUtc(); + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(wikiCompactionConfig), + auditInfo, + auditTime + ); + + Assert.assertEquals(1, wikiAuditHistory.getEntries().size()); + DataSourceCompactionConfigAuditEntry auditEntry = wikiAuditHistory.getEntries().get(0); + Assert.assertEquals(wikiCompactionConfig, auditEntry.getCompactionConfig()); Assert.assertEquals(auditInfo, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME, auditEntry.getAuditTime()); + Assert.assertEquals(auditTime, auditEntry.getAuditTime()); } @Test - public void testAddAndDeleteCompactionConfigShouldAddBothToHistory() + public void testAddDeleteDatasourceConfigShouldAddBothToHistory() { - target.add(compactionConfig, auditInfo, AUDIT_TIME); - Mockito.when(compactionConfig.getCompactionConfigs()).thenReturn(ImmutableList.of(configForDataSource2)); - target.add(compactionConfig, auditInfo2, AUDIT_TIME_2); - Assert.assertEquals(2, target.getHistory().size()); - DataSourceCompactionConfigAuditEntry auditEntry = target.getHistory().get(0); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME, auditEntry.getAuditTime()); - auditEntry = target.getHistory().get(1); - Assert.assertEquals(null, auditEntry.getCompactionConfig()); - Assert.assertEquals(auditInfo2, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME_2, auditEntry.getAuditTime()); + final DateTime auditTime = DateTimes.nowUtc(); + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(wikiCompactionConfig), + auditInfo, + auditTime + ); + wikiAuditHistory.add(DruidCompactionConfig.empty(), auditInfo, auditTime.plusHours(2)); + + final List entries = wikiAuditHistory.getEntries(); + Assert.assertEquals(2, entries.size()); + + final DataSourceCompactionConfigAuditEntry firstEntry = entries.get(0); + Assert.assertEquals(wikiCompactionConfig, firstEntry.getCompactionConfig()); + Assert.assertEquals(auditInfo, firstEntry.getAuditInfo()); + Assert.assertEquals(auditTime, firstEntry.getAuditTime()); + + final DataSourceCompactionConfigAuditEntry secondEntry = entries.get(1); + Assert.assertNull(secondEntry.getCompactionConfig()); + Assert.assertEquals(firstEntry.getGlobalConfig(), secondEntry.getGlobalConfig()); + Assert.assertEquals(auditInfo, secondEntry.getAuditInfo()); + Assert.assertEquals(auditTime.plusHours(2), secondEntry.getAuditTime()); } @Test - public void testAddAndDeleteAnotherCompactionConfigShouldNotAddToHistory() + public void testAddDeleteAnotherDatasourceConfigShouldNotAddToHistory() { - target.add(compactionConfig, auditInfo, AUDIT_TIME); - Mockito.when(compactionConfig.getCompactionConfigs()).thenReturn(ImmutableList.of(configForDataSource)); - target.add(compactionConfig, auditInfo2, AUDIT_TIME_2); - Assert.assertEquals(1, target.getHistory().size()); - DataSourceCompactionConfigAuditEntry auditEntry = target.getHistory().get(0); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME, auditEntry.getAuditTime()); + final DataSourceCompactionConfig koalaCompactionConfig + = DataSourceCompactionConfig.builder().forDataSource(DS.KOALA).build(); + + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(koalaCompactionConfig), + auditInfo, + DateTimes.nowUtc() + ); + wikiAuditHistory.add(DruidCompactionConfig.empty(), auditInfo, DateTimes.nowUtc()); + + Assert.assertTrue(wikiAuditHistory.getEntries().isEmpty()); } @Test - public void testAddDeletedAddCompactionConfigShouldAddAllToHistory() + public void testAddDeleteAddDatasourceConfigShouldAddAllToHistory() { - target.add(compactionConfig, auditInfo, AUDIT_TIME); - Mockito.when(compactionConfig.getCompactionConfigs()).thenReturn(ImmutableList.of(configForDataSource2)); - target.add(compactionConfig, auditInfo2, AUDIT_TIME_2); - Mockito.when(compactionConfig.getCompactionConfigs()) - .thenReturn(ImmutableList.of(configForDataSourceWithChange, configForDataSource2)); - target.add(compactionConfig, auditInfo3, AUDIT_TIME_3); - Assert.assertEquals(3, target.getHistory().size()); - DataSourceCompactionConfigAuditEntry auditEntry = target.getHistory().get(0); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME, auditEntry.getAuditTime()); - auditEntry = target.getHistory().get(2); - Assert.assertEquals(configForDataSourceWithChange, auditEntry.getCompactionConfig()); - Assert.assertEquals(auditInfo3, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME_3, auditEntry.getAuditTime()); + final DateTime auditTime = DateTimes.nowUtc(); + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(wikiCompactionConfig), + auditInfo, + auditTime + ); + wikiAuditHistory.add( + DruidCompactionConfig.empty(), + auditInfo, + auditTime.plusHours(2) + ); + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(wikiCompactionConfig), + auditInfo, + auditTime.plusHours(3) + ); + + final List entries = wikiAuditHistory.getEntries(); + Assert.assertEquals(3, entries.size()); + + final DataSourceCompactionConfigAuditEntry firstEntry = entries.get(0); + final DataSourceCompactionConfigAuditEntry thirdEntry = entries.get(2); + Assert.assertTrue(firstEntry.hasSameConfig(thirdEntry)); } @Test - public void testAddAndChangeCompactionConfigShouldAddBothToHistory() + public void testAddModifyDatasourceConfigShouldAddBothToHistory() { - target.add(compactionConfig, auditInfo, AUDIT_TIME); - Mockito.when(compactionConfig.getCompactionConfigs()).thenReturn(ImmutableList.of(configForDataSourceWithChange)); - target.add(compactionConfig, auditInfo2, AUDIT_TIME_2); - Assert.assertEquals(2, target.getHistory().size()); - DataSourceCompactionConfigAuditEntry auditEntry = target.getHistory().get(0); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME, auditEntry.getAuditTime()); - auditEntry = target.getHistory().get(1); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo2, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME_2, auditEntry.getAuditTime()); + final DateTime auditTime = DateTimes.nowUtc(); + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(wikiCompactionConfig), + auditInfo, + auditTime + ); + + + final DataSourceCompactionConfig updatedWikiConfig + = DataSourceCompactionConfig.builder() + .forDataSource(DS.WIKI) + .withSkipOffsetFromLatest(Period.hours(5)) + .build(); + wikiAuditHistory.add( + DruidCompactionConfig.empty().withDatasourceConfig(updatedWikiConfig), + auditInfo, + auditTime.plusHours(3) + ); + + final List entries = wikiAuditHistory.getEntries(); + Assert.assertEquals(2, entries.size()); + + final DataSourceCompactionConfigAuditEntry firstEntry = entries.get(0); + final DataSourceCompactionConfigAuditEntry secondEntry = entries.get(1); + Assert.assertEquals(firstEntry.getGlobalConfig(), secondEntry.getGlobalConfig()); + + Assert.assertEquals(wikiCompactionConfig, firstEntry.getCompactionConfig()); + Assert.assertEquals(updatedWikiConfig, secondEntry.getCompactionConfig()); + Assert.assertFalse(firstEntry.hasSameConfig(secondEntry)); } @Test - public void testAddAndChangeGlobalSettingsShouldAddTwice() + public void testAddAndModifyClusterConfigShouldAddTwice() { - target.add(compactionConfig, auditInfo, AUDIT_TIME); - int newMaxTaskSlots = MAX_COMPACTION_TASK_SLOTS - 1; - Mockito.when(compactionConfig.getMaxCompactionTaskSlots()).thenReturn(newMaxTaskSlots); - target.add(compactionConfig, auditInfo2, AUDIT_TIME_2); - Assert.assertEquals(2, target.getHistory().size()); - DataSourceCompactionConfigAuditEntry auditEntry = target.getHistory().get(0); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME, auditEntry.getAuditTime()); - Assert.assertEquals(MAX_COMPACTION_TASK_SLOTS, auditEntry.getGlobalConfig().getMaxCompactionTaskSlots()); - auditEntry = target.getHistory().get(1); - Assert.assertEquals(DATASOURCE, auditEntry.getCompactionConfig().getDataSource()); - Assert.assertEquals(auditInfo2, auditEntry.getAuditInfo()); - Assert.assertEquals(AUDIT_TIME_2, auditEntry.getAuditTime()); - Assert.assertEquals(newMaxTaskSlots, auditEntry.getGlobalConfig().getMaxCompactionTaskSlots()); + final DruidCompactionConfig originalConfig + = DruidCompactionConfig.empty().withDatasourceConfig(wikiCompactionConfig); + + wikiAuditHistory.add(originalConfig, auditInfo, DateTimes.nowUtc()); + + final DruidCompactionConfig updatedConfig = originalConfig.withClusterConfig( + new ClusterCompactionConfig(null, null, null, CompactionEngine.MSQ) + ); + wikiAuditHistory.add(updatedConfig, auditInfo, DateTimes.nowUtc()); + + final List entries = wikiAuditHistory.getEntries(); + Assert.assertEquals(2, entries.size()); + + final DataSourceCompactionConfigAuditEntry firstEntry = entries.get(0); + final DataSourceCompactionConfigAuditEntry secondEntry = entries.get(1); + Assert.assertEquals(secondEntry.getCompactionConfig(), firstEntry.getCompactionConfig()); + + Assert.assertEquals(originalConfig.clusterConfig(), firstEntry.getGlobalConfig()); + Assert.assertEquals(updatedConfig.clusterConfig(), secondEntry.getGlobalConfig()); + Assert.assertFalse(firstEntry.hasSameConfig(secondEntry)); } - @Test - public void testAddCompactionConfigDoesNotHaveDataSourceWithNoHistoryShouldNotAdd() + private static class DS { - target = new DataSourceCompactionConfigHistory(DATASOURCE_NOT_EXISTS); - target.add(compactionConfig, auditInfo, AUDIT_TIME); - Assert.assertTrue(target.getHistory().isEmpty()); + static final String KOALA = "koala"; + static final String WIKI = "wiki"; } - } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java index a9334f077a4..67f276f7651 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java @@ -42,9 +42,7 @@ import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Duration; import org.joda.time.Period; import org.junit.Assert; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import java.io.IOException; @@ -52,27 +50,15 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest { private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); - @Rule - public final ExpectedException expectedException = ExpectedException.none(); - @Test public void testSerdeBasic() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - null, - null, - new Period(3600), - null, - null, - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withSkipOffsetFromLatest(new Period(3600)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -90,21 +76,15 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeWithMaxRowsPerSegment() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - 30, - new Period(3600), - null, - null, - null, - null, - null, - null, - CompactionEngine.MSQ, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withMaxRowsPerSegment(30) + .withSkipOffsetFromLatest(new Period(3600)) + .withEngine(CompactionEngine.MSQ) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -121,41 +101,14 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeWithMaxTotalRows() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - 10000L, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - CompactionEngine.NATIVE, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withEngine(CompactionEngine.NATIVE) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -172,42 +125,14 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeMaxTotalRowsWithMaxRowsPerSegment() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - 10000, - new Period(3600), - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - 10000L, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); - + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withMaxRowsPerSegment(10000) + .withSkipOffsetFromLatest(new Period(3600)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -301,21 +226,14 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeGranularitySpec() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -332,21 +250,14 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeGranularitySpecWithQueryGranularity() throws Exception { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(null, Granularities.YEAR, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(null, Granularities.YEAR, null)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -366,21 +277,13 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeWithNullGranularitySpec() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - null, - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -397,21 +300,14 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeGranularitySpecWithNullValues() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(null, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(null, null, null)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -428,21 +324,14 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeGranularitySpecWithRollup() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(null, null, true), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(null, null, true)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -462,21 +351,15 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeIOConfigWithNonNullDropExisting() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - new UserCompactionTaskIOConfig(true), - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) + .withIoConfig(new UserCompactionTaskIOConfig(true)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -494,21 +377,15 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeIOConfigWithNullDropExisting() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - new UserCompactionTaskIOConfig(null), - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) + .withIoConfig(new UserCompactionTaskIOConfig(null)) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -526,21 +403,18 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeDimensionsSpec() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - null, - new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withDimensionsSpec( + new UserCompactionTaskDimensionsConfig( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo")) + ) + ) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -558,21 +432,14 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest public void testSerdeTransformSpec() throws IOException { NullHandling.initializeForTests(); - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - null, - null, - null, - new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null)), - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withTransformSpec(new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null))) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); @@ -589,21 +456,14 @@ public class DataSourceCompactionConfigTest extends InitializedNullHandlingTest @Test public void testSerdeMetricsSpec() throws IOException { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - null, - null, - new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, - null, - null, - null, - ImmutableMap.of("key", "val") - ); + final DataSourceCompactionConfig config = DataSourceCompactionConfig + .builder() + .forDataSource("dataSource") + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(new Period(3600)) + .withMetricsSpec(new AggregatorFactory[]{new CountAggregatorFactory("cnt")}) + .withTaskContext(ImmutableMap.of("key", "val")) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java new file mode 100644 index 00000000000..bf13a94e008 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +public class DruidCompactionConfigTest +{ + private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + + @Test + public void testSerdeDefaultConfig() throws Exception + { + final DruidCompactionConfig defaultConfig = DruidCompactionConfig.empty(); + final String json = MAPPER.writeValueAsString(defaultConfig); + + DruidCompactionConfig deserialized = MAPPER.readValue(json, DruidCompactionConfig.class); + Assert.assertEquals(defaultConfig, deserialized); + } + + @Test + public void testSerdeWithDatasourceConfigs() throws Exception + { + final DruidCompactionConfig config = new DruidCompactionConfig( + Arrays.asList( + DataSourceCompactionConfig + .builder() + .forDataSource(DS.WIKI) + .withSkipOffsetFromLatest(Period.hours(1)) + .build(), + DataSourceCompactionConfig + .builder() + .forDataSource(DS.KOALA) + .withSkipOffsetFromLatest(Period.hours(2)) + .build() + ), + null, + null, + null, + CompactionEngine.MSQ + ); + + final String json = MAPPER.writeValueAsString(config); + DruidCompactionConfig deserialized = MAPPER.readValue(json, DruidCompactionConfig.class); + Assert.assertEquals(config, deserialized); + } + + @Test + public void testCopyWithClusterConfig() + { + final DruidCompactionConfig config = DruidCompactionConfig.empty(); + + final ClusterCompactionConfig clusterConfig = new ClusterCompactionConfig(0.5, 10, false, CompactionEngine.MSQ); + final DruidCompactionConfig copy = config.withClusterConfig(clusterConfig); + + Assert.assertEquals(clusterConfig, copy.clusterConfig()); + Assert.assertNotEquals(clusterConfig, config.clusterConfig()); + } + + @Test + public void testCopyWithDatasourceConfigs() + { + final DruidCompactionConfig config = DruidCompactionConfig.empty(); + Assert.assertTrue(config.getCompactionConfigs().isEmpty()); + + final DataSourceCompactionConfig dataSourceConfig + = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).withEngine(CompactionEngine.NATIVE).build(); + final DruidCompactionConfig copy = config.withDatasourceConfigs(Collections.singletonList(dataSourceConfig)); + + Assert.assertEquals(1, copy.getCompactionConfigs().size()); + Assert.assertEquals(dataSourceConfig, copy.findConfigForDatasource(DS.WIKI).orNull()); + } + + private static class DS + { + static final String WIKI = "wiki"; + static final String KOALA = "koala"; + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 8c7e0ae14e5..801016eb819 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -136,11 +136,11 @@ public class DruidCoordinatorTest extends CuratorTestBase ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); EasyMock.expect( configManager.watch( - EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY), + EasyMock.eq(DruidCompactionConfig.CONFIG_KEY), EasyMock.anyObject(Class.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference<>(CoordinatorCompactionConfig.empty())).anyTimes(); + ).andReturn(new AtomicReference<>(DruidCompactionConfig.empty())).anyTimes(); EasyMock.replay(configManager); setupServerAndCurator(); curator.start(); @@ -734,11 +734,11 @@ public class DruidCoordinatorTest extends CuratorTestBase ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); EasyMock.expect( configManager.watch( - EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY), + EasyMock.eq(DruidCompactionConfig.CONFIG_KEY), EasyMock.anyObject(Class.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference<>(CoordinatorCompactionConfig.empty())).anyTimes(); + ).andReturn(new AtomicReference<>(DruidCompactionConfig.empty())).anyTimes(); EasyMock.replay(configManager); DruidDataSource dataSource = new DruidDataSource("dataSource1", Collections.emptyMap()); DataSegment dataSegment = new DataSegment( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 236cfaf7da5..eb03acc48bf 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -69,8 +69,8 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; @@ -1834,7 +1834,7 @@ public class CompactSegmentsTest .newBuilder(DateTimes.nowUtc()) .withDataSourcesSnapshot(dataSources) .withCompactionConfig( - new CoordinatorCompactionConfig( + new DruidCompactionConfig( compactionConfigs, numCompactionTaskSlots == null ? null : 1.0, // 100% when numCompactionTaskSlots is not null numCompactionTaskSlots, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java index 3d441d9b06d..3056a75c564 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java @@ -28,9 +28,9 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.SqlSegmentsMetadataManager; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; @@ -107,13 +107,13 @@ public class KillCompactionConfigTest ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY)) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY)) ).thenReturn(null); Mockito.when(mockJacksonConfigManager.convertByteToConfig( ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty())) - ).thenReturn(CoordinatorCompactionConfig.empty()); + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty())) + ).thenReturn(DruidCompactionConfig.empty()); final MetadataCleanupConfig config = new MetadataCleanupConfig(true, new Duration("PT6S"), null); @@ -129,14 +129,14 @@ public class KillCompactionConfigTest Mockito.verify(mockJacksonConfigManager).convertByteToConfig( ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty()) ); Mockito.verify(mockConnector).lookup( ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY) ); Mockito.verifyNoMoreInteractions(mockJacksonConfigManager); } @@ -177,24 +177,26 @@ public class KillCompactionConfigTest null, ImmutableMap.of("key", "val") ); - CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig, activeDatasourceConfig)); + DruidCompactionConfig originalCurrentConfig = DruidCompactionConfig.empty().withDatasourceConfigs( + ImmutableList.of(inactiveDatasourceConfig, activeDatasourceConfig) + ); byte[] originalCurrentConfigBytes = {1, 2, 3}; Mockito.when(mockConnector.lookup( ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY)) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY)) ).thenReturn(originalCurrentConfigBytes); Mockito.when(mockJacksonConfigManager.convertByteToConfig( ArgumentMatchers.eq(originalCurrentConfigBytes), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty())) + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty())) ).thenReturn(originalCurrentConfig); Mockito.when(mockSqlSegmentsMetadataManager.retrieveAllDataSourceNames()).thenReturn(ImmutableSet.of(activeDatasourceName)); final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass(CoordinatorCompactionConfig.class); + final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass(DruidCompactionConfig.class); Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY), oldConfigCaptor.capture(), newConfigCaptor.capture(), ArgumentMatchers.any()) @@ -221,19 +223,19 @@ public class KillCompactionConfigTest Mockito.verify(mockJacksonConfigManager).convertByteToConfig( ArgumentMatchers.eq(originalCurrentConfigBytes), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty()) ); Mockito.verify(mockConnector).lookup( ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY) ); Mockito.verify(mockJacksonConfigManager).set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY), ArgumentMatchers.any(byte[].class), - ArgumentMatchers.any(CoordinatorCompactionConfig.class), + ArgumentMatchers.any(DruidCompactionConfig.class), ArgumentMatchers.any() ); Mockito.verifyNoMoreInteractions(mockJacksonConfigManager); @@ -261,24 +263,26 @@ public class KillCompactionConfigTest ImmutableMap.of("key", "val") ); - CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig)); + DruidCompactionConfig originalCurrentConfig = DruidCompactionConfig.empty().withDatasourceConfigs( + ImmutableList.of(inactiveDatasourceConfig) + ); byte[] originalCurrentConfigBytes = {1, 2, 3}; Mockito.when(mockConnector.lookup( ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY)) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY)) ).thenReturn(originalCurrentConfigBytes); Mockito.when(mockJacksonConfigManager.convertByteToConfig( ArgumentMatchers.eq(originalCurrentConfigBytes), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty())) + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty())) ).thenReturn(originalCurrentConfig); Mockito.when(mockSqlSegmentsMetadataManager.retrieveAllDataSourceNames()).thenReturn(ImmutableSet.of()); Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY), ArgumentMatchers.any(byte[].class), - ArgumentMatchers.any(CoordinatorCompactionConfig.class), + ArgumentMatchers.any(DruidCompactionConfig.class), ArgumentMatchers.any()) ).thenReturn( // Return fail result with RetryableException the first three calls to updated set @@ -304,21 +308,21 @@ public class KillCompactionConfigTest // Should call convertByteToConfig and lookup (to refresh current compaction config) four times due to RetryableException when failed Mockito.verify(mockJacksonConfigManager, Mockito.times(4)).convertByteToConfig( ArgumentMatchers.eq(originalCurrentConfigBytes), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) + ArgumentMatchers.eq(DruidCompactionConfig.class), + ArgumentMatchers.eq(DruidCompactionConfig.empty()) ); Mockito.verify(mockConnector, Mockito.times(4)).lookup( ArgumentMatchers.anyString(), ArgumentMatchers.eq("name"), ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY) ); // Should call set (to try set new updated compaction config) four times due to RetryableException when failed Mockito.verify(mockJacksonConfigManager, Mockito.times(4)).set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), + ArgumentMatchers.eq(DruidCompactionConfig.CONFIG_KEY), ArgumentMatchers.any(byte[].class), - ArgumentMatchers.any(CoordinatorCompactionConfig.class), + ArgumentMatchers.any(DruidCompactionConfig.class), ArgumentMatchers.any() ); Mockito.verifyNoMoreInteractions(mockJacksonConfigManager); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java index 9d0f752869e..f4e8cdb3cd4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java @@ -225,6 +225,151 @@ public class KillUnusedSegmentsTest validateLastKillStateAndReset(DS2, null); } + /** + * Set up multiple datasources {@link #DS1}, {@link #DS2} and {@link #DS3} with unused segments with 2 kill task + * slots. Running the kill duty each time should pick at least one unique datasource in a round-robin manner. + */ + @Test + public void testRoundRobinKillMultipleDatasources() + { + configBuilder.withIgnoreDurationToRetain(true) + .withMaxSegmentsToKill(2); + dynamicConfigBuilder.withMaxKillTaskSlots(2); + + createAndAddUnusedSegment(DS1, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, MONTH_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, DAY_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, NEXT_DAY, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, NEXT_MONTH, VERSION, NOW.minusDays(1)); + + createAndAddUnusedSegment(DS2, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS2, DAY_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS2, NEXT_DAY, VERSION, NOW.minusDays(1)); + + createAndAddUnusedSegment(DS3, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS3, DAY_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS3, NEXT_DAY, VERSION, NOW.minusDays(1)); + + initDuty(); + CoordinatorRunStats stats = runDutyAndGetStats(); + + Assert.assertEquals(2, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(2, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS2_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(4, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(4, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(4, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS3_STAT_KEY)); + Assert.assertEquals(4, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(6, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(6, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(6, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS2_STAT_KEY)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS3_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(8, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(7, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(8, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(5, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS2_STAT_KEY)); + } + + /** + * The set of datasources to kill change in consecutive runs. The kill duty should avoid selecting two + * consecutive datasources across runs as long as there are other datasources to kill. + */ + @Test + public void testRoundRobinKillWhenDatasourcesChange() + { + configBuilder.withIgnoreDurationToRetain(true) + .withMaxSegmentsToKill(2); + dynamicConfigBuilder.withMaxKillTaskSlots(1); + + createAndAddUnusedSegment(DS1, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, MONTH_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, DAY_OLD, VERSION, NOW.minusDays(1)); + + initDuty(); + CoordinatorRunStats stats = runDutyAndGetStats(); + + Assert.assertEquals(1, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(1, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(1, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + validateLastKillStateAndReset(DS1, new Interval(YEAR_OLD.getStart(), MONTH_OLD.getEnd())); + + createAndAddUnusedSegment(DS2, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS2, DAY_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS2, NEXT_DAY, VERSION, NOW.minusDays(1)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(2, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(2, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS2_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(3, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(3, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(4, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(4, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(4, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS2_STAT_KEY)); + } + + @Test + public void testKillSingleDatasourceMultipleRuns() + { + configBuilder.withIgnoreDurationToRetain(true) + .withMaxSegmentsToKill(2); + dynamicConfigBuilder.withMaxKillTaskSlots(2); + + createAndAddUnusedSegment(DS1, YEAR_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, MONTH_OLD, VERSION, NOW.minusDays(1)); + createAndAddUnusedSegment(DS1, DAY_OLD, VERSION, NOW.minusDays(1)); + + initDuty(); + CoordinatorRunStats stats = runDutyAndGetStats(); + + Assert.assertEquals(2, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(1, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(2, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(4, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(4, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(6, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(6, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + } + /** * The {@code DAY_OLD} and {@code HOUR_OLD} segments are "more recent" in terms of last updated time. * Even though they fall within the umbrella kill interval computed by the duty, the kill task will narrow down to @@ -407,6 +552,36 @@ public class KillUnusedSegmentsTest validateLastKillStateAndReset(DS1, YEAR_OLD); } + @Test + public void testKillDatasourceWithNoUnusedSegmentsInInitialRun() + { + configBuilder.withMaxSegmentsToKill(1); + + // create a datasource but no unused segments yet. + createAndAddUsedSegment(DS1, YEAR_OLD, VERSION); + + initDuty(); + CoordinatorRunStats stats = runDutyAndGetStats(); + + Assert.assertEquals(10, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(0, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(10, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(0, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + createAndAddUnusedSegment(DS1, YEAR_OLD, VERSION, NOW.minusDays(10)); + createAndAddUnusedSegment(DS1, MONTH_OLD, VERSION, NOW.minusDays(10)); + createAndAddUnusedSegment(DS1, DAY_OLD, VERSION, NOW.minusDays(2)); + + stats = runDutyAndGetStats(); + + Assert.assertEquals(20, stats.get(Stats.Kill.AVAILABLE_SLOTS)); + Assert.assertEquals(1, stats.get(Stats.Kill.SUBMITTED_TASKS)); + Assert.assertEquals(20, stats.get(Stats.Kill.MAX_SLOTS)); + Assert.assertEquals(1, stats.get(Stats.Kill.ELIGIBLE_UNUSED_SEGMENTS, DS1_STAT_KEY)); + + validateLastKillStateAndReset(DS1, YEAR_OLD); + } + /** * The kill period is honored after the first indexing run. */ @@ -723,12 +898,7 @@ public class KillUnusedSegmentsTest overlordClient.deleteLastKillInterval(dataSource); } - private void createAndAddUnusedSegment( - final String dataSource, - final Interval interval, - final String version, - final DateTime lastUpdatedTime - ) + private DataSegment createAndAddUsedSegment(final String dataSource, final Interval interval, final String version) { final DataSegment segment = createSegment(dataSource, interval, version); try { @@ -737,6 +907,17 @@ public class KillUnusedSegmentsTest catch (IOException e) { throw new RuntimeException(e); } + return segment; + } + + private void createAndAddUnusedSegment( + final String dataSource, + final Interval interval, + final String version, + final DateTime lastUpdatedTime + ) + { + final DataSegment segment = createAndAddUsedSegment(dataSource, interval, version); sqlSegmentsMetadataManager.markSegmentsAsUnused(ImmutableSet.of(segment.getId())); derbyConnectorRule.segments().updateUsedStatusLastUpdated(segment.getId().toString(), lastUpdatedTime); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index 7b6eb280a39..b32f8c055ca 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -37,9 +37,9 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.MetadataManager; import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; @@ -512,11 +512,11 @@ public class CoordinatorSimulationBuilder EasyMock.expect( jacksonConfigManager.watch( - EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY), - EasyMock.eq(CoordinatorCompactionConfig.class), + EasyMock.eq(DruidCompactionConfig.CONFIG_KEY), + EasyMock.eq(DruidCompactionConfig.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference<>(CoordinatorCompactionConfig.empty())).anyTimes(); + ).andReturn(new AtomicReference<>(DruidCompactionConfig.empty())).anyTimes(); return jacksonConfigManager; } diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index 17db2285477..c44b9b2f358 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -19,528 +19,529 @@ package org.apache.druid.server.http; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Suppliers; +import org.apache.druid.audit.AuditEntry; +import org.apache.druid.audit.AuditInfo; import org.apache.druid.audit.AuditManager; import org.apache.druid.client.indexing.ClientMSQContext; import org.apache.druid.common.config.ConfigManager; import org.apache.druid.common.config.JacksonConfigManager; -import org.apache.druid.error.DruidException; +import org.apache.druid.common.config.TestConfigManagerConfig; import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.CompactionEngine; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.metadata.MetadataStorageConnector; +import org.apache.druid.metadata.MetadataCASUpdate; import org.apache.druid.metadata.MetadataStorageTablesConfig; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; +import org.apache.druid.metadata.TestMetadataStorageConnector; +import org.apache.druid.metadata.TestMetadataStorageTablesConfig; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DataSourceCompactionConfigAuditEntry; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; +import org.joda.time.Interval; import org.joda.time.Period; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.ArgumentMatchers; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.MockitoJUnitRunner; +import org.skife.jdbi.v2.Handle; +import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; -import java.util.Collection; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.UnaryOperator; @RunWith(MockitoJUnitRunner.class) public class CoordinatorCompactionConfigsResourceTest { - private static final DataSourceCompactionConfig OLD_CONFIG = new DataSourceCompactionConfig( - "oldDataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); - private static final DataSourceCompactionConfig NEW_CONFIG = new DataSourceCompactionConfig( - "newDataSource", - null, - 500L, - null, - new Period(1800), - null, - new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); - private static final byte[] OLD_CONFIG_IN_BYTES = {1, 2, 3}; - - private static final CoordinatorCompactionConfig ORIGINAL_CONFIG - = CoordinatorCompactionConfig.from(ImmutableList.of(OLD_CONFIG)); - - private static final String DATASOURCE_NOT_EXISTS = "notExists"; - - @Mock - private JacksonConfigManager mockJacksonConfigManager; + private static final double DELTA = 1e-9; + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); @Mock private HttpServletRequest mockHttpServletRequest; - @Mock - private MetadataStorageConnector mockConnector; - - @Mock - private MetadataStorageTablesConfig mockConnectorConfig; - - @Mock - private AuditManager mockAuditManager; - - private CoordinatorCompactionConfigsResource coordinatorCompactionConfigsResource; + private TestCoordinatorConfigManager configManager; + private CoordinatorCompactionConfigsResource resource; @Before public void setup() { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(OLD_CONFIG_IN_BYTES); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(OLD_CONFIG_IN_BYTES), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(ORIGINAL_CONFIG); - Mockito.when(mockConnectorConfig.getConfigTable()).thenReturn("druid_config"); - Mockito.when(mockAuditManager.fetchAuditHistory( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - ArgumentMatchers.any() - ) - ).thenReturn(ImmutableList.of()); - coordinatorCompactionConfigsResource = new CoordinatorCompactionConfigsResource( - new CoordinatorConfigManager(mockJacksonConfigManager, mockConnector, mockConnectorConfig), - mockAuditManager - ); Mockito.when(mockHttpServletRequest.getRemoteAddr()).thenReturn("123"); + final AuditManager auditManager = new TestAuditManager(); + configManager = TestCoordinatorConfigManager.create(auditManager); + resource = new CoordinatorCompactionConfigsResource(configManager, auditManager); + configManager.delegate.start(); + } + + @After + public void tearDown() + { + configManager.delegate.stop(); } @Test - public void testSetCompactionTaskLimitWithExistingConfig() + public void testGetDefaultClusterConfig() { - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); + Response response = resource.getCompactionConfig(); + final DruidCompactionConfig defaultConfig + = verifyAndGetPayload(response, DruidCompactionConfig.class); - double compactionTaskSlotRatio = 0.5; - int maxCompactionTaskSlots = 9; - Response result = coordinatorCompactionConfigsResource.setCompactionTaskLimit( - compactionTaskSlotRatio, - maxCompactionTaskSlots, - true, + Assert.assertEquals(0.1, defaultConfig.getCompactionTaskSlotRatio(), DELTA); + Assert.assertEquals(Integer.MAX_VALUE, defaultConfig.getMaxCompactionTaskSlots()); + Assert.assertFalse(defaultConfig.isUseAutoScaleSlots()); + Assert.assertTrue(defaultConfig.getCompactionConfigs().isEmpty()); + Assert.assertEquals(CompactionEngine.NATIVE, defaultConfig.getEngine()); + } + + @Test + public void testUpdateGlobalConfig() + { + Response response = resource.updateClusterCompactionConfig( + new ClusterCompactionConfig(0.5, 10, true, CompactionEngine.MSQ), mockHttpServletRequest ); - Assert.assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); - Assert.assertNotNull(oldConfigCaptor.getValue()); - Assert.assertEquals(oldConfigCaptor.getValue(), OLD_CONFIG_IN_BYTES); - Assert.assertNotNull(newConfigCaptor.getValue()); - Assert.assertEquals(newConfigCaptor.getValue().getMaxCompactionTaskSlots(), maxCompactionTaskSlots); - Assert.assertTrue(newConfigCaptor.getValue().isUseAutoScaleSlots()); - Assert.assertEquals(compactionTaskSlotRatio, newConfigCaptor.getValue().getCompactionTaskSlotRatio(), 0); + verifyStatus(Response.Status.OK, response); + + final DruidCompactionConfig updatedConfig = verifyAndGetPayload( + resource.getCompactionConfig(), + DruidCompactionConfig.class + ); + + Assert.assertNotNull(updatedConfig); + Assert.assertEquals(0.5, updatedConfig.getCompactionTaskSlotRatio(), DELTA); + Assert.assertEquals(10, updatedConfig.getMaxCompactionTaskSlots()); + Assert.assertTrue(updatedConfig.isUseAutoScaleSlots()); + Assert.assertEquals(CompactionEngine.MSQ, updatedConfig.getEngine()); } @Test - public void testAddOrUpdateCompactionConfigWithExistingConfig() + public void testSetCompactionTaskLimit() { - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); + final DruidCompactionConfig defaultConfig + = verifyAndGetPayload(resource.getCompactionConfig(), DruidCompactionConfig.class); - final DataSourceCompactionConfig newConfig = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, true), - null, - null, - null, - null, - CompactionEngine.NATIVE, - ImmutableMap.of("key", "val") - ); - Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - newConfig, - mockHttpServletRequest - ); - Assert.assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); - Assert.assertNotNull(oldConfigCaptor.getValue()); - Assert.assertEquals(oldConfigCaptor.getValue(), OLD_CONFIG_IN_BYTES); - Assert.assertNotNull(newConfigCaptor.getValue()); - Assert.assertEquals(2, newConfigCaptor.getValue().getCompactionConfigs().size()); - Assert.assertEquals(OLD_CONFIG, newConfigCaptor.getValue().getCompactionConfigs().get(0)); - Assert.assertEquals(newConfig, newConfigCaptor.getValue().getCompactionConfigs().get(1)); - Assert.assertEquals(newConfig.getEngine(), newConfigCaptor.getValue().getEngine()); + Response response = resource.setCompactionTaskLimit(0.5, 9, true, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); + + final DruidCompactionConfig updatedConfig + = verifyAndGetPayload(resource.getCompactionConfig(), DruidCompactionConfig.class); + + // Verify that the task slot fields have been updated + Assert.assertEquals(0.5, updatedConfig.getCompactionTaskSlotRatio(), DELTA); + Assert.assertEquals(9, updatedConfig.getMaxCompactionTaskSlots()); + Assert.assertTrue(updatedConfig.isUseAutoScaleSlots()); + + // Verify that the other fields are unchanged + Assert.assertEquals(defaultConfig.getCompactionConfigs(), updatedConfig.getCompactionConfigs()); + Assert.assertEquals(defaultConfig.getEngine(), updatedConfig.getEngine()); } @Test - public void testDeleteCompactionConfigWithExistingConfig() + public void testGetUnknownDatasourceConfigThrowsNotFound() { - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); - final String datasourceName = "dataSource"; - final DataSourceCompactionConfig toDelete = new DataSourceCompactionConfig( - datasourceName, - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); - final CoordinatorCompactionConfig originalConfig = CoordinatorCompactionConfig.from(ImmutableList.of(toDelete)); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(OLD_CONFIG_IN_BYTES), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(originalConfig); - - Response result = coordinatorCompactionConfigsResource.deleteCompactionConfig( - datasourceName, - mockHttpServletRequest - ); - Assert.assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); - Assert.assertNotNull(oldConfigCaptor.getValue()); - Assert.assertEquals(oldConfigCaptor.getValue(), OLD_CONFIG_IN_BYTES); - Assert.assertNotNull(newConfigCaptor.getValue()); - Assert.assertEquals(0, newConfigCaptor.getValue().getCompactionConfigs().size()); + Response response = resource.getDatasourceCompactionConfig(DS.WIKI); + verifyStatus(Response.Status.NOT_FOUND, response); } @Test - public void testUpdateShouldRetryIfRetryableException() + public void testAddDatasourceConfig() { - Mockito.when( - mockJacksonConfigManager.set( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() + final DataSourceCompactionConfig newDatasourceConfig + = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(); + Response response = resource.addOrUpdateDatasourceCompactionConfig(newDatasourceConfig, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); + + final DataSourceCompactionConfig fetchedDatasourceConfig + = verifyAndGetPayload(resource.getDatasourceCompactionConfig(DS.WIKI), DataSourceCompactionConfig.class); + Assert.assertEquals(newDatasourceConfig, fetchedDatasourceConfig); + + final DruidCompactionConfig fullCompactionConfig + = verifyAndGetPayload(resource.getCompactionConfig(), DruidCompactionConfig.class); + Assert.assertEquals(1, fullCompactionConfig.getCompactionConfigs().size()); + Assert.assertEquals(newDatasourceConfig, fullCompactionConfig.getCompactionConfigs().get(0)); + } + + @Test + public void testUpdateDatasourceConfig() + { + final DataSourceCompactionConfig originalDatasourceConfig = DataSourceCompactionConfig + .builder() + .forDataSource(DS.WIKI) + .withInputSegmentSizeBytes(500L) + .withSkipOffsetFromLatest(Period.hours(1)) + .withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, true) ) - ).thenReturn(ConfigManager.SetResult.retryableFailure(new ISE("retryable"))); + .withEngine(CompactionEngine.NATIVE) + .build(); - coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - NEW_CONFIG, + Response response = resource.addOrUpdateDatasourceCompactionConfig( + originalDatasourceConfig, mockHttpServletRequest ); + verifyStatus(Response.Status.OK, response); - // Verify that the update is retried upto the max number of retries - Mockito.verify( - mockJacksonConfigManager, - Mockito.times(CoordinatorCompactionConfigsResource.UPDATE_NUM_RETRY) - ).set( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); - } - - @Test - public void testUpdateShouldNotRetryIfNotRetryableException() - { - Mockito.when( - mockJacksonConfigManager.set( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() + final DataSourceCompactionConfig updatedDatasourceConfig = DataSourceCompactionConfig + .builder() + .forDataSource(DS.WIKI) + .withInputSegmentSizeBytes(1000L) + .withSkipOffsetFromLatest(Period.hours(3)) + .withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.DAY, null, true) ) - ).thenReturn(ConfigManager.SetResult.failure(new ISE("retryable"))); + .withEngine(CompactionEngine.MSQ) + .build(); - coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - NEW_CONFIG, - mockHttpServletRequest - ); + response = resource.addOrUpdateDatasourceCompactionConfig(updatedDatasourceConfig, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); - // Verify that the update is tried only once - Mockito.verify(mockJacksonConfigManager, Mockito.times(1)).set( - ArgumentMatchers.anyString(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any() - ); + final DataSourceCompactionConfig latestDatasourceConfig + = verifyAndGetPayload(resource.getDatasourceCompactionConfig(DS.WIKI), DataSourceCompactionConfig.class); + Assert.assertEquals(updatedDatasourceConfig, latestDatasourceConfig); + + final DruidCompactionConfig fullCompactionConfig + = verifyAndGetPayload(resource.getCompactionConfig(), DruidCompactionConfig.class); + Assert.assertEquals(1, fullCompactionConfig.getCompactionConfigs().size()); + Assert.assertEquals(updatedDatasourceConfig, fullCompactionConfig.getCompactionConfigs().get(0)); } @Test - public void testSetCompactionTaskLimitWithoutExistingConfig() + public void testDeleteDatasourceConfig() { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(null); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(CoordinatorCompactionConfig.empty()); - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); + final DataSourceCompactionConfig datasourceConfig + = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(); + Response response = resource.addOrUpdateDatasourceCompactionConfig(datasourceConfig, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); - double compactionTaskSlotRatio = 0.5; - int maxCompactionTaskSlots = 9; - Response result = coordinatorCompactionConfigsResource.setCompactionTaskLimit( - compactionTaskSlotRatio, - maxCompactionTaskSlots, - true, - mockHttpServletRequest - ); - Assert.assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); - Assert.assertNull(oldConfigCaptor.getValue()); - Assert.assertNotNull(newConfigCaptor.getValue()); - Assert.assertEquals(newConfigCaptor.getValue().getMaxCompactionTaskSlots(), maxCompactionTaskSlots); - Assert.assertTrue(newConfigCaptor.getValue().isUseAutoScaleSlots()); - Assert.assertEquals(compactionTaskSlotRatio, newConfigCaptor.getValue().getCompactionTaskSlotRatio(), 0); + response = resource.deleteCompactionConfig(DS.WIKI, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); + + response = resource.getDatasourceCompactionConfig(DS.WIKI); + verifyStatus(Response.Status.NOT_FOUND, response); } @Test - public void testAddOrUpdateCompactionConfigWithoutExistingConfig() + public void testDeleteUnknownDatasourceConfigThrowsNotFound() { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(null); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(CoordinatorCompactionConfig.empty()); - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); - - final DataSourceCompactionConfig newConfig = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - CompactionEngine.MSQ, - ImmutableMap.of("key", "val") - ); - String author = "maytas"; - String comment = "hello"; - Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - newConfig, - mockHttpServletRequest - ); - Assert.assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); - Assert.assertNull(oldConfigCaptor.getValue()); - Assert.assertNotNull(newConfigCaptor.getValue()); - Assert.assertEquals(1, newConfigCaptor.getValue().getCompactionConfigs().size()); - Assert.assertEquals(newConfig, newConfigCaptor.getValue().getCompactionConfigs().get(0)); - Assert.assertEquals(newConfig.getEngine(), newConfigCaptor.getValue().getCompactionConfigs().get(0).getEngine()); + Response response = resource.deleteCompactionConfig(DS.WIKI, mockHttpServletRequest); + verifyStatus(Response.Status.NOT_FOUND, response); } @Test - public void testAddOrUpdateCompactionConfigWithoutExistingConfigAndEngineAsNull() + public void testUpdateIsRetriedIfFailureIsRetryable() { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(null); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(CoordinatorCompactionConfig.empty()); - final ArgumentCaptor oldConfigCaptor = ArgumentCaptor.forClass(byte[].class); - final ArgumentCaptor newConfigCaptor = ArgumentCaptor.forClass( - CoordinatorCompactionConfig.class); - Mockito.when(mockJacksonConfigManager.set( - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY), - oldConfigCaptor.capture(), - newConfigCaptor.capture(), - ArgumentMatchers.any() - ) - ).thenReturn(ConfigManager.SetResult.ok()); - - final DataSourceCompactionConfig newConfig = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - null, - ImmutableMap.of("key", "val") - ); - coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - newConfig, + configManager.configUpdateResult + = ConfigManager.SetResult.retryableFailure(new Exception("retryable")); + resource.addOrUpdateDatasourceCompactionConfig( + DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(), mockHttpServletRequest ); - Assert.assertEquals(null, newConfigCaptor.getValue().getCompactionConfigs().get(0).getEngine()); - } - @Test - public void testAddOrUpdateCompactionConfigWithInvalidMaxNumTasksForMSQEngine() - { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(null); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(CoordinatorCompactionConfig.empty()); - - int maxNumTasks = 1; - - final DataSourceCompactionConfig newConfig = new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null), - null, - null, - null, - null, - CompactionEngine.MSQ, - ImmutableMap.of(ClientMSQContext.CTX_MAX_NUM_TASKS, maxNumTasks) - ); - Response response = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( - newConfig, - mockHttpServletRequest - ); - Assert.assertEquals(DruidException.Category.INVALID_INPUT.getExpectedStatus(), response.getStatus()); Assert.assertEquals( - "Compaction config not supported. Reason[MSQ context maxNumTasks [1] cannot be less than 2, " - + "since at least 1 controller and 1 worker is necessary.].", + CoordinatorCompactionConfigsResource.MAX_UPDATE_RETRIES, + configManager.numUpdateAttempts + ); + } + + @Test + public void testUpdateIsNotRetriedIfFailureIsNotRetryable() + { + configManager.configUpdateResult + = ConfigManager.SetResult.failure(new Exception("not retryable")); + resource.addOrUpdateDatasourceCompactionConfig( + DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(), + mockHttpServletRequest + ); + + Assert.assertEquals(1, configManager.numUpdateAttempts); + } + + @Test + public void testGetDatasourceConfigHistory() + { + final DataSourceCompactionConfig.Builder builder + = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI); + + final DataSourceCompactionConfig configV1 = builder.build(); + resource.addOrUpdateDatasourceCompactionConfig(configV1, mockHttpServletRequest); + + final DataSourceCompactionConfig configV2 = builder.withEngine(CompactionEngine.NATIVE).build(); + resource.addOrUpdateDatasourceCompactionConfig(configV2, mockHttpServletRequest); + + final DataSourceCompactionConfig configV3 = builder + .withEngine(CompactionEngine.MSQ) + .withSkipOffsetFromLatest(Period.hours(1)) + .build(); + resource.addOrUpdateDatasourceCompactionConfig(configV3, mockHttpServletRequest); + + Response response = resource.getCompactionConfigHistory(DS.WIKI, null, null); + verifyStatus(Response.Status.OK, response); + + final List history + = (List) response.getEntity(); + Assert.assertEquals(3, history.size()); + Assert.assertEquals(configV1, history.get(0).getCompactionConfig()); + Assert.assertEquals(configV2, history.get(1).getCompactionConfig()); + Assert.assertEquals(configV3, history.get(2).getCompactionConfig()); + } + + @Test + public void testGetHistoryOfUnknownDatasourceReturnsEmpty() + { + Response response = resource.getCompactionConfigHistory(DS.WIKI, null, null); + verifyStatus(Response.Status.OK, response); + Assert.assertTrue(((List) response.getEntity()).isEmpty()); + } + + @Test + public void testAddInvalidDatasourceConfigThrowsBadRequest() + { + final DataSourceCompactionConfig datasourceConfig = DataSourceCompactionConfig + .builder() + .forDataSource(DS.WIKI) + .withTaskContext(Collections.singletonMap(ClientMSQContext.CTX_MAX_NUM_TASKS, 1)) + .withEngine(CompactionEngine.MSQ) + .build(); + + final Response response = resource.addOrUpdateDatasourceCompactionConfig(datasourceConfig, mockHttpServletRequest); + verifyStatus(Response.Status.BAD_REQUEST, response); + Assert.assertTrue(response.getEntity() instanceof ErrorResponse); + Assert.assertEquals( + "Compaction config not supported. Reason[MSQ: Context maxNumTasks[1]" + + " must be at least 2 (1 controller + 1 worker)].", ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage() ); } @Test - public void testDeleteCompactionConfigWithoutExistingConfigShouldFailAsDatasourceNotExist() + public void testUpdateEngineToMSQWithInvalidDatasourceConfigThrowsBadRequest() { - Mockito.when(mockConnector.lookup( - ArgumentMatchers.anyString(), - ArgumentMatchers.eq("name"), - ArgumentMatchers.eq("payload"), - ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY) - ) - ).thenReturn(null); - Mockito.when(mockJacksonConfigManager.convertByteToConfig( - ArgumentMatchers.eq(null), - ArgumentMatchers.eq(CoordinatorCompactionConfig.class), - ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()) - ) - ).thenReturn(CoordinatorCompactionConfig.empty()); - Response result = coordinatorCompactionConfigsResource.deleteCompactionConfig( - DATASOURCE_NOT_EXISTS, + final DataSourceCompactionConfig datasourceConfig = DataSourceCompactionConfig + .builder() + .forDataSource(DS.WIKI) + .withTaskContext(Collections.singletonMap(ClientMSQContext.CTX_MAX_NUM_TASKS, 1)) + .build(); + Response response = resource.addOrUpdateDatasourceCompactionConfig(datasourceConfig, mockHttpServletRequest); + verifyStatus(Response.Status.OK, response); + + response = resource.updateClusterCompactionConfig( + new ClusterCompactionConfig(null, null, null, CompactionEngine.MSQ), mockHttpServletRequest ); - Assert.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), result.getStatus()); + verifyStatus(Response.Status.BAD_REQUEST, response); + Assert.assertTrue(response.getEntity() instanceof ErrorResponse); + Assert.assertEquals( + "Cannot update engine to [msq] as it does not support compaction config of DataSource[wiki]." + + " Reason[MSQ: Context maxNumTasks[1] must be at least 2 (1 controller + 1 worker)].", + ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage() + ); } - @Test - public void testGetCompactionConfigHistoryForUnknownDataSourceShouldReturnEmptyList() + @SuppressWarnings("unchecked") + private T verifyAndGetPayload(Response response, Class type) { - Response response = coordinatorCompactionConfigsResource.getCompactionConfigHistory( - DATASOURCE_NOT_EXISTS, - null, - null - ); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertTrue(((Collection) response.getEntity()).isEmpty()); + + Assert.assertTrue(type.isInstance(response.getEntity())); + return (T) response.getEntity(); + } + + private void verifyStatus(Response.Status expectedStatus, Response response) + { + Assert.assertEquals(expectedStatus.getStatusCode(), response.getStatus()); + } + + /** + * Test implementation of AuditManager that keeps audit entries in memory. + */ + private static class TestAuditManager implements AuditManager + { + private final List audits = new ArrayList<>(); + + @Override + public void doAudit(AuditEntry event, Handle handle) + { + // do nothing + } + + @Override + public void doAudit(AuditEntry event) + { + final String json; + try { + json = OBJECT_MAPPER.writeValueAsString(event.getPayload().raw()); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + + final AuditEntry eventWithSerializedPayload + = AuditEntry.builder() + .key(event.getKey()) + .type(event.getType()) + .auditInfo(event.getAuditInfo()) + .auditTime(event.getAuditTime()) + .request(event.getRequest()) + .serializedPayload(json) + .build(); + audits.add(eventWithSerializedPayload); + } + + @Override + public List fetchAuditHistory(String key, String type, Interval interval) + { + return audits; + } + + @Override + public List fetchAuditHistory(String type, int limit) + { + return audits; + } + + @Override + public List fetchAuditHistory(String type, Interval interval) + { + return audits; + } + + @Override + public List fetchAuditHistory(String key, String type, int limit) + { + return audits; + } + + @Override + public int removeAuditLogsOlderThan(long timestamp) + { + return 0; + } + } + + /** + * Test implementation of CoordinatorConfigManager to track number of update attempts. + */ + private static class TestCoordinatorConfigManager extends CoordinatorConfigManager + { + private final ConfigManager delegate; + private int numUpdateAttempts; + private ConfigManager.SetResult configUpdateResult; + + static TestCoordinatorConfigManager create(AuditManager auditManager) + { + final MetadataStorageTablesConfig tablesConfig = new TestMetadataStorageTablesConfig() + { + @Override + public String getConfigTable() + { + return "druid_config"; + } + }; + + final TestDBConnector dbConnector = new TestDBConnector(); + final ConfigManager configManager = new ConfigManager( + dbConnector, + Suppliers.ofInstance(tablesConfig), + Suppliers.ofInstance(new TestConfigManagerConfig()) + ); + + return new TestCoordinatorConfigManager(configManager, dbConnector, tablesConfig, auditManager); + } + + TestCoordinatorConfigManager( + ConfigManager configManager, + TestDBConnector dbConnector, + MetadataStorageTablesConfig tablesConfig, + AuditManager auditManager + ) + { + super( + new JacksonConfigManager(configManager, OBJECT_MAPPER, auditManager), + dbConnector, + tablesConfig + ); + this.delegate = configManager; + } + + @Override + public ConfigManager.SetResult getAndUpdateCompactionConfig( + UnaryOperator operator, + AuditInfo auditInfo + ) + { + ++numUpdateAttempts; + if (configUpdateResult == null) { + return super.getAndUpdateCompactionConfig(operator, auditInfo); + } else { + return configUpdateResult; + } + } + } + + /** + * Test implementation for in-memory insert, lookup and compareAndSwap operations. + */ + private static class TestDBConnector extends TestMetadataStorageConnector + { + private final Map, byte[]> values = new HashMap<>(); + + @Override + public Void insertOrUpdate(String tableName, String keyColumn, String valueColumn, String key, byte[] value) + { + values.put( + Arrays.asList(tableName, keyColumn, valueColumn, key), + value + ); + return null; + } + + @Nullable + @Override + public byte[] lookup(String tableName, String keyColumn, String valueColumn, String key) + { + return values.get(Arrays.asList(tableName, keyColumn, valueColumn, key)); + } + + @Override + public boolean compareAndSwap(List updates) + { + for (MetadataCASUpdate update : updates) { + final List key = Arrays.asList( + update.getTableName(), + update.getKeyColumn(), + update.getValueColumn(), + update.getKey() + ); + + final byte[] currentValue = values.get(key); + if (currentValue == null || Arrays.equals(currentValue, update.getOldValue())) { + values.put(key, update.getNewValue()); + } else { + return false; + } + } + + return true; + } + } + + private static class DS + { + static final String WIKI = "wiki"; } } diff --git a/services/src/main/java/org/apache/druid/cli/CliIndexer.java b/services/src/main/java/org/apache/druid/cli/CliIndexer.java index 312b6f6b05a..5decef93018 100644 --- a/services/src/main/java/org/apache/druid/cli/CliIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliIndexer.java @@ -36,7 +36,6 @@ import org.apache.druid.discovery.NodeRole; import org.apache.druid.discovery.WorkerNodeService; import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.IndexerServiceModule; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.guice.IndexingServiceTaskLogsModule; @@ -238,7 +237,6 @@ public class CliIndexer extends ServerRunnable } }, new ShuffleModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), diff --git a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java index 37b6501bf87..2e542bc6974 100644 --- a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java @@ -36,7 +36,6 @@ import com.google.inject.util.Providers; import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; import org.apache.druid.discovery.WorkerNodeService; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.guice.IndexingServiceTaskLogsModule; @@ -73,10 +72,10 @@ import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; @@ -242,7 +241,6 @@ public class CliMiddleManager extends ServerRunnable } }, new ShuffleModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 4dd23f2faf2..636d775f030 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -39,7 +39,6 @@ import com.google.inject.servlet.GuiceFilter; import com.google.inject.util.Providers; import org.apache.druid.client.indexing.IndexingService; import org.apache.druid.discovery.NodeRole; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.guice.IndexingServiceTaskLogsModule; @@ -67,6 +66,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervi import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; import org.apache.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; +import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.ForkingTaskRunnerFactory; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; @@ -107,10 +107,10 @@ import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.http.RedirectFilter; import org.apache.druid.server.http.RedirectInfo; @@ -210,6 +210,7 @@ public class CliOverlord extends ServerRunnable JsonConfigProvider.bind(binder, "druid.indexer.task.default", DefaultTaskConfig.class); binder.bind(RetryPolicyFactory.class).in(LazySingleton.class); + binder.bind(DruidOverlord.class).in(ManageLifecycle.class); binder.bind(TaskMaster.class).in(ManageLifecycle.class); binder.bind(TaskCountStatsProvider.class).to(TaskMaster.class); binder.bind(TaskSlotCountStatsProvider.class).to(TaskMaster.class); @@ -382,11 +383,11 @@ public class CliOverlord extends ServerRunnable @Provides @LazySingleton @Named(ServiceStatusMonitor.HEARTBEAT_TAGS_BINDING) - public Supplier> getHeartbeatSupplier(TaskMaster taskMaster) + public Supplier> getHeartbeatSupplier(DruidOverlord overlord) { return () -> { Map heartbeatTags = new HashMap<>(); - heartbeatTags.put("leader", taskMaster.isLeader() ? 1 : 0); + heartbeatTags.put("leader", overlord.isLeader() ? 1 : 0); return heartbeatTags; }; @@ -426,7 +427,6 @@ public class CliOverlord extends ServerRunnable .to(TaskLogAutoCleaner.class); } }, - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 5edb83d0a3b..eb572850cda 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -48,7 +48,6 @@ import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Binders; import org.apache.druid.guice.CacheModule; import org.apache.druid.guice.DruidProcessingModule; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceTaskLogsModule; import org.apache.druid.guice.IndexingServiceTuningConfigModule; @@ -116,11 +115,11 @@ import org.apache.druid.segment.loading.OmniDataSegmentKiller; import org.apache.druid.segment.loading.OmniDataSegmentMover; import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.ServiceAnnouncingChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.PeonAppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.ResponseContextConfig; import org.apache.druid.server.SegmentManager; @@ -343,7 +342,6 @@ public class CliPeon extends GuiceRunnable } }, new QueryablePeonModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTuningConfigModule(), new InputSourceModule(), diff --git a/services/src/main/java/org/apache/druid/cli/CoordinatorOverlordRedirectInfo.java b/services/src/main/java/org/apache/druid/cli/CoordinatorOverlordRedirectInfo.java index 60415ea4f32..0f77da1b4c4 100644 --- a/services/src/main/java/org/apache/druid/cli/CoordinatorOverlordRedirectInfo.java +++ b/services/src/main/java/org/apache/druid/cli/CoordinatorOverlordRedirectInfo.java @@ -20,7 +20,7 @@ package org.apache.druid.cli; import com.google.inject.Inject; -import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.http.OverlordRedirectInfo; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.http.CoordinatorRedirectInfo; @@ -36,9 +36,9 @@ public class CoordinatorOverlordRedirectInfo implements RedirectInfo private final CoordinatorRedirectInfo coordinatorRedirectInfo; @Inject - public CoordinatorOverlordRedirectInfo(TaskMaster taskMaster, DruidCoordinator druidCoordinator) + public CoordinatorOverlordRedirectInfo(DruidOverlord druidOverlord, DruidCoordinator druidCoordinator) { - this.overlordRedirectInfo = new OverlordRedirectInfo(taskMaster); + this.overlordRedirectInfo = new OverlordRedirectInfo(druidOverlord); this.coordinatorRedirectInfo = new CoordinatorRedirectInfo(druidCoordinator); } diff --git a/services/src/main/java/org/apache/druid/cli/validate/DruidJsonValidator.java b/services/src/main/java/org/apache/druid/cli/validate/DruidJsonValidator.java index 0518c7259e6..50684cee0e8 100644 --- a/services/src/main/java/org/apache/druid/cli/validate/DruidJsonValidator.java +++ b/services/src/main/java/org/apache/druid/cli/validate/DruidJsonValidator.java @@ -40,8 +40,6 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.ExtensionsLoader; -import org.apache.druid.guice.FirehoseModule; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.LocalDataStorageDruidModule; import org.apache.druid.guice.QueryRunnerFactoryModule; @@ -130,9 +128,7 @@ public class DruidJsonValidator extends GuiceRunnable Iterables.concat( extnLoader.getModules(), Arrays.asList( - new FirehoseModule(), new IndexingHadoopModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new LocalDataStorageDruidModule() ) diff --git a/services/src/test/java/org/apache/druid/cli/CoordinatorOverlordRedirectInfoTest.java b/services/src/test/java/org/apache/druid/cli/CoordinatorOverlordRedirectInfoTest.java new file mode 100644 index 00000000000..d33c05838a2 --- /dev/null +++ b/services/src/test/java/org/apache/druid/cli/CoordinatorOverlordRedirectInfoTest.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.cli; + +import org.apache.druid.indexing.overlord.DruidOverlord; +import org.apache.druid.server.coordinator.DruidCoordinator; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class CoordinatorOverlordRedirectInfoTest +{ + private DruidOverlord overlord; + private DruidCoordinator coordinator; + private CoordinatorOverlordRedirectInfo redirectInfo; + + @Before + public void setUp() + { + overlord = EasyMock.createMock(DruidOverlord.class); + coordinator = EasyMock.createMock(DruidCoordinator.class); + redirectInfo = new CoordinatorOverlordRedirectInfo(overlord, coordinator); + } + + @Test + public void testDoLocalIndexerWhenLeading() + { + EasyMock.expect(overlord.isLeader()).andReturn(true).anyTimes(); + EasyMock.replay(overlord); + Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/leader")); + Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/isLeader")); + Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/other/path")); + EasyMock.verify(overlord); + } +} diff --git a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java index b843465147e..b617b7e6b87 100644 --- a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java @@ -142,7 +142,6 @@ public class DruidJsonValidatorTest jsonMapper ), new IndexTask.IndexIOConfig( - null, new LocalInputSource(new File("lol"), "rofl"), new JsonInputFormat(null, null, null, null, null), false, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java index eceb4ebbf80..7d66eebcad1 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java @@ -21,10 +21,7 @@ package org.apache.druid.sql.calcite.aggregation; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.sql.SqlAggFunction; -import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.type.InferTypes; -import org.apache.calcite.sql.type.OperandTypes; import org.apache.calcite.sql.type.ReturnTypes; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.Optionality; @@ -44,20 +41,20 @@ import java.util.List; */ public class ApproxCountDistinctSqlAggregator implements SqlAggregator { - private static final SqlAggFunction FUNCTION_INSTANCE = new ApproxCountDistinctSqlAggFunction(); private static final String NAME = "APPROX_COUNT_DISTINCT"; - + private final SqlAggFunction delegateFunction; private final SqlAggregator delegate; public ApproxCountDistinctSqlAggregator(final SqlAggregator delegate) { this.delegate = delegate; + this.delegateFunction = new ApproxCountDistinctSqlAggFunction(delegate.calciteFunction()); } @Override public SqlAggFunction calciteFunction() { - return FUNCTION_INSTANCE; + return delegateFunction; } @Nullable @@ -83,18 +80,19 @@ public class ApproxCountDistinctSqlAggregator implements SqlAggregator ); } + @NativelySupportsDistinct private static class ApproxCountDistinctSqlAggFunction extends SqlAggFunction { - ApproxCountDistinctSqlAggFunction() + ApproxCountDistinctSqlAggFunction(SqlAggFunction delegate) { super( NAME, null, SqlKind.OTHER_FUNCTION, ReturnTypes.explicit(SqlTypeName.BIGINT), - InferTypes.VARCHAR_1024, - OperandTypes.ANY, - SqlFunctionCategory.STRING, + delegate.getOperandTypeInference(), + delegate.getOperandTypeChecker(), + delegate.getFunctionType(), false, false, Optionality.FORBIDDEN diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/NativelySupportsDistinct.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/NativelySupportsDistinct.java new file mode 100644 index 00000000000..19bbaf8a0f2 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/NativelySupportsDistinct.java @@ -0,0 +1,36 @@ +/* + * 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.aggregation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * This annotation is to distinguish {@link org.apache.calcite.sql.SqlAggFunction} + * which supports the distinct aggregation natively + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE}) +public @interface NativelySupportsDistinct +{ + +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java index a5e62f5e2a9..d20999d3afc 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java @@ -39,6 +39,7 @@ import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.aggregation.Aggregation; +import org.apache.druid.sql.calcite.aggregation.NativelySupportsDistinct; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; @@ -142,6 +143,7 @@ public class ArrayConcatSqlAggregator implements SqlAggregator } } + @NativelySupportsDistinct private static class ArrayConcatAggFunction extends SqlAggFunction { ArrayConcatAggFunction() diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java index efb84dca625..1045a79870b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java @@ -41,6 +41,7 @@ import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.aggregation.Aggregation; +import org.apache.druid.sql.calcite.aggregation.NativelySupportsDistinct; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; @@ -165,6 +166,7 @@ public class ArraySqlAggregator implements SqlAggregator } } + @NativelySupportsDistinct private static class ArrayAggFunction extends SqlAggFunction { private static final ArrayAggReturnTypeInference RETURN_TYPE_INFERENCE = new ArrayAggReturnTypeInference(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BuiltinApproxCountDistinctSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BuiltinApproxCountDistinctSqlAggregator.java index 699c7a8d1c6..c756aa64cc3 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BuiltinApproxCountDistinctSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BuiltinApproxCountDistinctSqlAggregator.java @@ -46,13 +46,16 @@ import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; import org.apache.druid.sql.calcite.planner.Calcites; +import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.InputAccessor; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; +import org.apache.druid.sql.calcite.table.RowSignatures; import javax.annotation.Nullable; import java.util.Collections; import java.util.List; +import java.util.Objects; public class BuiltinApproxCountDistinctSqlAggregator implements SqlAggregator { @@ -94,7 +97,7 @@ public class BuiltinApproxCountDistinctSqlAggregator implements SqlAggregator if (arg.isDirectColumnAccess() && inputAccessor.getInputRowSignature() .getColumnType(arg.getDirectColumn()) - .map(type -> type.is(ValueType.COMPLEX)) + .map(this::isValidComplexInputType) .orElse(false)) { aggregatorFactory = new HyperUniquesAggregatorFactory(aggregatorName, arg.getDirectColumn(), false, true); } else { @@ -118,6 +121,15 @@ public class BuiltinApproxCountDistinctSqlAggregator implements SqlAggregator } if (inputType.is(ValueType.COMPLEX)) { + if (!isValidComplexInputType(inputType)) { + plannerContext.setPlanningError( + "Using APPROX_COUNT_DISTINCT() or enabling approximation with COUNT(DISTINCT) is not supported for" + + " column type [%s]. You can disable approximation by setting [%s: false] in the query context.", + arg.getDruidType(), + PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT + ); + return null; + } aggregatorFactory = new HyperUniquesAggregatorFactory( aggregatorName, dimensionSpec.getOutputName(), @@ -151,7 +163,11 @@ public class BuiltinApproxCountDistinctSqlAggregator implements SqlAggregator SqlKind.OTHER_FUNCTION, ReturnTypes.explicit(SqlTypeName.BIGINT), InferTypes.VARCHAR_1024, - OperandTypes.ANY, + OperandTypes.or( + OperandTypes.STRING, + OperandTypes.NUMERIC, + RowSignatures.complexTypeChecker(HyperUniquesAggregatorFactory.TYPE) + ), SqlFunctionCategory.STRING, false, false, @@ -159,4 +175,10 @@ public class BuiltinApproxCountDistinctSqlAggregator implements SqlAggregator ); } } + + private boolean isValidComplexInputType(ColumnType columnType) + { + return Objects.equals(columnType.getComplexTypeName(), HyperUniquesAggregatorFactory.TYPE.getComplexTypeName()) || + Objects.equals(columnType.getComplexTypeName(), HyperUniquesAggregatorFactory.PRECOMPUTED_TYPE.getComplexTypeName()); + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java index a78b3a7a479..49469decf99 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java @@ -47,6 +47,7 @@ import org.apache.druid.query.filter.NullFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.aggregation.Aggregation; +import org.apache.druid.sql.calcite.aggregation.NativelySupportsDistinct; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; @@ -226,6 +227,7 @@ public class StringSqlAggregator implements SqlAggregator } } + @NativelySupportsDistinct private static class StringAggFunction extends SqlAggFunction { private static final StringAggReturnTypeInference RETURN_TYPE_INFERENCE = new StringAggReturnTypeInference(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java index ec518ee4522..689db77c0fa 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java @@ -28,6 +28,7 @@ import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexOver; import org.apache.calcite.rex.RexUtil; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlOperator; @@ -36,6 +37,7 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExpressionType; @@ -66,6 +68,7 @@ import org.apache.druid.sql.calcite.filtration.Ranges; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.ExpressionParser; import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.CannotBuildQueryException; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; import org.apache.druid.sql.calcite.table.RowSignatures; import org.joda.time.Interval; @@ -238,6 +241,10 @@ public class Expressions final SqlKind kind = rexNode.getKind(); if (kind == SqlKind.INPUT_REF) { return inputRefToDruidExpression(rowSignature, rexNode); + } else if (rexNode instanceof RexOver) { + throw new CannotBuildQueryException( + StringUtils.format("Unexpected OVER expression during translation [%s]", rexNode) + ); } else if (rexNode instanceof RexCall) { return rexCallToDruidExpression(plannerContext, rowSignature, rexNode, postAggregatorVisitor); } else if (kind == SqlKind.LITERAL) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java index 75778daf559..857c8cb0d12 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java @@ -28,6 +28,7 @@ import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rel.type.RelRecordType; import org.apache.calcite.runtime.CalciteContextException; import org.apache.calcite.runtime.CalciteException; +import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlInsert; @@ -35,7 +36,9 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlOperatorTable; +import org.apache.calcite.sql.SqlOverOperator; import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.SqlSelectKeyword; import org.apache.calcite.sql.SqlUpdate; import org.apache.calcite.sql.SqlUtil; import org.apache.calcite.sql.SqlWindow; @@ -44,6 +47,8 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeUtil; +import org.apache.calcite.sql.util.SqlBasicVisitor; +import org.apache.calcite.sql.util.SqlVisitor; import org.apache.calcite.sql.validate.IdentifierNamespace; import org.apache.calcite.sql.validate.SelectNamespace; import org.apache.calcite.sql.validate.SqlNonNullableAccessors; @@ -64,6 +69,7 @@ import org.apache.druid.query.QueryContext; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.sql.calcite.aggregation.NativelySupportsDistinct; import org.apache.druid.sql.calcite.expression.builtin.ScalarInArrayOperatorConversion; import org.apache.druid.sql.calcite.parser.DruidSqlIngest; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; @@ -80,6 +86,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.function.Predicate; import java.util.regex.Pattern; /** @@ -110,6 +117,10 @@ public class DruidSqlValidator extends BaseDruidSqlValidator @Override public void validateWindow(SqlNode windowOrId, SqlValidatorScope scope, @Nullable SqlCall call) { + if (isSqlCallDistinct(call)) { + throw buildCalciteContextException("DISTINCT is not supported for window functions", windowOrId); + } + final SqlWindow targetWindow; switch (windowOrId.getKind()) { case IDENTIFIER: @@ -755,8 +766,10 @@ public class DruidSqlValidator extends BaseDruidSqlValidator throw buildCalciteContextException( StringUtils.format( "The query contains window functions; To run these window functions, specify [%s] in query context.", - PlannerContext.CTX_ENABLE_WINDOW_FNS), - call); + PlannerContext.CTX_ENABLE_WINDOW_FNS + ), + call + ); } } if (call.getKind() == SqlKind.NULLS_FIRST) { @@ -771,9 +784,38 @@ public class DruidSqlValidator extends BaseDruidSqlValidator throw buildCalciteContextException("ASCENDING ordering with NULLS LAST is not supported!", call); } } + if (plannerContext.getPlannerConfig().isUseApproximateCountDistinct() && isSqlCallDistinct(call)) { + if (call.getOperator().getKind() != SqlKind.COUNT && call.getOperator() instanceof SqlAggFunction) { + if (!call.getOperator().getClass().isAnnotationPresent(NativelySupportsDistinct.class)) { + throw buildCalciteContextException( + StringUtils.format( + "Aggregation [%s] with DISTINCT is not supported when useApproximateCountDistinct is enabled. Run with disabling it.", + call.getOperator().getName() + ), + call + ); + } + } + } super.validateCall(call, scope); } + @Override + protected void validateWindowClause(SqlSelect select) + { + SqlNodeList windows = select.getWindowList(); + for (SqlNode sqlNode : windows) { + if (SqlUtil.containsAgg(sqlNode)) { + throw buildCalciteContextException( + "Aggregation inside window is currently not supported with syntax WINDOW W AS . " + + "Try providing window definition directly without alias", + sqlNode + ); + } + } + super.validateWindowClause(select); + } + @Override protected SqlNode performUnconditionalRewrites(SqlNode node, final boolean underFrom) { @@ -857,4 +899,57 @@ public class DruidSqlValidator extends BaseDruidSqlValidator } return src; } + + private boolean isSqlCallDistinct(@Nullable SqlCall call) + { + return call != null + && call.getFunctionQuantifier() != null + && call.getFunctionQuantifier().getValue() == SqlSelectKeyword.DISTINCT; + } + + @Override + protected void validateHavingClause(SqlSelect select) + { + super.validateHavingClause(select); + SqlNode having = select.getHaving(); + if (containsOver(having)) { + throw buildCalciteContextException("Window functions are not allowed in HAVING", having); + } + } + + private boolean containsOver(SqlNode having) + { + if (having == null) { + return false; + } + final Predicate callPredicate = call -> call.getOperator() instanceof SqlOverOperator; + return containsCall(having, callPredicate); + } + + // copy of SqlUtil#containsCall + /** Returns whether an AST tree contains a call that matches a given + * predicate. */ + private static boolean containsCall(SqlNode node, + Predicate callPredicate) + { + try { + SqlVisitor visitor = + new SqlBasicVisitor() { + @Override public Void visit(SqlCall call) + { + if (callPredicate.test(call)) { + throw new Util.FoundOne(call); + } + return super.visit(call); + } + }; + node.accept(visitor); + return false; + } + catch (Util.FoundOne e) { + Util.swallow(e, null); + return true; + } + } + } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index a7d534db700..bec5335f7c9 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -1678,7 +1678,6 @@ public class DruidQuery orderByColumns, filtration.getDimFilter(), scanColumnsList, - false, withScanSignatureIfNeeded( virtualColumns, scanColumnsList, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java index afd775ef4ee..8e1fc3ee275 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java @@ -160,15 +160,13 @@ public class Windowing Collections.emptyList(), aggName, aggregateCall, - false // Windowed aggregations don't currently finalize. This means that sketches won't work as expected. + false // Windowed aggregations finalize later when we write the computed value to result RAC ); if (aggregation == null || aggregation.getPostAggregator() != null || aggregation.getAggregatorFactories().size() != 1) { - if (null == plannerContext.getPlanningError()) { - plannerContext.setPlanningError("Aggregation [%s] is not supported", aggregateCall); - } + plannerContext.setPlanningError("Aggregation [%s] is currently not supported for window functions", aggregateCall.getAggregation().getName()); throw new CannotBuildQueryException(window, aggregateCall); } @@ -451,19 +449,22 @@ public class Windowing if (group.lowerBound.isUnbounded() && group.upperBound.isUnbounded()) { return WindowFrame.unbounded(); } - return new WindowFrame( - group.isRows ? WindowFrame.PeerType.ROWS : WindowFrame.PeerType.RANGE, - group.lowerBound.isUnbounded(), - figureOutOffset(group.lowerBound), - group.upperBound.isUnbounded(), - figureOutOffset(group.upperBound), - group.isRows ? null : getOrdering() - ); + if (group.isRows) { + return WindowFrame.rows(getBoundAsInteger(group.lowerBound), getBoundAsInteger(group.upperBound)); + } else { + /* Right now we support GROUPS based framing in the native layer; + * but the SQL layer doesn't accept that as of now. + */ + return WindowFrame.groups(getBoundAsInteger(group.lowerBound), getBoundAsInteger(group.upperBound), getOrderingColumNames()); + } } - private int figureOutOffset(RexWindowBound bound) + private Integer getBoundAsInteger(RexWindowBound bound) { - if (bound.isUnbounded() || bound.isCurrentRow()) { + if (bound.isUnbounded()) { + return null; + } + if (bound.isCurrentRow()) { return 0; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/GroupByRules.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/GroupByRules.java index fecabd00ec3..f0632006d10 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/GroupByRules.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/GroupByRules.java @@ -22,11 +22,13 @@ package org.apache.druid.sql.calcite.rule; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlKind; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.aggregation.Aggregation; +import org.apache.druid.sql.calcite.aggregation.NativelySupportsDistinct; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.Expressions; import org.apache.druid.sql.calcite.filtration.Filtration; @@ -69,6 +71,16 @@ public class GroupByRules return null; } + if (call.isDistinct() && call.getAggregation().getKind() != SqlKind.COUNT) { + if (!call.getAggregation().getClass().isAnnotationPresent(NativelySupportsDistinct.class)) { + plannerContext.setPlanningError( + "Aggregation [%s] with DISTINCT is not supported when useApproximateCountDistinct is enabled. Run with disabling it.", + call.getAggregation().getName() + ); + return null; + } + } + final DimFilter filter; if (call.filterArg >= 0) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java index d02d302437b..2477ac38dec 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java @@ -109,6 +109,7 @@ public class NativeSqlEngine implements SqlEngine case ALLOW_TOP_LEVEL_UNION_ALL: case TIME_BOUNDARY_QUERY: case GROUPBY_IMPLICITLY_SORTS: + case WINDOW_LEAF_OPERATOR: return true; case CAN_INSERT: case CAN_REPLACE: @@ -116,7 +117,6 @@ public class NativeSqlEngine implements SqlEngine case WRITE_EXTERNAL_DATA: case SCAN_ORDER_BY_NON_TIME: case SCAN_NEEDS_SIGNATURE: - case WINDOW_LEAF_OPERATOR: return false; default: throw SqlEngines.generateUnrecognizedFeatureException(NativeSqlEngine.class.getSimpleName(), feature); 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 a2e45c4af8c..0ad2ba48c58 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 @@ -608,8 +608,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase public static Druids.ScanQueryBuilder newScanQueryBuilder() { - return new Druids.ScanQueryBuilder().resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false); + return new Druids.ScanQueryBuilder().resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST); } protected static DruidExceptionMatcher invalidSqlIs(String s) @@ -1212,6 +1211,11 @@ public class BaseCalciteQueryTest extends CalciteTestBase skipVectorize = true; } + protected void sqlNativeIncompatible() + { + assumeTrue(testBuilder().config.isRunningMSQ(), "test case is not SQL native compatible"); + } + protected void msqIncompatible() { assumeFalse(testBuilder().config.isRunningMSQ(), "test case is not MSQ compatible"); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index 10abd67c2b3..2fe1919f6a1 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -1490,7 +1490,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest .columns(ImmutableList.of("v0")) .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -1522,7 +1521,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest .columns("v0", "v1", "v2") .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -3493,7 +3491,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest .columns("dim4", "j0.a0", "v0") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), @@ -3840,7 +3837,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest .columns("dim1", "dim2") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), @@ -3950,7 +3946,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of( "EXPR$0" @@ -4010,7 +4005,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -4054,7 +4048,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -4148,7 +4141,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -4194,7 +4186,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -4247,7 +4238,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -4296,7 +4286,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -4349,7 +4338,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -4431,7 +4419,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("_j0.unnest", "dim1", "j0.unnest", "v0", "v1")) .build() @@ -4495,7 +4482,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("_j0.unnest", "arrayLongNulls", "arrayStringNulls", "j0.unnest")) .build() @@ -4595,7 +4581,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("dim1", "j0.unnest", "v0", "v1", "v2")) .build() @@ -4656,7 +4641,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .virtualColumns(expressionVirtualColumn( "v0", @@ -4758,7 +4742,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest "'Baz'", ColumnType.STRING )) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "dimZipf", "v0")) .build() @@ -4826,7 +4809,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest expressionVirtualColumn("v0", "array('a','b')", ColumnType.STRING_ARRAY), expressionVirtualColumn("v1", "1", ColumnType.LONG) ) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "v0", "v1")) .build() @@ -4897,7 +4879,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "dimZipf", "j0.unnest")) .build() @@ -4974,7 +4955,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest .virtualColumns( expressionVirtualColumn("v0", "array('a','b')", ColumnType.STRING_ARRAY) ) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "j0.unnest", "v0")) .build() @@ -5204,7 +5184,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .limit(3) @@ -5233,7 +5212,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -5280,7 +5258,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .filters(or( NullHandling.sqlCompatible() ? range("m1", ColumnType.LONG, null, "10", false, false) @@ -5320,7 +5297,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) // (j0.unnest = a || (m1 = 2 && j0.unnest = b)) + // (j0.unnest = a || (m1 = 2 && j0.unnest = b)) .filters(or( equality("j0.unnest", "a", ColumnType.STRING), and( @@ -5357,7 +5334,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -5393,7 +5369,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ColumnType.LONG )) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest", "v0")) .build() @@ -5427,7 +5402,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest "timestamp_floor(\"__time\",'PT1H',null,'UTC')", ColumnType.LONG)) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest", "v0")) .build() @@ -5456,7 +5430,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .filters(equality("dim2", "a", ColumnType.STRING)) .columns("dim3") .limit(2) @@ -5468,7 +5441,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -5511,7 +5483,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -5551,7 +5522,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -5592,7 +5562,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -5622,7 +5591,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -5652,7 +5620,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -5747,7 +5714,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .filters(in("dim2", ImmutableList.of("a", "b", "ab", "abc"))) .columns("dim2") .context(QUERY_CONTEXT_UNNEST) @@ -5762,7 +5728,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("_j0.unnest")) .build() @@ -5818,7 +5783,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.EXPR$0")) .build() @@ -5862,7 +5826,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .virtualColumns(expressionVirtualColumn("v0", "strlen(\"j0.unnest\")", ColumnType.LONG)) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("v0")) .build() @@ -5909,7 +5872,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -5937,7 +5899,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -5968,7 +5929,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -5992,7 +5952,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6032,7 +5991,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6061,7 +6019,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6088,7 +6045,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6124,7 +6080,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6151,7 +6106,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .filters( or( @@ -6185,7 +6139,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6213,7 +6166,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6244,7 +6196,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6292,7 +6243,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6319,7 +6269,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .filters( or( @@ -6361,7 +6310,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest equality("dim3", "d", ColumnType.STRING) ) ) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6395,7 +6343,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest equality("j0.unnest", "a", ColumnType.STRING) ) ) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("dim3", "j0.unnest")) .build() @@ -6686,8 +6633,8 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ? not(istrue(expressionFilter("(\"j0.unnest\" == \"m2\")"))) : not(expressionFilter("(\"j0.unnest\" == \"m2\")")) ) - )) - .legacy(false) + ) + ) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest", "m2")) .build() @@ -6728,7 +6675,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6760,7 +6706,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Intervals.of("2023-01-02T00:00:00.000Z/2023-01-03T00:10:00.001Z"))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6803,7 +6748,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6838,7 +6782,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6874,7 +6817,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("_j0.unnest")) .build() @@ -6918,7 +6860,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest )) .intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("__j0.unnest")) .build() @@ -6966,7 +6907,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Intervals.of( "2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .columns("j0.unnest", "m1") .limit(2) .context(QUERY_CONTEXT_UNNEST) @@ -6980,7 +6920,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest : in("m1", ImmutableList.of("1", "2")) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -7018,7 +6957,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Intervals.of( "2023-01-02T00:00:00.000Z/2023-01-03T00:10:00.001Z"))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .columns("arrayLongNulls", "j0.unnest") .limit(2) .context(QUERY_CONTEXT_UNNEST) @@ -7030,7 +6968,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest new ArrayContainsElementFilter("arrayLongNulls", ColumnType.LONG, 2L, null) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -7110,7 +7047,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) .intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .virtualColumns(expressionVirtualColumn("v0", "2.0", ColumnType.FLOAT)) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "j0.unnest", "v0")) @@ -7142,7 +7078,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .virtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest", "v", "v0")) .build() @@ -7200,7 +7135,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) .eternityInterval() .columns("d0", "j0.unnest") - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -7322,7 +7256,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest .virtualColumns(expressionVirtualColumn("v0", "substring(\"j0.unnest\", 0, -1)", ColumnType.STRING)) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("v0")) .build() @@ -7362,7 +7295,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest .virtualColumns(expressionVirtualColumn("v0", "substring(\"j0.unnest\", 0, -1)", ColumnType.STRING)) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("v0")) .build() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExplainQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExplainQueryTest.java index 3d7c820902b..7964bca842c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExplainQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExplainQueryTest.java @@ -40,9 +40,6 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest skipVectorize(); final String query = "EXPLAIN PLAN FOR SELECT COUNT(*) FROM view.aview WHERE dim1_firstchar <> 'z'"; - final String legacyExplanation = NullHandling.replaceWithDefault() - ? "DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"and\",\"fields\":[{\"type\":\"selector\",\"dimension\":\"dim2\",\"value\":\"a\"},{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"z\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}}]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n" - : "DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"substring(\\\"dim1\\\", 0, 1)\",\"outputType\":\"STRING\"}],\"filter\":{\"type\":\"and\",\"fields\":[{\"type\":\"equals\",\"column\":\"dim2\",\"matchValueType\":\"STRING\",\"matchValue\":\"a\"},{\"type\":\"not\",\"field\":{\"type\":\"equals\",\"column\":\"v0\",\"matchValueType\":\"STRING\",\"matchValue\":\"z\"}}]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n"; final String explanation = NullHandling.replaceWithDefault() ? "[{" + "\"query\":{\"queryType\":\"timeseries\"," @@ -70,15 +67,6 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest final String resources = "[{\"name\":\"aview\",\"type\":\"VIEW\"}]"; final String attributes = "{\"statementType\":\"SELECT\"}"; - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - query, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of( - new Object[]{legacyExplanation, resources, attributes} - ) - ); testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, query, @@ -127,17 +115,6 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest + " SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 IS NOT NULL\n" + " )\n" + ")"; - final String legacyExplanation = NullHandling.replaceWithDefault() - ? - "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n" - + " DruidJoinQueryRel(condition=[=(SUBSTRING($2, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n" - : - "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n" - + " DruidJoinQueryRel(condition=[=(SUBSTRING($2, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"null\",\"column\":\"dim1\"}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n"; final String explanation = NullHandling.replaceWithDefault() ? "[" + "{\"query\":{\"queryType\":\"groupBy\"," @@ -160,89 +137,20 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest ImmutableList.of(), ImmutableList.of(new Object[]{explanation, resources, attributes}) ); - - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - query, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of(new Object[]{legacyExplanation, resources, attributes}) - ); } - // This testcase has been added here and not in CalciteSelectQueryTest since this checks if the overrides are working - // properly when displaying the output of "EXPLAIN PLAN FOR ..." queries @Test - public void testExplainSelectStarWithOverrides() + public void testExplainSelectStar() { - Map useRegularExplainContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); - useRegularExplainContext.put(PlannerConfig.CTX_KEY_USE_NATIVE_QUERY_EXPLAIN, true); - - Map legacyExplainContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); - legacyExplainContext.put(PlannerConfig.CTX_KEY_USE_NATIVE_QUERY_EXPLAIN, false); - - // Skip vectorization since otherwise the "context" will change for each subtest. skipVectorize(); - String legacyExplanationWithContext = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":false,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"; - String explanation = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," - + "\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]" - + "}]"; + String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; - String explanationWithContext = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," - + "\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]" - + "}]"; String sql = "EXPLAIN PLAN FOR SELECT * FROM druid.foo"; String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"SELECT\"}"; - // Test when default config and no overrides testQuery(sql, ImmutableList.of(), ImmutableList.of(new Object[]{explanation, resources, attributes})); - - // Test when default config and useNativeQueryExplain is overridden in the context - testQuery( - sql, - legacyExplainContext, - ImmutableList.of(), - ImmutableList.of(new Object[]{legacyExplanationWithContext, resources, attributes}) - ); - - // Test when useNativeQueryExplain enabled by default and no overrides - testQuery( - PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, - sql, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of(new Object[]{explanation, resources, attributes}) - ); - - // Test when useNativeQueryExplain enabled by default but is overriden in the context - testQuery( - PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, - useRegularExplainContext, - sql, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of(new Object[]{explanationWithContext, resources, attributes}) - ); } @Test @@ -254,46 +162,11 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest final String query = "EXPLAIN PLAN FOR SELECT dim1 FROM druid.foo\n" + "UNION ALL (SELECT dim1 FROM druid.foo WHERE dim1 = '42'\n" + "UNION ALL SELECT dim1 FROM druid.foo WHERE dim1 = '44')"; - final String legacyExplanation = NullHandling.replaceWithDefault() - ? "DruidUnionRel(limit=[-1])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidUnionRel(limit=[-1])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - : "DruidUnionRel(limit=[-1])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidUnionRel(limit=[-1])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"; final String explanation = NullHandling.replaceWithDefault() - ? "[" - + "{" - + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" - + "}," - + "{" - + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" - + "}," - + "{" - + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" - + "}]" - : "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]}]"; + ? "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]}]" + : "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]}]"; final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"SELECT\"}"; - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - query, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of( - new Object[]{legacyExplanation, resources, attributes} - ) - ); testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, query, @@ -321,23 +194,7 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest defaultExprContext.put(PlannerConfig.CTX_KEY_USE_NATIVE_QUERY_EXPLAIN, true); defaultExprContext.put(PlannerConfig.CTX_KEY_FORCE_EXPRESSION_VIRTUAL_COLUMNS, true); - final String expectedPlanWithDefaultExpressions = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[" - + "{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"filter((x) -> array_contains(array('true','false'), x), \\\"dim1\\\")\",\"outputType\":\"STRING\"}," - + "{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"filter((x) -> !array_contains(array('true','false'), x), \\\"dim1\\\")\",\"outputType\":\"STRING\"}" - + "]," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"v0\",\"v1\"]," - + "\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"forceExpressionVirtualColumns\":true,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"STRING\",\"STRING\"]," - + "\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"EXPR$1\"}]" - + "}]"; + final String expectedPlanWithDefaultExpressions = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"filter((x) -> array_contains(array('true','false'), x), \\\"dim1\\\")\",\"outputType\":\"STRING\"},{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"filter((x) -> !array_contains(array('true','false'), x), \\\"dim1\\\")\",\"outputType\":\"STRING\"}],\"resultFormat\":\"compactedList\",\"columns\":[\"v0\",\"v1\"],\"context\":{\"defaultTimeout\":300000,\"forceExpressionVirtualColumns\":true,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"EXPR$1\"}]}]"; final String expectedResources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String expectedAttributes = "{\"statementType\":\"SELECT\"}"; testQuery( @@ -348,23 +205,7 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest ); // Test plan as mv-filtered virtual columns - final String expectedPlanWithMvfiltered = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[" - + "{\"type\":\"mv-filtered\",\"name\":\"v0\",\"delegate\":{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"dim1\",\"outputType\":\"STRING\"},\"values\":[\"true\",\"false\"],\"isAllowList\":true}," - + "{\"type\":\"mv-filtered\",\"name\":\"v1\",\"delegate\":{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"dim1\",\"outputType\":\"STRING\"},\"values\":[\"true\",\"false\"],\"isAllowList\":false}" - + "]," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"v0\",\"v1\"]," - + "\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"STRING\",\"STRING\"]," - + "\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"EXPR$1\"}]" - + "}]"; + final String expectedPlanWithMvfiltered = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"mv-filtered\",\"name\":\"v0\",\"delegate\":{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"dim1\",\"outputType\":\"STRING\"},\"values\":[\"true\",\"false\"],\"isAllowList\":true},{\"type\":\"mv-filtered\",\"name\":\"v1\",\"delegate\":{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"dim1\",\"outputType\":\"STRING\"},\"values\":[\"true\",\"false\"],\"isAllowList\":false}],\"resultFormat\":\"compactedList\",\"columns\":[\"v0\",\"v1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"EXPR$1\"}]}]"; final Map mvFilteredContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); mvFilteredContext.put(PlannerConfig.CTX_KEY_USE_NATIVE_QUERY_EXPLAIN, true); @@ -389,22 +230,7 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest final Map queryContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); queryContext.put(PlannerConfig.CTX_KEY_USE_NATIVE_QUERY_EXPLAIN, true); - final String expectedPlan = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[" - + "{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"dim1\\\",null,'UTC')\",\"outputType\":\"LONG\"}" - + "]," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"v0\"]," - + "\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\"]," - + "\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"}]" - + "}]"; + final String expectedPlan = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"dim1\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"columns\":[\"v0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"}]}]"; final String expectedResources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String expectedAttributes = "{\"statementType\":\"SELECT\"}"; // Verify the query plan diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExportTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExportTest.java index 244d7226acb..af77671cee4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExportTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExportTest.java @@ -123,7 +123,6 @@ public class CalciteExportTest extends CalciteIngestionDmlTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) .expectResources(dataSourceRead("foo"), externalWrite(LocalFileExportStorageProvider.TYPE_NAME)) @@ -198,7 +197,6 @@ public class CalciteExportTest extends CalciteIngestionDmlTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) .expectResources(dataSourceRead("foo"), externalWrite(LocalFileStorageConnectorProvider.TYPE_NAME)) @@ -224,7 +222,6 @@ public class CalciteExportTest extends CalciteIngestionDmlTest .filters(equality("dim2", "val", ColumnType.STRING)) .columns("dim2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) .expectResources(dataSourceRead("foo"), externalWrite(LocalFileStorageConnectorProvider.TYPE_NAME)) @@ -251,7 +248,6 @@ public class CalciteExportTest extends CalciteIngestionDmlTest .filters(equality("dim2", "val", ColumnType.STRING)) .columns("dim2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) .expectResources(dataSourceRead("foo"), externalWrite(LocalFileStorageConnectorProvider.TYPE_NAME)) @@ -306,7 +302,6 @@ public class CalciteExportTest extends CalciteIngestionDmlTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) .expectResources(dataSourceRead("foo"), dataSourceWrite("csv")) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index b40a4c87c3a..ba7b19311ce 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -57,7 +57,6 @@ import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.jupiter.api.Test; import java.io.File; -import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -742,48 +741,9 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest .columnTypes(ColumnType.LONG, ColumnType.STRING, ColumnType.FLOAT, ColumnType.DOUBLE) .build(); - final String legacyExplanation = - "DruidQueryRel(query=[" - + queryJsonMapper.writeValueAsString(expectedQuery) - + "], signature=[{__time:LONG, v0:FLOAT, dim1:STRING, v1:DOUBLE}])\n"; - - - // Use testQuery for EXPLAIN (not testIngestionQuery). - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - ImmutableMap.of("sqlQueryId", "dummy"), - Collections.emptyList(), - sql, - CalciteTests.SUPER_USER_AUTH_RESULT, - ImmutableList.of(), - new DefaultResultsVerifier( - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ), - null - ) - ); - // Test correctness of the query when only the CLUSTERED BY clause is present final String explanation = - "[" - + "{\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"floor(\\\"m1\\\")\",\"outputType\":\"FLOAT\"}," - + "{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}]," - + "\"resultFormat\":\"compactedList\"," - + "\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"}," - + "{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"FLOAT\",\"DOUBLE\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"}," - + "{\"name\":\"v1\",\"type\":\"DOUBLE\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"v0\",\"outputColumn\":\"floor_m1\"}," - + "{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"ceil_m2\"}]" - + "}]"; + "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"floor(\\\"m1\\\")\",\"outputType\":\"FLOAT\"},{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"},{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"FLOAT\",\"DOUBLE\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"DOUBLE\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"v0\",\"outputColumn\":\"floor_m1\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"ceil_m2\"}]}]"; testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, @@ -835,51 +795,8 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest + "PARTITIONED BY ALL\n" + "CLUSTERED BY 2, 3"; - final String legacyExplanation = "DruidQueryRel(" - + "query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\"," - + "\"inputSource\":{\"type\":\"inline\",\"data\":\"{\\\" \\\": 1681794225551, \\\"namespace\\\": \\\"day1\\\", \\\"country\\\": \\\"one\\\"}\\n" - + "{\\\"__time\\\": 1681794225558, \\\"namespace\\\": \\\"day2\\\", \\\"country\\\": \\\"two\\\"}\"}," - + "\"inputFormat\":{\"type\":\"json\"}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"namespace\",\"order\":\"ascending\"},{\"columnName\":\"country\",\"order\":\"ascending\"}]," - + "\"columns\":[\"__time\",\"country\",\"namespace\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\"," - + "\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}]," - + " signature=[{__time:LONG, namespace:STRING, country:STRING}])\n"; - - // Use testQuery for EXPLAIN (not testIngestionQuery). - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - ImmutableMap.of("sqlQueryId", "dummy"), - Collections.emptyList(), - sql, - CalciteTests.SUPER_USER_AUTH_RESULT, - ImmutableList.of(), - new DefaultResultsVerifier( - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ), - null - ) - ); - // Test correctness of the query when only the CLUSTERED BY clause is present - final String explanation = "[{\"query\":{\"queryType\":\"scan\"," + "\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\"," - + "\"data\":\"{\\\" \\\": 1681794225551, \\\"namespace\\\": \\\"day1\\\", \\\"country\\\": \\\"one\\\"}\\n" - + "{\\\"__time\\\": 1681794225558, \\\"namespace\\\": \\\"day2\\\", \\\"country\\\": \\\"two\\\"}\"}," - + "\"inputFormat\":{\"type\":\"json\"}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"namespace\",\"order\":\"ascending\"},{\"columnName\":\"country\",\"order\":\"ascending\"}]," - + "\"columns\":[\"__time\",\"country\",\"namespace\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\"," - + "\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"namespace\",\"outputColumn\":\"namespace\"}," - + "{\"queryColumn\":\"country\",\"outputColumn\":\"country\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\",\"data\":\"{\\\" \\\": 1681794225551, \\\"namespace\\\": \\\"day1\\\", \\\"country\\\": \\\"one\\\"}\\n{\\\"__time\\\": 1681794225558, \\\"namespace\\\": \\\"day2\\\", \\\"country\\\": \\\"two\\\"}\"},\"inputFormat\":{\"type\":\"json\"},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"namespace\",\"order\":\"ascending\"},{\"columnName\":\"country\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"country\",\"namespace\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"namespace\",\"outputColumn\":\"namespace\"},{\"queryColumn\":\"country\",\"outputColumn\":\"country\"}]}]"; testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, @@ -940,49 +857,9 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest + "PARTITIONED BY HOUR\n" + "CLUSTERED BY 1, 2, 3, regionName"; - final String legacyExplanation = "DruidJoinQueryRel(condition=[=($3, $6)], joinType=[left], query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\"," - + "\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"}," - + "{\"columnName\":\"Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"Capital\",\"isRobot\",\"regionName\",\"v0\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\"," - + "\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"}}], signature=[{v0:LONG, isRobot:STRING, Capital:STRING, regionName:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://boo.gz\"]},\"inputFormat\":{\"type\":\"json\"}," - + "\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"cityName\",\"countryIsoCode\",\"isRobot\",\"regionName\",\"timestamp\"],\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{isRobot:STRING, timestamp:STRING, cityName:STRING, countryIsoCode:STRING, regionName:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true}," - + "\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{Capital:STRING, ISO2:STRING}])\n"; - // Use testQuery for EXPLAIN (not testIngestionQuery). - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - ImmutableMap.of("sqlQueryId", "dummy"), - Collections.emptyList(), - sql, - CalciteTests.SUPER_USER_AUTH_RESULT, - ImmutableList.of(), - new DefaultResultsVerifier( - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ), - null - ) - ); // Test correctness of the query when only the CLUSTERED BY clause is present - final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"join\",\"left\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://boo.gz\"]},\"inputFormat\":{\"type\":\"json\"}" - + ",\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"}," - + "{\"name\":\"regionName\",\"type\":\"STRING\"}]},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true}," - + "\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\"," - + "\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"," - + "\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}},\"rightPrefix\":\"j0.\",\"condition\":\"(\\\"countryIsoCode\\\" == \\\"j0.ISO2\\\")\",\"joinType\":\"LEFT\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"}," - + "{\"columnName\":\"j0.Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"isRobot\",\"j0.Capital\",\"regionName\",\"v0\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"," - + "\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"j0.Capital\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"__time\"}," - + "{\"queryColumn\":\"isRobot\",\"outputColumn\":\"isRobotAlias\"},{\"queryColumn\":\"j0.Capital\",\"outputColumn\":\"countryCapital\"},{\"queryColumn\":\"regionName\",\"outputColumn\":\"regionName\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"join\",\"left\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://boo.gz\"]},\"inputFormat\":{\"type\":\"json\"},\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}]},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true},\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false}},\"rightPrefix\":\"j0.\",\"condition\":\"(\\\"countryIsoCode\\\" == \\\"j0.ISO2\\\")\",\"joinType\":\"LEFT\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"},{\"columnName\":\"j0.Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"isRobot\",\"j0.Capital\",\"regionName\",\"v0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"j0.Capital\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"isRobot\",\"outputColumn\":\"isRobotAlias\"},{\"queryColumn\":\"j0.Capital\",\"outputColumn\":\"countryCapital\"},{\"queryColumn\":\"regionName\",\"outputColumn\":\"regionName\"}]}]"; testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, @@ -1274,7 +1151,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest } @Test - public void testExplainInsertFromExternal() throws IOException + public void testExplainInsertFromExternal() { // Skip vectorization since otherwise the "context" will change for each subtest. skipVectorize(); @@ -1284,62 +1161,12 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest externSql(externalDataSource) ); - ObjectMapper queryJsonMapper = queryFramework().queryJsonMapper(); - final ScanQuery expectedQuery = newScanQueryBuilder() - .dataSource(externalDataSource) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("x", "y", "z") - .context( - queryJsonMapper.readValue( - "{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}", - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ) - ) - .columnTypes(STRING, STRING, LONG) - .build(); - - final String legacyExplanation = - "DruidQueryRel(query=[" - + queryJsonMapper.writeValueAsString(expectedQuery) - + "], signature=[{x:STRING, y:STRING, z:LONG}])\n"; - final String explanation = - "[" - + "{\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\",\"data\":\"a,b,1\\nc,d,2\\n\"}," - + "\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]}," - + "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]" - + "}]"; + "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\",\"data\":\"a,b,1\\nc,d,2\\n\"},\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"}}"; - // Use testQuery for EXPLAIN (not testIngestionQuery). - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - ImmutableMap.of("sqlQueryId", "dummy"), - Collections.emptyList(), - query, - CalciteTests.SUPER_USER_AUTH_RESULT, - ImmutableList.of(), - new DefaultResultsVerifier( - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ), - null - ) - ); - - testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, ImmutableMap.of("sqlQueryId", "dummy"), @@ -1397,51 +1224,12 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest .columnTypes(LONG, STRING, FLOAT, DOUBLE) .build(); - - final String legacyExplanation = - "DruidQueryRel(query=[" - + queryJsonMapper.writeValueAsString(expectedQuery) - + "], signature=[{__time:LONG, v0:FLOAT, dim1:STRING, v1:DOUBLE}])\n"; - final String explanation = - "[" - + "{\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"floor(\\\"m1\\\")\",\"outputType\":\"FLOAT\"}," - + "{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}]," - + "\"resultFormat\":\"compactedList\"," - + "\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"}," - + "{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"FLOAT\",\"DOUBLE\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"}," - + "{\"name\":\"v1\",\"type\":\"DOUBLE\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"v0\",\"outputColumn\":\"floor_m1\"}," - + "{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"ceil_m2\"}]" - + "}]"; + "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"floor(\\\"m1\\\")\",\"outputType\":\"FLOAT\"},{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"},{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"FLOAT\",\"DOUBLE\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"DOUBLE\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"v0\",\"outputColumn\":\"floor_m1\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"ceil_m2\"}]}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"floor_m1\",\"dim1\",\"CEIL(\\\"m2\\\")\"]}"; - // Use testQuery for EXPLAIN (not testIngestionQuery). - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - ImmutableMap.of("sqlQueryId", "dummy"), - Collections.emptyList(), - query, - CalciteTests.SUPER_USER_AUTH_RESULT, - ImmutableList.of(), - new DefaultResultsVerifier( - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ), - null - ) - ); - testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, ImmutableMap.of("sqlQueryId", "dummy"), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index 8f3fde509d5..264ebf6890e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -3515,7 +3515,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .filters(equality("m2", "1000", ColumnType.DOUBLE)) .columns("dim2") - .legacy(false) .build() ); @@ -3545,7 +3544,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest false )) .columns("dim2") - .legacy(false) .build() ), rightTable, @@ -5633,7 +5631,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .columns("l1") .intervals(querySegmentSpec(Filtration.eternity())) .context(queryContext) - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() ), @@ -6173,7 +6170,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .dataSource(CalciteTests.DATASOURCE3) .columns("dim2") - .legacy(false) .context(context) .build() ), @@ -6232,7 +6228,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .dataSource(CalciteTests.DATASOURCE3) .columns("dim2") - .legacy(false) .context(context) .build() ), @@ -6287,7 +6282,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .dataSource(CalciteTests.DATASOURCE1) .filters(in("dim2", ImmutableList.of("a", "b", "ab", "abc"))) - .legacy(false) .context(context) .columns("dim2") .build() @@ -6304,7 +6298,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .dataSource(CalciteTests.DATASOURCE3) .columns("dim2") - .legacy(false) .context(context) .build() ), @@ -6379,7 +6372,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest null )) .columns("dim2", "j0.unnest") - .legacy(false) .context(context) .build() ), @@ -6451,7 +6443,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ImmutableList.of("a", "ab", "abc", "b") ) ) - .legacy(false) .context(context) .build() ), @@ -6471,7 +6462,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .dataSource(CalciteTests.DATASOURCE3) .columns("dim2") - .legacy(false) .context(context) .build() ), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java index cb0a836fb24..ba4e32b8101 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java @@ -1748,7 +1748,6 @@ public class CalciteLookupFunctionQueryTest extends BaseCalciteQueryTest expressionVirtualColumn("v0", "null", ColumnType.STRING) ) .columns("v0") - .legacy(false) .filters(isNull("dim2")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) @@ -1785,7 +1784,6 @@ public class CalciteLookupFunctionQueryTest extends BaseCalciteQueryTest expressionVirtualColumn("v0", "null", ColumnType.STRING) ) .columns("v0") - .legacy(false) .filters(isNull("dim2")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) @@ -2140,7 +2138,6 @@ public class CalciteLookupFunctionQueryTest extends BaseCalciteQueryTest .columns("$f1", "EXPR$0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT) - .legacy(false) .build() ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java index 275b4e9a164..8b21d336fdc 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java @@ -195,7 +195,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest .columns(ImmutableList.of("v0")) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -223,7 +222,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest .columns(ImmutableList.of("v0")) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -252,7 +250,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest .columns(ImmutableList.of("v0")) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -456,7 +453,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest .columns(ImmutableList.of("v0")) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( 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 e2bc9d45eb2..9e6e5da2bb0 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 @@ -80,6 +80,7 @@ import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.hamcrest.CoreMatchers; +import org.junit.Assert; import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -1182,7 +1183,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ) .columns("v0", "v1", "v2", "v3") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) ) @@ -1270,7 +1270,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("j0.unnest") .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() ) @@ -1330,7 +1329,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("j0.unnest") .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() ) @@ -1387,7 +1385,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("j0.unnest") .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() ) @@ -4858,7 +4855,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ) .columns("v0", "v1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -4908,7 +4904,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -4956,7 +4951,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5007,7 +5001,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ) .columns("string", "v0", "v1", "v2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5103,7 +5096,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ) .columns("v0", "v1", "v2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5140,7 +5132,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ) .columns("v0", "v1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5179,7 +5170,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ) .columns("v0", "v1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5308,7 +5298,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .filters(isNull("v0")) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5512,7 +5501,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .filters(notNull("v0")) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), expectedResults, @@ -6018,7 +6006,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest "variantWithArrays" ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), useDefault ? @@ -6612,7 +6599,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .columns("nest") .filters(notNull("nest")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), NullHandling.replaceWithDefault() @@ -6642,7 +6628,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .columns("nest", "nester") .filters(isNull("nest")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), // selector filter is wrong @@ -6698,7 +6683,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("j0.unnest", "long", "v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ) @@ -6746,7 +6730,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("long") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ) : @@ -6761,7 +6744,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .virtualColumns(expressionVirtualColumn("v0", "nvl(\"long\",\"j0.unnest\")", ColumnType.LONG)) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -6847,7 +6829,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ) .columns("v0", "v1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -6888,7 +6869,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ) .columns("v0") .context(QUERY_CONTEXT_DEFAULT) - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() ) @@ -6943,7 +6923,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .columns("v0") .limit(1) .context(QUERY_CONTEXT_DEFAULT) - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() ) @@ -6983,7 +6962,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("j0.unnest") .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() ) @@ -7165,7 +7143,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -7321,6 +7298,33 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testApproxCountDistinctOnUnsupportedComplexColumn() + { + assertQueryIsUnplannable( + "SELECT COUNT(DISTINCT nester) FROM druid.nested", + "Query could not be planned. A possible reason is [Using APPROX_COUNT_DISTINCT() or enabling " + + "approximation with COUNT(DISTINCT) is not supported for column type [COMPLEX]. " + + "You can disable approximation by setting [useApproximateCountDistinct: false] in the query context." + ); + } + + @Test + public void testApproxCountDistinctFunctionOnUnsupportedComplexColumn() + { + DruidException druidException = Assert.assertThrows( + DruidException.class, + () -> testQuery( + "SELECT APPROX_COUNT_DISTINCT(nester) FROM druid.nested", + ImmutableList.of(), + ImmutableList.of() + ) + ); + Assert.assertTrue(druidException.getMessage().contains( + "Cannot apply 'APPROX_COUNT_DISTINCT' to arguments of type 'APPROX_COUNT_DISTINCT(>)'" + )); + } + @Test public void testNvlJsonValueDoubleSometimesMissingEqualityFilter() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java index 6bd273696e2..bd9c5649df1 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java @@ -75,7 +75,6 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -685,7 +684,6 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .columns("EXPR$0") - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 2ae095d41c7..84b437c20e0 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import org.apache.calcite.rel.RelNode; import org.apache.calcite.runtime.CalciteContextException; import org.apache.druid.common.config.NullHandling; import org.apache.druid.error.DruidException; @@ -149,6 +150,7 @@ import java.util.Map; import java.util.stream.Collectors; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assumptions.assumeFalse; @@ -1351,7 +1353,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .columns("EXPR$0", "dim2") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of() @@ -1376,7 +1377,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .columns("EXPR$0", "dim2") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of() @@ -2528,7 +2528,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .limit(1) .columns(ImmutableList.of("v0")) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -3470,7 +3469,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) : ImmutableList.of( @@ -3508,7 +3506,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) : ImmutableList.of( @@ -3546,7 +3543,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) : ImmutableList.of( @@ -3965,7 +3961,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of(new Object[]{0L}) @@ -4243,7 +4238,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0", "EXPR$1") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -4316,7 +4310,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0", "EXPR$1") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of() @@ -6389,7 +6382,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -6777,7 +6769,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .virtualColumns(new ExpressionVirtualColumn("v0", ExprEval.of(1L).toExpr(), ColumnType.LONG)) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_LOS_ANGELES) .build() ), @@ -12709,7 +12700,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1", "dim2", "dimX") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), @@ -13851,7 +13841,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest expressionVirtualColumn("v0", "round(\"f1\")", ColumnType.FLOAT) ) .columns("f1", "v0") - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -14932,7 +14921,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .columns("l1") .intervals(querySegmentSpec(Filtration.eternity())) .context(QUERY_CONTEXT_DEFAULT) - .legacy(false) .filters( in("l1", ColumnType.LONG, ImmutableList.of(4842L, 4844L, 4845L, 14905L, 4853L, 29064L)) ) @@ -15178,7 +15166,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ) ) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .limit(10) .build() ), @@ -15516,6 +15503,20 @@ public class CalciteQueryTest extends BaseCalciteQueryTest assertThat(e, invalidSqlIs("The query contains window functions; To run these window functions, specify [enableWindowing] in query context. (line [1], column [13])")); } + @Test + public void testDistinctSumNotSupportedWithApproximation() + { + DruidException e = assertThrows( + DruidException.class, + () -> testBuilder() + .queryContext(ImmutableMap.of(PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT, true)) + .sql("SELECT sum(distinct m1) from druid.foo") + .run() + ); + + assertThat(e, invalidSqlContains("Aggregation [SUM] with DISTINCT is not supported")); + } + @Test public void testUnSupportedNullsFirst() { @@ -15577,6 +15578,39 @@ public class CalciteQueryTest extends BaseCalciteQueryTest assertThat(e, invalidSqlContains("Framing of NTILE is not supported")); } + @Test + public void testDistinctNotSupportedWithWindow() + { + assumeFeatureAvailable(EngineFeature.WINDOW_FUNCTIONS); + + DruidException e = assertThrows( + DruidException.class, + () -> testBuilder() + .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) + .sql("SELECT count(distinct dim1) OVER () from druid.foo") + .run() + ); + + assertThat(e, invalidSqlContains("DISTINCT is not supported for window functions")); + } + + @Test + public void testUnSupportedAggInSelectWindow() + { + assertEquals( + "1.37.0", + RelNode.class.getPackage().getImplementationVersion(), + "Calcite version changed; check if CALCITE-6500 is fixed and update:\n * method DruidSqlValidator#validateWindowClause" + ); + + DruidException e = assertThrows(DruidException.class, () -> testBuilder() + .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) + .sql("SELECT dim1, ROW_NUMBER() OVER W from druid.foo WINDOW W as (ORDER BY max(length(dim1)))") + .run()); + + assertThat(e, invalidSqlContains("not supported with syntax WINDOW W AS ")); + } + @Test public void testInGroupByLimitOutGroupByOrderBy() { @@ -15818,7 +15852,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .columns("a0", "w0") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .limit(10) .build() ) @@ -15886,7 +15919,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest OperatorFactoryBuilders.naivePartitionOperator(), OperatorFactoryBuilders.windowOperators( OperatorFactoryBuilders.framedAggregateProcessor( - WindowFrame.forOrderBy(ColumnWithDirection.ascending("d0")), + WindowFrame.forOrderBy("d0"), new LongSumAggregatorFactory("w0", "a0") ) ) @@ -15938,7 +15971,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .run(); } - @NotYetSupported(Modes.WINDOW_OPERATOR_QUERY_ON_UNSUPPORTED_DATASOURCE) + @NotYetSupported(Modes.CANNOT_RETRIEVE_ROWS) @Test public void testWindowingOverJoin() { @@ -15993,7 +16026,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .columns(ImmutableList.of( "EXPR$0" )) @@ -16078,7 +16110,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .columns("EXPR$0") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) : ImmutableList.of( @@ -16094,7 +16125,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .virtualColumns(expressionVirtualColumn("v0", "0", ColumnType.LONG)) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of(NullHandling.sqlCompatible() ? new Object[]{null} : new Object[]{0}) @@ -16205,7 +16235,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .filters(isNull("dim3")) .limit(1) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_LOS_ANGELES) .build() ) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java index 444db2f4204..721eec21900 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java @@ -699,20 +699,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest + queryJsonMapper.writeValueAsString(expectedQuery) + "], signature=[{x:STRING, y:STRING, z:LONG}])\n"; - final String explanation = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\",\"data\":\"a,b,1\\nc,d,2\\n\"}," - + "\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]}," - + "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\"," - + "\"sqlReplaceTimeChunks\":\"all\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"]," - + "\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; - + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\",\"data\":\"a,b,1\\nc,d,2\\n\"},\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"all\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"},\"replaceTimeChunks\":\"all\"}"; @@ -790,7 +777,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest + "], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"; - final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"dim1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"dim1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"dim1\"],\"replaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\"}"; @@ -875,24 +862,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest + queryJsonMapper.writeValueAsString(expectedQuery) + "], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"; - final String explanation = "[" - + "{\"query\":{\"queryType\":\"scan\",\"dataSource\":" - + "{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\"," - + "\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"limit\":10," - + "\"orderBy\":[{\"columnName\":\"__time\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"}," - + "{\"columnName\":\"dim3\",\"order\":\"ascending\"},{\"columnName\":\"dim2\",\"order\":\"ascending\"}]," - + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," - + "\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\"," - + "\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\"," - + "\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"]," - + "\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"}," - + "{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}," - + "{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"}," - + "{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"limit\":10,\"orderBy\":[{\"columnName\":\"__time\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"},{\"columnName\":\"dim3\",\"order\":\"ascending\"},{\"columnName\":\"dim2\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"HOUR\"," + "\"clusteredBy\":[\"__time\",\"dim1\",\"dim3\",\"dim2\"],\"replaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\"}"; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java index d2699e77f79..6a0f1742155 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.calcite.rel.RelNode; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -92,7 +93,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest ) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -126,7 +126,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest ) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -176,7 +175,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0", "EXPR$1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -204,7 +202,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0", "EXPR$1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -232,7 +229,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0", "EXPR$1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -270,7 +266,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0", "EXPR$1", "EXPR$2", "EXPR$3", "EXPR$4") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -497,7 +492,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest ) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), expected @@ -533,7 +527,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest expressionVirtualColumn("v4", "5", ColumnType.LONG) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(context) .build() ), @@ -591,7 +584,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest ) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), expected @@ -604,36 +596,10 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest // Skip vectorization since otherwise the "context" will change for each subtest. skipVectorize(); final String query = "EXPLAIN PLAN FOR SELECT 1 + 1"; - final String explanation = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"EXPR$0\"]," - + "\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"EXPR$0\",\"type\":\"LONG\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"EXPR$0\",\"outputColumn\":\"EXPR$0\"}]" - + "}]"; - final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"EXPR$0\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\"],\"granularity\":{\"type\":\"all\"}}], signature=[{EXPR$0:LONG}])\n"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"EXPR$0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"EXPR$0\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"EXPR$0\",\"outputColumn\":\"EXPR$0\"}]}]"; final String resources = "[]"; final String attributes = "{\"statementType\":\"SELECT\"}"; - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - query, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ) - ); testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, query, @@ -896,7 +862,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -922,7 +887,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of() @@ -947,7 +911,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of() @@ -1012,7 +975,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .columns("CURRENT_DATE", "CURRENT_TIMESTAMP", "EXPR$1", "EXPR$2", "EXPR$3", "EXPR$4", "EXPR$6") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(context) .build() ), @@ -1344,34 +1306,10 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest skipVectorize(); final String query = "EXPLAIN PLAN FOR SELECT * FROM druid.foo"; - final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"; - final String explanation = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," - + "\"legacy\":false," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"SELECT\"}"; - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - query, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ) - ); testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, query, @@ -2202,4 +2140,21 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest ) .run(); } + + @Test + public void testRejectHavingWithWindowExpression() + { + assertEquals( + "1.37.0", + RelNode.class.getPackage().getImplementationVersion(), + "Calcite version changed; check if CALCITE-6473 is fixed and remove:\n * this assertion\n * DruidSqlValidator#validateHavingClause" + ); + + testQueryThrows( + "SELECT cityName,sum(1) OVER () as w FROM wikipedia group by cityName HAVING w > 10", + ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true), + DruidException.class, + invalidSqlContains("Window functions are not allowed in HAVING") + ); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java index 6269e2a5c8c..c1236162fa0 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java @@ -51,10 +51,12 @@ import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongMinAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.SingleValueAggregatorFactory; +import org.apache.druid.query.aggregation.firstlast.first.StringFirstAggregatorFactory; import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.ExtractionDimensionSpec; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.SubstringDimExtractionFn; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; @@ -731,7 +733,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest .limit(10) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) - .legacy(false) .build() ), ImmutableList.of() @@ -952,7 +953,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .columns("__time") - .legacy(false) .context(queryContext) .build()), "j0.", @@ -1307,7 +1307,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest .limit(1L) .order(ScanQuery.Order.DESCENDING) .columns("__time", "channel") - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -1388,6 +1387,220 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest ); } + @MethodSource("constructorFeeder") + @ParameterizedTest(name = "{0}") + public void testGroupBySubqueryWithEarliestAggregator(String testName, Map queryContext) + { + cannotVectorize(); + + // Note: EARLIEST aggregator is used because the intermediate type "serializablePair" is different from the finalized type + final List expectedResults; + if (NullHandling.replaceWithDefault()) { + expectedResults = ImmutableList.of( + new Object[]{"1", "", "a", "1"}, + new Object[]{"10.1", "b", "", "10.1"}, + new Object[]{"10.1", "c", "", "10.1"}, + new Object[]{"2", "d", "", "2"}, + new Object[]{"abc", "", "", "abc"}, + new Object[]{"def", "", "abc", "def"} + ); + } else { + expectedResults = ImmutableList.of( + new Object[]{"", "a", "a", ""}, + new Object[]{"", "b", "a", ""}, + new Object[]{"1", "", "a", "1"}, + new Object[]{"10.1", "b", null, "10.1"}, + new Object[]{"10.1", "c", null, "10.1"}, + new Object[]{"2", "d", "", "2"}, + new Object[]{"abc", null, null, "abc"}, + new Object[]{"def", null, "abc", "def"} + ); + } + + testQuery( + "SELECT a.dim1, a.dim3, a.e_dim2, b.dim1 " + + "FROM (" + + " SELECT dim1, dim3, EARLIEST(dim2) AS e_dim2 " + + " FROM foo GROUP BY 1, 2 LIMIT 100" + + ") a " + + "INNER JOIN foo b ON a.dim1 = b.dim1", + queryContext, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + JoinDataSource.create( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource("foo") + .setInterval(querySegmentSpec(Intervals.ETERNITY)) + .setGranularity(Granularities.ALL) + .setDimensions( + new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING), + new DefaultDimensionSpec("dim3", "d1", ColumnType.STRING) + ) + .addAggregator(new StringFirstAggregatorFactory("a0", "dim2", "__time", 1024)) + .setLimitSpec(new DefaultLimitSpec(Collections.emptyList(), 100)) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource("foo") + .intervals(querySegmentSpec(Intervals.ETERNITY)) + .columns("dim1") + .build() + ), + "j0.", + "(\"d0\" == \"j0.dim1\")", + JoinType.INNER, + null, + TestExprMacroTable.INSTANCE, + null + ) + ) + .intervals(querySegmentSpec(Intervals.ETERNITY)) + .columns("a0", "d0", "d1", "j0.dim1") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + expectedResults + ); + } + + @MethodSource("constructorFeeder") + @ParameterizedTest(name = "{0}") + public void testTopNSubqueryWithEarliestAggregator(String testName, Map queryContext) + { + final List expectedResults; + if (NullHandling.replaceWithDefault()) { + expectedResults = ImmutableList.of( + new Object[]{"1", "a", "1"}, + new Object[]{"10.1", "", "10.1"}, + new Object[]{"2", "", "2"}, + new Object[]{"abc", "", "abc"}, + new Object[]{"def", "abc", "def"} + ); + } else { + expectedResults = ImmutableList.of( + new Object[]{"", "a", ""}, + new Object[]{"1", "a", "1"}, + new Object[]{"10.1", null, "10.1"}, + new Object[]{"2", "", "2"}, + new Object[]{"abc", null, "abc"}, + new Object[]{"def", "abc", "def"} + ); + } + + testQuery( + "SELECT a.dim1, a.e_dim2, b.dim1 " + + "FROM (" + + " SELECT dim1, EARLIEST(dim2) AS e_dim2 " + + " FROM foo " + + " GROUP BY 1 " + + " LIMIT 100" + + ") a " + + "INNER JOIN foo b ON a.dim1 = b.dim1", + queryContext, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + JoinDataSource.create( + new QueryDataSource( + new TopNQueryBuilder() + .dataSource("foo") + .dimension(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING)) + .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) + .threshold(100) + .intervals(querySegmentSpec(Intervals.ETERNITY)) + .granularity(Granularities.ALL) + .aggregators( + new StringFirstAggregatorFactory("a0", "dim2", "__time", 1024) + ) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource("foo") + .intervals(querySegmentSpec(Intervals.ETERNITY)) + .columns("dim1") + .build() + ), + "j0.", + "(\"d0\" == \"j0.dim1\")", + JoinType.INNER, + null, + TestExprMacroTable.INSTANCE, + null + ) + ) + .intervals(querySegmentSpec(Intervals.ETERNITY)) + .columns("a0", "d0", "j0.dim1") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + expectedResults + ); + } + + @MethodSource("constructorFeeder") + @ParameterizedTest(name = "{0}") + public void testTimeseriesSubqueryWithEarliestAggregator(String testName, Map queryContext) + { + testQuery( + "SELECT a.__time, a.e_dim2, b.__time " + + "FROM (" + + " SELECT TIME_FLOOR(\"__time\", 'PT24H') as __time, EARLIEST(dim2) AS e_dim2 " + + " FROM foo " + + " GROUP BY 1 " + + ") a " + + "INNER JOIN foo b ON a.__time = b.__time", + queryContext, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + JoinDataSource.create( + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource("foo") + .intervals(querySegmentSpec(Intervals.ETERNITY)) + .granularity(new PeriodGranularity( + new Period("PT24H"), + null, + DateTimeZone.UTC + )) + .aggregators(new StringFirstAggregatorFactory("a0", "dim2", "__time", 1024)) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource("foo") + .intervals(querySegmentSpec(Intervals.ETERNITY)) + .columns("__time") + .build() + ), + "j0.", + "(\"d0\" == \"j0.__time\")", + JoinType.INNER, + null, + TestExprMacroTable.INSTANCE, + null + ) + ) + .intervals(querySegmentSpec(Intervals.ETERNITY)) + .columns("a0", "d0", "j0.__time") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{946684800000L, "a", 946684800000L}, + new Object[]{946771200000L, NullHandling.defaultStringValue(), 946771200000L}, + new Object[]{946857600000L, "", 946857600000L}, + new Object[]{978307200000L, "a", 978307200000L}, + new Object[]{978393600000L, "abc", 978393600000L}, + new Object[]{978480000000L, NullHandling.defaultStringValue(), 978480000000L} + ) + ); + } + public static class SubqueryComponentSupplier extends SqlTestFramework.StandardComponentSupplier { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteTableAppendTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteTableAppendTest.java index 8b6170cbfaa..02b8787df23 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteTableAppendTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteTableAppendTest.java @@ -49,7 +49,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest .virtualColumns( expressionVirtualColumn("v0", "null", null) ) - .legacy(false) .build(), Druids.newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) @@ -57,7 +56,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest .columns("dim1", "dim4") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) ) @@ -94,7 +92,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest .columns("d1", "dim1", "dim4", "f1") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) .expectedResults( @@ -137,7 +134,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .filters(equality("dim1", "2", ColumnType.STRING)) - .legacy(false) .build() ) .expectedResults( @@ -165,7 +161,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest .columns("dim1") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) .expectedResults( @@ -196,7 +191,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest .columns("dim3") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) .expectedResults( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java index ab20925107f..165b4aa3f63 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java @@ -45,8 +45,10 @@ import org.junit.jupiter.params.provider.MethodSource; import java.io.File; import java.net.URL; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.Objects; import static org.junit.Assert.assertEquals; @@ -66,6 +68,11 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest private static final ObjectMapper YAML_JACKSON = new DefaultObjectMapper(new YAMLFactory(), "tests"); + private static final Map DEFAULT_QUERY_CONTEXT = ImmutableMap.of( + PlannerContext.CTX_ENABLE_WINDOW_FNS, true, + QueryContexts.ENABLE_DEBUG, true + ); + public static Object[] parametersForWindowQueryTest() throws Exception { final URL windowFolderUrl = ClassLoader.getSystemResource("calcite/tests/window"); @@ -184,6 +191,11 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest log.info("Actual results:\n%s", sb.toString()); } } + + public Map getQueryContext() + { + return input.queryContext == null ? Collections.emptyMap() : input.queryContext; + } } @MethodSource("parametersForWindowQueryTest") @@ -199,127 +211,17 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest testBuilder() .skipVectorize(true) .sql(testCase.getSql()) - .queryContext(ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true - )) - .addCustomVerification(QueryVerification.ofResults(testCase)) - .run(); - } - } - - @MethodSource("parametersForWindowQueryTest") - @ParameterizedTest(name = "{0}") - @SuppressWarnings("unchecked") - public void windowQueryTestWithCustomContextMaxSubqueryBytes(String filename) throws Exception - { - TestCase testCase = new TestCase(filename); - - assumeTrue(testCase.getType() != TestType.failingTest); - - if (testCase.getType() == TestType.operatorValidation) { - testBuilder() - .skipVectorize(true) - .sql(testCase.getSql()) - .queryContext(ImmutableMap.of(QueryContexts.ENABLE_DEBUG, true, - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000" - ) + .queryContext( + ImmutableMap.builder() + .putAll(DEFAULT_QUERY_CONTEXT) + .putAll(testCase.getQueryContext()) + .build() ) .addCustomVerification(QueryVerification.ofResults(testCase)) .run(); } } - @Test - public void testEmptyWindowInSubquery() - { - testBuilder() - .sql( - "select c from (\n" - + " select channel, row_number() over () as c\n" - + " from wikipedia\n" - + " group by channel\n" - + ") LIMIT 5" - ) - .queryContext(ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true - )) - .expectedResults(ImmutableList.of( - new Object[]{1L}, - new Object[]{2L}, - new Object[]{3L}, - new Object[]{4L}, - new Object[]{5L} - )) - .run(); - } - - @Test - public void testWindow() - { - testBuilder() - .sql("SELECT\n" + - "(rank() over (order by count(*) desc)),\n" + - "(rank() over (order by count(*) desc))\n" + - "FROM \"wikipedia\"") - .queryContext(ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true - )) - .expectedResults(ImmutableList.of( - new Object[]{1L, 1L} - )) - .run(); - } - - @Test - public void testWindowAllBoundsCombination() - { - testBuilder() - .sql("select\n" - + "cityName,\n" - + "count(*) over (partition by cityName order by countryName rows between unbounded preceding and 1 preceding) c1,\n" - + "count(*) over (partition by cityName order by countryName rows between unbounded preceding and current row) c2,\n" - + "count(*) over (partition by cityName order by countryName rows between unbounded preceding and 1 following) c3,\n" - + "count(*) over (partition by cityName order by countryName rows between unbounded preceding and unbounded following) c4,\n" - + "count(*) over (partition by cityName order by countryName rows between 3 preceding and 1 preceding) c5,\n" - + "count(*) over (partition by cityName order by countryName rows between 1 preceding and current row) c6,\n" - + "count(*) over (partition by cityName order by countryName rows between 1 preceding and 1 FOLLOWING) c7,\n" - + "count(*) over (partition by cityName order by countryName rows between 1 preceding and unbounded FOLLOWING) c8,\n" - + "count(*) over (partition by cityName order by countryName rows between 1 FOLLOWING and unbounded FOLLOWING) c9,\n" - + "count(*) over (partition by cityName order by countryName rows between 1 FOLLOWING and 3 FOLLOWING) c10,\n" - + "count(*) over (partition by cityName order by countryName rows between current row and 1 following) c11,\n" - + "count(*) over (partition by cityName order by countryName rows between current row and unbounded following) c12\n" - + "from wikipedia\n" - + "where cityName in ('Vienna', 'Seoul')\n" - + "group by countryName, cityName, added") - .queryContext(ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true - )) - .expectedResults(ImmutableList.of( - new Object[]{"Seoul", 0L, 1L, 2L, 13L, 0L, 1L, 2L, 13L, 12L, 3L, 2L, 13L}, - new Object[]{"Seoul", 1L, 2L, 3L, 13L, 1L, 2L, 3L, 13L, 11L, 3L, 2L, 12L}, - new Object[]{"Seoul", 2L, 3L, 4L, 13L, 2L, 2L, 3L, 12L, 10L, 3L, 2L, 11L}, - new Object[]{"Seoul", 3L, 4L, 5L, 13L, 3L, 2L, 3L, 11L, 9L, 3L, 2L, 10L}, - new Object[]{"Seoul", 4L, 5L, 6L, 13L, 3L, 2L, 3L, 10L, 8L, 3L, 2L, 9L}, - new Object[]{"Seoul", 5L, 6L, 7L, 13L, 3L, 2L, 3L, 9L, 7L, 3L, 2L, 8L}, - new Object[]{"Seoul", 6L, 7L, 8L, 13L, 3L, 2L, 3L, 8L, 6L, 3L, 2L, 7L}, - new Object[]{"Seoul", 7L, 8L, 9L, 13L, 3L, 2L, 3L, 7L, 5L, 3L, 2L, 6L}, - new Object[]{"Seoul", 8L, 9L, 10L, 13L, 3L, 2L, 3L, 6L, 4L, 3L, 2L, 5L}, - new Object[]{"Seoul", 9L, 10L, 11L, 13L, 3L, 2L, 3L, 5L, 3L, 3L, 2L, 4L}, - new Object[]{"Seoul", 10L, 11L, 12L, 13L, 3L, 2L, 3L, 4L, 2L, 2L, 2L, 3L}, - new Object[]{"Seoul", 11L, 12L, 13L, 13L, 3L, 2L, 3L, 3L, 1L, 1L, 2L, 2L}, - new Object[]{"Seoul", 12L, 13L, 13L, 13L, 3L, 2L, 2L, 2L, 0L, 0L, 1L, 1L}, - new Object[]{"Vienna", 0L, 1L, 2L, 3L, 0L, 1L, 2L, 3L, 2L, 2L, 2L, 3L}, - new Object[]{"Vienna", 1L, 2L, 3L, 3L, 1L, 2L, 3L, 3L, 1L, 1L, 2L, 2L}, - new Object[]{"Vienna", 2L, 3L, 3L, 3L, 2L, 2L, 2L, 2L, 0L, 0L, 1L, 1L} - )) - .run(); - } - @Test public void testWithArrayConcat() { @@ -330,10 +232,7 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest + "where countryName in ('Austria', 'Republic of Korea') " + "and (cityName in ('Vienna', 'Seoul') or cityName is null)\n" + "group by countryName, cityName, channel") - .queryContext(ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true - )) + .queryContext(DEFAULT_QUERY_CONTEXT) .expectedResults( ResultMatchMode.RELAX_NULLS, ImmutableList.of( @@ -366,9 +265,13 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest failingTest, operatorValidation } + @JsonProperty public TestType type; + @JsonProperty + public Map queryContext; + @JsonProperty public String sql; 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 4e958383945..4a2f0945087 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 @@ -4426,7 +4426,7 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.DISTINCT_AGGREGATE_NOT_SUPPORTED) @DrillTest("nestedAggs/emtyOvrCls_7") @Test public void test_nestedAggs_emtyOvrCls_7() @@ -7274,7 +7274,7 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) + @NotYetSupported(Modes.DISTINCT_AGGREGATE_NOT_SUPPORTED) @DrillTest("nestedAggs/emtyOvrCls_8") @Test public void test_nestedAggs_emtyOvrCls_8() @@ -7607,4 +7607,168 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest { windowQueryTest(); } + + @DrillTest("druid_queries/empty_over_clause/single_empty_over_1") + @Test + public void test_empty_over_single_empty_over_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/empty_over_clause/single_empty_over_2") + @Test + public void test_empty_over_single_empty_over_2() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/empty_over_clause/multiple_empty_over_1") + @Test + public void test_empty_over_multiple_empty_over_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_sorting/single_over_1") + @Test + public void test_over_clause_with_only_sorting_single_over_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_sorting/single_over_2") + @Test + public void test_over_clause_with_only_sorting_single_over_2() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_sorting/single_over_multiple_sort_columns") + @Test + public void test_over_clause_with_only_sorting_single_over_multiple_sort_columns() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_sorting/multiple_over_same_sort_column") + @Test + public void test_over_clause_with_only_sorting_multiple_over_same_sort_column() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_sorting/multiple_over_different_sort_column") + @Test + public void test_over_clause_with_only_sorting_multiple_over_different_sort_column() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_1") + @Test + public void test_over_clause_with_only_sorting_multiple_over_multiple_sort_columns_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_2") + @Test + public void test_over_clause_with_only_sorting_multiple_over_multiple_sort_columns_2() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_partitioning/single_over_1") + @Test + public void test_over_clause_with_only_partitioning_single_over_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_partitioning/single_over_2") + @Test + public void test_over_clause_with_only_partitioning_single_over_2() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_partitioning/single_over_multiple_partition_columns") + @Test + public void test_over_clause_with_only_partitioning_single_over_multiple_partition_columns() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_partitioning/multiple_over_same_partition_column") + @Test + public void test_over_clause_with_only_partitioning_multiple_over_same_partition_column() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_partitioning/multiple_over_different_partition_column") + @Test + public void test_over_clause_with_only_partitioning_multiple_over_different_partition_column() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_1") + @Test + public void test_over_clause_with_only_partitioning_multiple_over_multiple_partition_columns_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_2") + @Test + public void test_over_clause_with_only_partitioning_multiple_over_multiple_partition_columns_2() + { + windowQueryTest(); + } + + // This test gives the following error on sql-native engine: + // Column[w0] of type[class org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn] cannot be sorted. + @DrillTest("druid_queries/empty_and_non_empty_over/wikipedia_query_1") + @Test + public void test_empty_and_non_empty_over_wikipedia_query_1() + { + sqlNativeIncompatible(); + windowQueryTest(); + } + + @DrillTest("druid_queries/empty_and_non_empty_over/wikipedia_query_2") + @Test + public void test_empty_and_non_empty_over_wikipedia_query_2() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/empty_and_non_empty_over/wikipedia_query_3") + @Test + public void test_empty_and_non_empty_over_wikipedia_query_3() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/partition_by_array/wikipedia_query_1") + @Test + public void test_partition_by_array_wikipedia_query_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/partition_by_array/wikipedia_query_2") + @Test + public void test_partition_by_array_wikipedia_query_2() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/partition_by_array/wikipedia_query_3") + @Test + public void test_partition_by_array_wikipedia_query_3() + { + windowQueryTest(); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java index 97556b16fb2..29200730fee 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java @@ -313,19 +313,7 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest " format => 'csv'))\n" + " EXTEND (x VARCHAR, y VARCHAR, z BIGINT)\n" + "PARTITIONED BY ALL TIME"; - final String explanation = "[{" + - "\"query\":{\"queryType\":\"scan\"," + - "\"dataSource\":{\"type\":\"external\"," + - "\"inputSource\":{\"type\":\"http\",\"uris\":[\"http://foo.com/bar.csv\"],\"httpAuthenticationUsername\":\"bob\",\"httpAuthenticationPassword\":{\"type\":\"default\",\"password\":\"secret\"}}," + - "\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]}," + - "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," + - "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"legacy\":false," + - "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"," + - "\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\"," + - "\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," + - "\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"}}," + - "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]," + - "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"http://foo.com/bar.csv\"],\"httpAuthenticationUsername\":\"bob\",\"httpAuthenticationPassword\":{\"type\":\"default\",\"password\":\"secret\"}},\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"}}"; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java index 5d53593b7ce..e5442a2bda2 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java @@ -77,7 +77,7 @@ public @interface NotYetSupported enum Modes { // @formatter:off - NOT_ENOUGH_RULES(DruidException.class, "not enough rules"), + DISTINCT_AGGREGATE_NOT_SUPPORTED(DruidException.class, "DISTINCT is not supported"), ERROR_HANDLING(AssertionError.class, "targetPersona: is <[A-Z]+> and category: is <[A-Z_]+> and errorCode: is"), EXPRESSION_NOT_GROUPED(DruidException.class, "Expression '[a-z]+' is not being grouped"), NULLS_FIRST_LAST(DruidException.class, "NULLS (FIRST|LAST)"), @@ -89,7 +89,7 @@ public @interface NotYetSupported RESULT_MISMATCH(AssertionError.class, "(assertResulEquals|AssertionError: column content mismatch)"), LONG_CASTING(AssertionError.class, "expected: java.lang.Long"), UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering with NULLS (LAST|FIRST)"), - WINDOW_OPERATOR_QUERY_ON_UNSUPPORTED_DATASOURCE(DruidException.class, "WindowOperatorQuery must run on top of a query or inline data source"), + CANNOT_RETRIEVE_ROWS(UnsupportedOperationException.class, "Cannot retrieve number of rows from join segment"), UNION_WITH_COMPLEX_OPERAND(DruidException.class, "Only Table and Values are supported as inputs for Union"), UNION_MORE_STRICT_ROWTYPE_CHECK(DruidException.class, "Row signature mismatch in Union inputs"), JOIN_CONDITION_NOT_PUSHED_CONDITION(DruidException.class, "SQL requires a join with '.*' condition"), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java index 86b5d38639b..c94eac39703 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java @@ -24,6 +24,8 @@ import com.google.common.collect.ImmutableMap; import org.apache.calcite.avatica.util.TimeUnit; import org.apache.calcite.avatica.util.TimeUnitRange; import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexWindowBounds; import org.apache.calcite.sql.SqlFunction; import org.apache.calcite.sql.SqlIntervalQualifier; import org.apache.calcite.sql.SqlOperator; @@ -72,7 +74,9 @@ import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.DruidOperatorTable; import org.apache.druid.sql.calcite.planner.DruidTypeSystem; import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.CannotBuildQueryException; import org.apache.druid.sql.calcite.util.CalciteTestBase; +import org.hamcrest.core.StringContains; import org.joda.time.DateTimeZone; import org.joda.time.Period; import org.junit.Assert; @@ -84,6 +88,8 @@ import java.math.BigDecimal; import java.util.Collections; import java.util.Map; +import static org.hamcrest.MatcherAssert.assertThat; + public class ExpressionsTest extends CalciteTestBase { private static final RowSignature ROW_SIGNATURE = RowSignature @@ -2827,6 +2833,36 @@ public class ExpressionsTest extends CalciteTestBase ); } + @Test + public void testPresenceOfOverIsInvalid() + { + final RexBuilder rexBuilder = new RexBuilder(DruidTypeSystem.TYPE_FACTORY); + final PlannerContext plannerContext = Mockito.mock(PlannerContext.class); + Mockito.when(plannerContext.getTimeZone()).thenReturn(DateTimeZone.UTC); + + RexNode rexNode = rexBuilder.makeOver( + testHelper.createSqlType(SqlTypeName.BIGINT), + SqlStdOperatorTable.SUM, + Collections.emptyList(), + Collections.emptyList(), + ImmutableList.of(), + RexWindowBounds.CURRENT_ROW, + RexWindowBounds.CURRENT_ROW, + false, + true, + false, + false, + false + ); + + CannotBuildQueryException t = Assert.assertThrows( + CannotBuildQueryException.class, + () -> testHelper.testExpression(rexNode, null, plannerContext) + ); + + assertThat(t.getMessage(), StringContains.containsString("Unexpected OVER expression")); + } + @Test public void testCalciteLiteralToDruidLiteral() { diff --git a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq index e1ae27eef0d..05f67289ab5 100644 --- a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq +++ b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq @@ -101,11 +101,11 @@ select v.*,e.* from v inner join e on (e.cityName = v.cityName); }, "resultFormat" : "compactedList", "columns" : [ "a0", "d0", "j0.a0", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING", "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan LogicalProject(cityName=[$0], cnt=[$1], cityName0=[$2], cnt0=[$3]) diff --git a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/simple.iq b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/simple.iq index 92ce8c26172..7d4ed9909b8 100644 --- a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/simple.iq +++ b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/simple.iq @@ -39,11 +39,11 @@ select cityName, countryName from wikipedia where cityName='New York' limit 1; "matchValue" : "New York" }, "columns" : [ "cityName", "countryName" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan LogicalSort(fetch=[1]) diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinLeftFunction.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinLeftFunction.iq index 7cbc21bd551..07245be4ca1 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinLeftFunction.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinLeftFunction.iq @@ -60,11 +60,11 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default.iq index f9c384b6ad2..794195713ed 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default.iq @@ -82,11 +82,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -108,11 +108,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=sql.iq index 99b6b8653fc..188abdd4e4c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=sql.iq @@ -83,11 +83,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,11 +110,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default.iq index 191120de0f7..37a87f4a74c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default.iq @@ -82,11 +82,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -108,11 +108,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=sql.iq index 8b2a57ef9c5..b1bca799e36 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=sql.iq @@ -83,11 +83,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,11 +110,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default.iq index c327a62751a..dbc1d7eaf96 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default.iq @@ -79,11 +79,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,11 +105,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=sql.iq index 404071a7b2a..0f5d1b8fe12 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=sql.iq @@ -80,11 +80,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -107,11 +107,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default.iq index 614328ce40a..fe58d1e4ebe 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default.iq @@ -82,11 +82,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -108,11 +108,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=sql.iq index f100ba33fd5..5fe30d6b657 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=sql.iq @@ -83,11 +83,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,11 +110,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default.iq index 03d22e1c0d5..57accf2fbc3 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default.iq @@ -82,11 +82,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -108,11 +108,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=sql.iq index ad90e641a99..4883e3b9894 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=sql.iq @@ -83,11 +83,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,11 +110,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default.iq index bfb82dce17d..072fac7ae75 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default.iq @@ -82,11 +82,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -108,11 +108,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=sql.iq index a2866beb2ed..f874137e09f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=sql.iq @@ -83,11 +83,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,11 +110,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default.iq index da25c7a6524..b3b0e2a346c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default.iq @@ -82,11 +82,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -108,11 +108,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=sql.iq index 20124df76e2..8980b7c4160 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=sql.iq @@ -83,11 +83,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,11 +110,11 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=default.iq index 4581d237a81..b834b63636a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=default.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=sql.iq index 588444b911a..2449b942774 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=sql.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=default.iq index 98f34424e83..b898d2f1c0d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=default.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=sql.iq index 5d6e4aa2941..fb1f9aa9151 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=sql.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=default.iq index 85dffdbac3d..41c9b0393aa 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=default.iq @@ -57,11 +57,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -78,10 +78,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=sql.iq index 28cc7fc8f79..b0a97e2b299 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=sql.iq @@ -57,11 +57,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -78,10 +78,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default.iq index b787c9f6b2c..3b58962a142 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=sql.iq index 3ca909dbbe7..2f250bce5b9 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=sql.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default.iq index 23363f31d94..0c6f38cfafb 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=sql.iq index 20f06ae17de..1dbe2ef067f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=sql.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=default.iq index 7ec2378fe2a..8783d7b3734 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=default.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=sql.iq index fca3275f788..bdc92546763 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=sql.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=default.iq index a17fe3fbce4..517a5e8f148 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=default.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=sql.iq index 94cf3420227..9593087abad 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=sql.iq @@ -60,11 +60,11 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "j0.k", "j0.v", "m1" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_disabled.iq index e5bd0124fcf..098749d6225 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_disabled.iq @@ -63,11 +63,11 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -84,10 +84,10 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_enabled.iq index 659a20e4533..19d8ce99529 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_enabled.iq @@ -63,11 +63,11 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -84,10 +84,10 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@default.iq index 44f2c01da3c..a5092ed5c4c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@default.iq @@ -60,11 +60,11 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -81,10 +81,10 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-on-value-column_disabled.iq index cb70cfd6786..fc27dfddef6 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-on-value-column_disabled.iq @@ -63,11 +63,11 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -84,10 +84,10 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites-disabled.iq index c6a24964fbf..b16cd786e18 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites-disabled.iq @@ -63,11 +63,11 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -84,10 +84,10 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites.iq index 238d9fcddb9..3fc0468e7a3 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites.iq @@ -63,11 +63,11 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -84,10 +84,10 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@join-to-filter.iq index f5a8444cd95..8df1ddf12e0 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@join-to-filter.iq @@ -63,11 +63,11 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -84,10 +84,10 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.k", "j0.v" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq index 1c97bb7ee96..e0132c50e97 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq index d3cf5105a91..e95f2f14e85 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq index bd8b49ead4e..b7fe7b9911f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq index 3fdf39476ee..f5cf5cd53aa 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq index 2f80224ec77..a1a9eb3a74e 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq @@ -72,11 +72,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -98,11 +98,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -121,10 +121,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq index 5dafdf1cb58..4f1a6c50e7d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq @@ -73,11 +73,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,11 +100,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -123,10 +123,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq index 94526e1ddb2..e499257505e 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq index dd795fb1c51..047a0d9f3d6 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq index 9db67c9d733..2ed19caa6dd 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq index e60075a6574..6e1a7ec7603 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq index b07e13731bb..7ba1e5e14d7 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq index cb5807a2693..d5e130f03a5 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq index 6ebe59515c8..609ae044013 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq index 935f19e21a6..1753c081a4b 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq index a644d80a90d..20e69c705a2 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq index d77f13502a3..2d034cb9200 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq index f905ad0706e..9d96525fbea 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq index 64d4eddf488..b5533f93632 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq index b7c64a78847..1c782f5f394 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq @@ -72,11 +72,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -98,11 +98,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -121,10 +121,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq index 11994dcdafb..62cc25dc99d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq @@ -73,11 +73,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,11 +100,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -123,10 +123,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq index 0410dbbc2dd..e2d47b9b815 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq index 81b0a62d042..27133219625 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq index 5335021cba1..db70d749d5b 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq index 8be64f51fc3..f2d6e9b0a0b 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq index 7c7b1545637..d17043dbd49 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq index 2750b09b2ad..ef9aff735e4 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq index 8bf56ebc476..eebc5217765 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq index 8190f45efaf..8578eedb067 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq index fbff0879b5c..da6ada101d4 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq @@ -104,10 +104,10 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.a0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq index 3409cd00e56..30482302b2e 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq @@ -104,10 +104,10 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.a0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq index 0c4f8abd6f9..ec13b17cbc0 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq @@ -101,10 +101,10 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.a0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq index 3d61195af63..ad63ef44426 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq @@ -104,10 +104,10 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.a0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq index 30b667f33cb..48890391a6b 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq @@ -104,10 +104,10 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.a0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq index 279e3a590c0..1fe9c554541 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq @@ -104,10 +104,10 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.a0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq index 52c6bc9980a..0ad9125b151 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq @@ -104,10 +104,10 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.a0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_disabled.iq index c8d92aff808..0784836b05d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_disabled.iq @@ -70,11 +70,11 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -87,10 +87,10 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_enabled.iq index 95cdbe25858..00ec67ad0cb 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_enabled.iq @@ -70,11 +70,11 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -87,10 +87,10 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@default.iq index 3fb227e9c34..c9c45c4e75e 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@default.iq @@ -67,11 +67,11 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -84,10 +84,10 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled.iq index 94433129db0..8c077ff9936 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled.iq @@ -70,11 +70,11 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -87,10 +87,10 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled.iq index 87be9556a35..b4157fdc894 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled.iq @@ -70,11 +70,11 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -87,10 +87,10 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites.iq index 207003ad6f3..92ba66af86a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites.iq @@ -70,11 +70,11 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -87,10 +87,10 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@join-to-filter.iq index c2fc24a6307..a47285f0c41 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@join-to-filter.iq @@ -70,11 +70,11 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -87,10 +87,10 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "dim2", "j0.v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default.iq index c395fdf3ba9..c08b970f161 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default.iq @@ -70,11 +70,11 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3)], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "dim3", "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=sql.iq index 4c8cbccbacf..07124f8cac0 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=sql.iq @@ -70,11 +70,11 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3)], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "dim3", "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default.iq index 5478c6b9381..2f78608ebff 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default.iq @@ -70,11 +70,11 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "dim3", "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=sql.iq index 5e815f2c3cb..2a03f6e2516 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=sql.iq @@ -70,11 +70,11 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "dim3", "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=default.iq index 7167a4ce8db..f2275d4db00 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=default.iq @@ -70,11 +70,11 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "dim3", "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=sql.iq index 073386da08c..077841f3fe9 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=sql.iq @@ -70,11 +70,11 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "dim3", "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=default.iq index babde45b3ce..b29ee72ab4f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=default.iq @@ -59,11 +59,11 @@ DruidAggregate(group=[{}], EXPR$0=[COUNT() FILTER $0], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=sql.iq index 7b233392a0e..1f2ac13cefe 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=sql.iq @@ -67,11 +67,11 @@ DruidAggregate(group=[{}], EXPR$0=[COUNT() FILTER $0], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq index d4dfbd0ad1d..5df8aeeaff3 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq index bcefbd4c4eb..d43b5711444 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq index daa8e4e768f..76955adfbeb 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq index 9645a60b007..05f624ac37a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq index 33c10207f8f..4d2b6332c5b 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq @@ -72,11 +72,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -98,11 +98,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -121,10 +121,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq index d9d19e39d77..31c46ccc3b2 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq @@ -73,11 +73,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,11 +100,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -123,10 +123,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq index 5b7547275fa..05a49528a40 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq index a15a61f877c..07d5d256b61 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq index 12362cdaaaa..61e0354cd15 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq index 91b4312973e..d764e7d5e3c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq index e7dc799ba59..2103374735d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq index b1d869fc008..ae6924c1b87 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq index 9d917f9bc97..b86beedd9ab 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq index 8df872ddb8a..904f6fec8d8 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default.iq index 18d5b2c281e..bfdea514aa3 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -107,11 +107,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -130,10 +130,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=sql.iq index 223cf55e124..4e6d825da40 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,11 +109,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -132,10 +132,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default.iq index 38c060edc25..dd9ac1e021d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -107,11 +107,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -130,10 +130,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=sql.iq index 399581bceb5..ea1bcd75f44 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,11 +109,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -132,10 +132,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default.iq index e151ac97f52..10f1c84c330 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default.iq @@ -72,11 +72,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -104,11 +104,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,10 +127,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=sql.iq index 6fb8344b350..ebe0fbc3f71 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=sql.iq @@ -73,11 +73,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -106,11 +106,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -129,10 +129,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq index 0005b6f2199..ea5fdff6a88 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -107,11 +107,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -130,10 +130,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq index b58ccfb7442..dcd84462bbe 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,11 +109,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -132,10 +132,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq index 1b66e080ef2..d870dc637b8 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -107,11 +107,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -130,10 +130,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq index 466504c9ee4..11442bae8a5 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,11 +109,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -132,10 +132,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq index dfc9e307dca..560f7c30f74 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -107,11 +107,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -130,10 +130,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq index a9678e70060..4f254f1976b 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,11 +109,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -132,10 +132,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default.iq index 583c592d396..ff2df59c513 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -107,11 +107,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -130,10 +130,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq index 689bf3b8562..5d8f487a780 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,11 +109,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -132,10 +132,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq index bbde46874f6..d13c86ab395 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq index 27682661b4c..aa4220f4f42 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq index 28a80ff003c..aec23daced0 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq index c27a4b7a120..844b53dccc3 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq index 69788862333..4c84575901e 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq @@ -72,11 +72,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -98,11 +98,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -121,10 +121,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq index a4342bed659..ccbdf92c457 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq @@ -73,11 +73,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,11 +100,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -123,10 +123,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq index a18be661b59..c22c33b3ea4 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq index 6f2c0cfa476..68f6a9250e6 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq index bad3fea5e40..aa6d4b632ac 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq index c4ee8dcac08..9c1a8d23796 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq index a69dfe749f6..993c14f6869 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq index 69f6925ce7e..76ee90cd877 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq index c6de594d4ba..cf7cdcc09f3 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -75,11 +75,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -101,11 +101,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,10 +124,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq index 63131da8ae3..ecd48b65d91 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq @@ -76,11 +76,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,11 +103,11 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -126,10 +126,10 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq index f69e5c22b5e..2fb49c488d8 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq @@ -91,10 +91,10 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq index 0fcfdbe725d..f8c20e42d6a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq @@ -91,10 +91,10 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq index b2f470dcb74..a365f6899bf 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq @@ -88,10 +88,10 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq index e0b664541d4..398b3cc3a85 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq @@ -91,10 +91,10 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq index f9e5916b799..5c86e793f30 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq @@ -91,10 +91,10 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq index 07aac4aeaca..e1be3203710 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq @@ -91,10 +91,10 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq index 2b975eab86a..590296d4904 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq @@ -91,10 +91,10 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default.iq index bc0fcae56d4..dcff8e213ae 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=sql.iq index 915925987a1..cb04db28506 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=sql.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default.iq index 612567cb0a3..e45394da7f8 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=sql.iq index b08fd8766fd..105c345ee9c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=sql.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=default.iq index 4f3ddcb829f..ad6509bd826 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=default.iq @@ -62,11 +62,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=sql.iq index e158564b801..6dffa2cc78d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=sql.iq @@ -62,11 +62,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default.iq index 021473e3f44..346a65b9504 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=sql.iq index ac02ef44b47..1bd6bffa1f2 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=sql.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default.iq index 5d2e94ea90f..9b72fc597c0 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=sql.iq index 44563315f01..b27b5b8cb57 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=sql.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default.iq index c771a5e6a9f..c2c40dc9952 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=sql.iq index 4e31bf0d985..c03d202cdae 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=sql.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default.iq index e8201c2800a..b08872faf7d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=sql.iq index 62bfdc1a0c5..40229ea1ec1 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=sql.iq @@ -65,11 +65,11 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_disabled.iq index 389fb0eeaef..8c460cad242 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_disabled.iq @@ -67,11 +67,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,11 +100,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -117,10 +117,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_enabled.iq index 01e268234fc..ed9c27dc2d5 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_enabled.iq @@ -67,11 +67,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,11 +100,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -117,10 +117,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@default.iq index ae58b2deb85..4a3b1ddb5cf 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@default.iq @@ -64,11 +64,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -97,11 +97,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -114,10 +114,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled.iq index ec901fab83b..5bc156562d7 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled.iq @@ -67,11 +67,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,11 +100,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -117,10 +117,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled.iq index be025d6afb7..8b2cffb58c2 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled.iq @@ -67,11 +67,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,11 +100,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -117,10 +117,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites.iq index 95250d0012d..d196685bbe4 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites.iq @@ -67,11 +67,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,11 +100,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -117,10 +117,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@join-to-filter.iq index 6028287a7be..d29ec916a0f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@join-to-filter.iq @@ -67,11 +67,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,11 +100,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -117,10 +117,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled.iq index c026e25e85c..af6c617d6d6 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled.iq @@ -75,11 +75,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -114,11 +114,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -147,11 +147,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,11 +176,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -193,10 +193,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled.iq index 2e13a83a2a8..e82f708d2bb 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled.iq @@ -75,11 +75,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -114,11 +114,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -147,11 +147,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,11 +176,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -193,10 +193,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@default.iq index fe86f2126f5..5c154e61c29 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@default.iq @@ -72,11 +72,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -111,11 +111,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -144,11 +144,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -173,11 +173,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -190,10 +190,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled.iq index 40462afd1d0..8e960092276 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled.iq @@ -75,11 +75,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -114,11 +114,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -147,11 +147,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,11 +176,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -193,10 +193,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled.iq index 8a33035d040..a258e98d01f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled.iq @@ -75,11 +75,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -114,11 +114,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -147,11 +147,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,11 +176,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -193,10 +193,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites.iq index d0736234946..63ea77bfbbe 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites.iq @@ -75,11 +75,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -114,11 +114,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -147,11 +147,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,11 +176,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -193,10 +193,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter.iq index a844b048a2e..8f9b506aa1a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter.iq @@ -75,11 +75,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -114,11 +114,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -147,11 +147,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,11 +176,11 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -193,10 +193,10 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default.iq index e7322a876fc..8bc6869aff8 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default.iq @@ -222,10 +222,10 @@ DruidProject(dim2=[$0], EXPR$1=[$1], EXPR$2=[$3], EXPR$3=[$5], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.a0", "a0", "d0", "j0.a0" ], - "legacy" : false, "columnTypes" : [ "LONG", "LONG", "STRING", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql.iq index 098415f6b1b..0f899348f83 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql.iq @@ -233,10 +233,10 @@ DruidProject(dim2=[$0], EXPR$1=[$1], EXPR$2=[$3], EXPR$3=[$5], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.a0", "a0", "d0", "j0.a0" ], - "legacy" : false, "columnTypes" : [ "LONG", "LONG", "STRING", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=default.iq index 5368d23981f..8b2ca662621 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=default.iq @@ -139,11 +139,11 @@ DruidProject(ranking=[$2], trend_score=[$1], druid=[logical]) "resultFormat" : "compactedList", "limit" : 10, "columns" : [ "a0", "d1", "w0" ], - "legacy" : false, "columnTypes" : [ "LONG", "DOUBLE", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "intervals" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=sql.iq index 86e22e97353..ae521d6b098 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=sql.iq @@ -150,11 +150,11 @@ DruidProject(ranking=[$2], trend_score=[$1], druid=[logical]) "resultFormat" : "compactedList", "limit" : 10, "columns" : [ "a0", "d1", "w0" ], - "legacy" : false, "columnTypes" : [ "LONG", "DOUBLE", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "intervals" : { diff --git a/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest b/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest index 0c9d88b5041..a2f82ff2905 100644 --- a/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest @@ -15,13 +15,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: - peerType: "ROWS" - lowUnbounded: true - lowOffset: 0 - uppUnbounded: true - uppOffset: 0 - orderBy: null + frame: { type: "rows" } aggregations: - { "type": "doubleSum", "name": "w1", "fieldName": "_d1" } - type: "naiveSort" @@ -33,13 +27,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: - peerType: "ROWS" - lowUnbounded: true - lowOffset: 0 - uppUnbounded: true - uppOffset: 0 - orderBy: null + frame: { type: "rows" } aggregations: - { "type": "doubleSum", "name": "w0", "fieldName": "_d0" } expectedResults: diff --git a/sql/src/test/resources/calcite/tests/window/aggregateConstant.sqlTest b/sql/src/test/resources/calcite/tests/window/aggregateConstant.sqlTest index 16dbe924fdb..e65e27e8794 100644 --- a/sql/src/test/resources/calcite/tests/window/aggregateConstant.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/aggregateConstant.sqlTest @@ -13,7 +13,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: "rows" } aggregations: - { type: "count", name: "w0" } diff --git a/sql/src/test/resources/calcite/tests/window/allBoundsCombination.sqlTest b/sql/src/test/resources/calcite/tests/window/allBoundsCombination.sqlTest new file mode 100644 index 00000000000..af6b0451761 --- /dev/null +++ b/sql/src/test/resources/calcite/tests/window/allBoundsCombination.sqlTest @@ -0,0 +1,38 @@ +type: "operatorValidation" + +sql: | + select + cityName, + count(*) over (partition by cityName order by countryName rows between unbounded preceding and 1 preceding) c1, + count(*) over (partition by cityName order by countryName rows between unbounded preceding and current row) c2, + count(*) over (partition by cityName order by countryName rows between unbounded preceding and 1 following) c3, + count(*) over (partition by cityName order by countryName rows between unbounded preceding and unbounded following) c4, + count(*) over (partition by cityName order by countryName rows between 3 preceding and 1 preceding) c5, + count(*) over (partition by cityName order by countryName rows between 1 preceding and current row) c6, + count(*) over (partition by cityName order by countryName rows between 1 preceding and 1 FOLLOWING) c7, + count(*) over (partition by cityName order by countryName rows between 1 preceding and unbounded FOLLOWING) c8, + count(*) over (partition by cityName order by countryName rows between 1 FOLLOWING and unbounded FOLLOWING) c9, + count(*) over (partition by cityName order by countryName rows between 1 FOLLOWING and 3 FOLLOWING) c10, + count(*) over (partition by cityName order by countryName rows between current row and 1 following) c11, + count(*) over (partition by cityName order by countryName rows between current row and unbounded following) c12 + from wikipedia + where cityName in ('Vienna', 'Seoul') + group by countryName, cityName, added + +expectedResults: + - ["Seoul",0,1,2,13,0,1,2,13,12,3,2,13] + - ["Seoul",1,2,3,13,1,2,3,13,11,3,2,12] + - ["Seoul",2,3,4,13,2,2,3,12,10,3,2,11] + - ["Seoul",3,4,5,13,3,2,3,11,9,3,2,10] + - ["Seoul",4,5,6,13,3,2,3,10,8,3,2,9] + - ["Seoul",5,6,7,13,3,2,3,9,7,3,2,8] + - ["Seoul",6,7,8,13,3,2,3,8,6,3,2,7] + - ["Seoul",7,8,9,13,3,2,3,7,5,3,2,6] + - ["Seoul",8,9,10,13,3,2,3,6,4,3,2,5] + - ["Seoul",9,10,11,13,3,2,3,5,3,3,2,4] + - ["Seoul",10,11,12,13,3,2,3,4,2,2,2,3] + - ["Seoul",11,12,13,13,3,2,3,3,1,1,2,2] + - ["Seoul",12,13,13,13,3,2,2,2,0,0,1,1] + - ["Vienna",0,1,2,3,0,1,2,3,2,2,2,3] + - ["Vienna",1,2,3,3,1,2,3,3,1,1,2,2] + - ["Vienna",2,3,3,3,2,2,2,2,0,0,1,1] diff --git a/sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest b/sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest index d5a324c9e2d..aa0a4a2a019 100644 --- a/sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest @@ -19,60 +19,43 @@ expectedOperators: processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: true - lowOffset: 0 - uppUnbounded: false - uppOffset: 0 - orderBy: null + type: "rows" + upperOffset: 0 aggregations: - { type: "count", name: "w0" } - type: "window" processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: -1 - uppUnbounded: false - uppOffset: 0 - orderBy: null + type: "rows" + lowerOffset: -1 + upperOffset: 0 aggregations: - { type: "count", name: "w1" } - type: "window" processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: 0 - uppUnbounded: false - uppOffset: 0 - orderBy: null + type: "rows" + lowerOffset: 0 + upperOffset: 0 aggregations: - { type: "count", name: "w2" } - type: "window" processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: 0 - uppUnbounded: false - uppOffset: 1 - orderBy: null + type: "rows" + lowerOffset: 0 + upperOffset: 1 aggregations: - { type: "count", name: "w3" } - type: "window" processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: 0 - uppUnbounded: true - uppOffset: 0 - orderBy: null + type: "rows" + lowerOffset: 0 aggregations: - { type: "count", name: "w4" } diff --git a/sql/src/test/resources/calcite/tests/window/duplicateAggregation.sqlTest b/sql/src/test/resources/calcite/tests/window/duplicateAggregation.sqlTest new file mode 100644 index 00000000000..fd79234a562 --- /dev/null +++ b/sql/src/test/resources/calcite/tests/window/duplicateAggregation.sqlTest @@ -0,0 +1,10 @@ +type: "operatorValidation" + +sql: | + select + rank() over (order by count(*) desc), + rank() over (order by count(*) desc) + from wikipedia + +expectedResults: + - [1,1] diff --git a/sql/src/test/resources/calcite/tests/window/no_grouping.sqlTest b/sql/src/test/resources/calcite/tests/window/no_grouping.sqlTest index 7c9dae4aad3..2b6f7f7fddb 100644 --- a/sql/src/test/resources/calcite/tests/window/no_grouping.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/no_grouping.sqlTest @@ -1,9 +1,9 @@ -type: "failingTest" +type: "operatorValidation" sql: | SELECT m1, - COUNT(m1) OVER () cc + SUM(m1) OVER () cc FROM druid.foo expectedOperators: @@ -12,18 +12,16 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: "rows" } aggregations: - - type: "filtered" - aggregator: {"type":"count","name":"w0"} - filter: - type: not - field: {"type":"null","column":"m1"} - name: null + - type: doubleSum + name: w0 + fieldName: m1 + expectedResults: - - [1.0,6] - - [2.0,6] - - [3.0,6] - - [4.0,6] - - [5.0,6] - - [6.0,6] + - [1.0,21.0] + - [2.0,21.0] + - [3.0,21.0] + - [4.0,21.0] + - [5.0,21.0] + - [6.0,21.0] diff --git a/sql/src/test/resources/calcite/tests/window/no_grouping2.sqlTest b/sql/src/test/resources/calcite/tests/window/no_grouping2.sqlTest index 7a579c3fc33..4d78b197e31 100644 --- a/sql/src/test/resources/calcite/tests/window/no_grouping2.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/no_grouping2.sqlTest @@ -12,7 +12,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: rows } aggregations: - type: "doubleSum" name: "w0" diff --git a/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest b/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest index 0e66ed87460..1e4de22dfca 100644 --- a/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest @@ -1,5 +1,8 @@ type: "operatorValidation" +queryContext: + maxSubqueryBytes: 100000 + sql: | SELECT __time diff --git a/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest b/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest index 9ca9f88e850..84bd5ca71af 100644 --- a/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest @@ -14,12 +14,9 @@ expectedOperators: processor: type: "framedAgg" frame: - peerType: "RANGE" - lowUnbounded: true - lowOffset: 0 - uppUnbounded: false - uppOffset: 0 - orderBy: [ {column: "d0", direction: ASC} ] + type: groups + upperOffset: 0 + orderByColumns: [ "d0" ] aggregations: - { type: "longSum", name: "w0", fieldName: "a0" } diff --git a/sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest b/sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest index 0a86a691e26..9b5aa6e1365 100644 --- a/sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest @@ -19,7 +19,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: "rows" } aggregations: - { type: "doubleMin", name: "w0", fieldName: "_v0" } - { type: "longMin", name: "w1", fieldName: "v1" } diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest index cc59868482a..b4ef8006ea9 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest @@ -16,12 +16,9 @@ expectedOperators: processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: -3 - uppUnbounded: false - uppOffset: 2 - orderBy: null + type: rows + lowerOffset: -3 + upperOffset: 2 aggregations: - { type: "longSum", name: "w0", fieldName: "a0" } - { type: "naiveSort", columns: [ { column: "d1", direction: "ASC" }, { column: "a0", direction: "ASC"} ]} diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest index 9368f00e9b4..ebcc060eaa5 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest @@ -39,14 +39,9 @@ expectedOperators: - { "type": "cumeDist", "group": [ "a0" ], "outputColumn": "w9" } - type: "framedAgg" frame: - peerType: "RANGE" - lowUnbounded: true - lowOffset: 0 - uppUnbounded: false - uppOffset: 0 - orderBy: - - column: a0 - direction: ASC + type: groups + upperOffset: 0 + orderByColumns: [ a0 ] aggregations: - { "type": "longSum", "name": "w0", "fieldName": "a0" } diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest index c25f1ff0352..87873d44c48 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest @@ -15,11 +15,9 @@ expectedOperators: processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: -3 - uppUnbounded: false - uppOffset: 2 + type: "rows" + lowerOffset: -3 + upperOffset: 2 orderBy: null aggregations: - { type: "longSum", name: "w0", fieldName: "a0" } diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest index 1e75e69b97b..3843519aa79 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest @@ -19,7 +19,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: "rows" } aggregations: - { "type": "longSum", "name": "w0", "fieldName": "a0" } - type: "window" diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartitionInitialSort.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartitionInitialSort.sqlTest index d310f6a8f1c..4939057621e 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartitionInitialSort.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartitionInitialSort.sqlTest @@ -28,7 +28,7 @@ expectedOperators: - { "type": "last", "inputColumn": "a0", "outputColumn": "w2" } - { "type": "percentile", "outputColumn": "w3", "numBuckets": 3 } - type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: "rows" } aggregations: - { "type": "longSum", "name": "w0", "fieldName": "a0" } - type: "window" diff --git a/sql/src/test/resources/calcite/tests/window/windowInsideSubquery.sqlTest b/sql/src/test/resources/calcite/tests/window/windowInsideSubquery.sqlTest new file mode 100644 index 00000000000..5672a5a17f3 --- /dev/null +++ b/sql/src/test/resources/calcite/tests/window/windowInsideSubquery.sqlTest @@ -0,0 +1,19 @@ +type: "operatorValidation" + +sql: | + select + c + from + ( + select channel, row_number() over () as c + from wikipedia + group by channel + ) + LIMIT 5 + +expectedResults: + - [1] + - [2] + - [3] + - [4] + - [5] diff --git a/sql/src/test/resources/calcite/tests/window/windowed_long_null.sqlTest b/sql/src/test/resources/calcite/tests/window/windowed_long_null.sqlTest index c96b979c0da..7c7fd03c3c8 100644 --- a/sql/src/test/resources/calcite/tests/window/windowed_long_null.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/windowed_long_null.sqlTest @@ -19,7 +19,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: rows } aggregations: - { type: "longMin", name: "w0", fieldName: "l2" } - type: "naiveSort" @@ -31,7 +31,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "RANGE", lowUnbounded: true, lowOffset: 0, uppUnbounded: false, uppOffset: 0, orderBy: [{ column: l1, direction: ASC }] } + frame: { type: groups, upperOffset: 0, orderByColumns: [ l1 ] } aggregations: - { type: "longMin", name: "w1", fieldName: "l2" } diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_1.e new file mode 100644 index 00000000000..b1913bc200c --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_1.e @@ -0,0 +1,14 @@ +Austria null #de.wikipedia 0 1 1 1 +Austria null #de.wikipedia 14 2 2 2 +Austria null #de.wikipedia 94 3 3 3 +Austria null #de.wikipedia 272 4 4 4 +Austria null #de.wikipedia 4685 5 5 5 +Austria null #de.wikipedia 6979 6 6 6 +Guatemala null #es.wikipedia 0 12 1 1 +Guatemala El Salvador #es.wikipedia 1 13 1 1 +Guatemala Guatemala City #es.wikipedia 173 14 1 1 +Austria Horsching #de.wikipedia 0 7 1 1 +Austria Vienna #de.wikipedia 0 8 1 1 +Austria Vienna #de.wikipedia 72 9 2 2 +Austria Vienna #es.wikipedia 0 10 3 1 +Austria Vienna #tr.wikipedia 93 11 4 1 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_1.q new file mode 100644 index 00000000000..b490523a0dc --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_1.q @@ -0,0 +1,8 @@ +select +countryName, cityName, channel, added, +row_number() over () as c1, +row_number() over (PARTITION BY countryName, cityName) as c2, +row_number() over (PARTITION BY cityName, channel ORDER BY channel) as c3 +from wikipedia +where countryName in ('Guatemala', 'Austria') +group by countryName, cityName, channel, added diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_2.e new file mode 100644 index 00000000000..dc9f77f71e0 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_2.e @@ -0,0 +1,14 @@ +Austria null #de.wikipedia 1 null null +Guatemala null #es.wikipedia 10 null null +Austria null #de.wikipedia 2 null null +Austria null #de.wikipedia 3 null null +Austria null #de.wikipedia 4 null null +Austria null #de.wikipedia 5 null null +Austria null #de.wikipedia 6 null null +Guatemala El Salvador #es.wikipedia 11 null null +Guatemala Guatemala City #es.wikipedia 12 null null +Austria Horsching #de.wikipedia 7 null null +Austria Vienna #de.wikipedia 8 null Vienna +Austria Vienna #es.wikipedia 13 Vienna null +Austria Vienna #de.wikipedia 9 Vienna null +Austria Vienna #tr.wikipedia 14 Vienna null diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_2.q new file mode 100644 index 00000000000..6dc676e3385 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_2.q @@ -0,0 +1,8 @@ +select +countryName, cityName, channel, +row_number() over () as c1, +lag(cityName) over (PARTITION BY countryName, cityName) as c2, +lead(cityName) over (PARTITION BY cityName, added) as c3 +from wikipedia +where countryName in ('Guatemala', 'Austria') +group by countryName, cityName, channel, added diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_3.e b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_3.e new file mode 100644 index 00000000000..8cd625edad2 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_3.e @@ -0,0 +1,16 @@ +Republic of Korea Seoul 0 0 16 4 2 null 1 2 0 0 1 2 13 null 12 3 1 2 13 +Republic of Korea Seoul 2 4 16 5 3 0 2 3 1 1 2 3 13 Austria 11 3 2 2 12 +Republic of Korea Seoul 4 4 16 6 4 2 3 4 2 2 2 3 12 Republic of Korea 10 3 3 2 11 +Republic of Korea Seoul 5 5 16 7 5 4 4 5 3 3 2 3 11 Republic of Korea 9 3 4 2 10 +Republic of Korea Seoul 6 6 16 8 6 5 5 6 4 3 2 3 10 Republic of Korea 8 3 5 2 9 +Republic of Korea Seoul 12 12 16 9 7 6 6 7 5 3 2 3 9 Republic of Korea 7 3 6 2 8 +Republic of Korea Seoul 15 15 16 10 8 12 7 8 6 3 2 3 8 Republic of Korea 6 3 7 2 7 +Republic of Korea Seoul 19 19 16 11 9 15 8 9 7 3 2 3 7 Republic of Korea 5 3 8 2 6 +Republic of Korea Seoul 22 22 16 12 10 19 9 10 8 3 2 3 6 Republic of Korea 4 3 9 2 5 +Republic of Korea Seoul 24 24 16 13 11 22 10 11 9 3 2 3 5 Republic of Korea 3 3 10 2 4 +Republic of Korea Seoul 243 243 16 14 14 24 11 12 10 3 2 3 4 Austria 2 2 11 2 3 +Republic of Korea Seoul 663 663 16 15 15 243 12 13 11 3 2 3 3 Republic of Korea 1 1 12 2 2 +Republic of Korea Seoul 1036 1036 16 16 16 663 13 13 12 3 2 2 2 Republic of Korea 0 0 13 1 1 +Austria Vienna 0 0 16 1 1 1036 1 2 0 0 1 2 3 Republic of Korea 2 2 1 2 3 +Austria Vienna 72 72 16 2 12 0 2 3 1 1 2 3 3 Republic of Korea 1 1 2 2 2 +Austria Vienna 93 93 16 3 13 72 3 3 2 2 2 2 2 Austria 0 0 3 1 1 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_3.q b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_3.q new file mode 100644 index 00000000000..041a4ef3963 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_and_non_empty_over/wikipedia_query_3.q @@ -0,0 +1,23 @@ +select +countryName, cityName, added, +sum(added), +count(added) over () e1, +ROW_NUMBER() over () e2, +ROW_NUMBER() over (order by added) e3, +lag(added) over (order by cityName, countryName) e4, +count(*) over (partition by cityName order by countryName rows between unbounded preceding and current row) c2, +count(*) over (partition by cityName order by countryName rows between unbounded preceding and 1 following) c3, +count(*) over (partition by cityName order by countryName rows between unbounded preceding and 1 preceding) c4, +count(*) over (partition by cityName order by countryName rows between 3 preceding and 1 preceding) c5, +count(*) over (partition by cityName order by countryName rows between 1 preceding and current row) c5, +count(*) over (partition by cityName order by countryName rows between 1 preceding and 1 FOLLOWING) c7, +count(*) over (partition by cityName order by countryName rows between 1 preceding and unbounded FOLLOWING) c5, +lag(countryName) over (order by added) e5, +count(*) over (partition by cityName order by countryName rows between 1 FOLLOWING and unbounded FOLLOWING) c5, +count(*) over (partition by cityName order by countryName rows between 1 FOLLOWING and 3 FOLLOWING) c10, +ROW_NUMBER() over (partition by cityName order by added) e6, +count(*) over (partition by cityName order by countryName rows between current row and 1 following) c11, +count(*) over (partition by cityName order by countryName rows between current row and unbounded following) c12 +from wikipedia +where cityName in ('Vienna', 'Seoul') +group by countryName, cityName, added diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/multiple_empty_over_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/multiple_empty_over_1.e new file mode 100644 index 00000000000..ad0dead8add --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/multiple_empty_over_1.e @@ -0,0 +1,13 @@ +Austria 1 null +Austria 2 Austria +Austria 3 Austria +Republic of Korea 4 Austria +Austria 5 Republic of Korea +Republic of Korea 6 Austria +Republic of Korea 7 Republic of Korea +Republic of Korea 8 Republic of Korea +Republic of Korea 9 Republic of Korea +Republic of Korea 10 Republic of Korea +Republic of Korea 11 Republic of Korea +Republic of Korea 12 Republic of Korea +Austria 13 Republic of Korea diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/multiple_empty_over_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/multiple_empty_over_1.q new file mode 100644 index 00000000000..b62aaabc7a2 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/multiple_empty_over_1.q @@ -0,0 +1,7 @@ +select +countryName, +row_number() over () as c1, +lag(countryName) over () as c2 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_1.e new file mode 100644 index 00000000000..722700444fc --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_1.e @@ -0,0 +1,13 @@ +Austria 1 +Austria 2 +Austria 3 +Republic of Korea 4 +Austria 5 +Republic of Korea 6 +Republic of Korea 7 +Republic of Korea 8 +Republic of Korea 9 +Republic of Korea 10 +Republic of Korea 11 +Republic of Korea 12 +Austria 13 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_1.q new file mode 100644 index 00000000000..47452f7c97f --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_1.q @@ -0,0 +1,4 @@ +select countryName, row_number() over () as c1 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_2.e new file mode 100644 index 00000000000..970b68a4d5c --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_2.e @@ -0,0 +1,13 @@ +Austria null +Austria Austria +Austria Austria +Republic of Korea Austria +Austria Republic of Korea +Republic of Korea Austria +Republic of Korea Republic of Korea +Republic of Korea Republic of Korea +Republic of Korea Republic of Korea +Republic of Korea Republic of Korea +Republic of Korea Republic of Korea +Republic of Korea Republic of Korea +Austria Republic of Korea diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_2.q new file mode 100644 index 00000000000..ac530688e53 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/empty_over_clause/single_empty_over_2.q @@ -0,0 +1,4 @@ +select countryName, lag(countryName) over () as c1 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_different_partition_column.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_different_partition_column.e new file mode 100644 index 00000000000..6a827d2285a --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_different_partition_column.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia 1 null +Austria Horsching #de.wikipedia 2 null +Austria Vienna #de.wikipedia 3 Horsching +Austria Vienna #es.wikipedia 1 Vienna +Austria Vienna #tr.wikipedia 1 Vienna +Republic of Korea null #en.wikipedia 1 null +Republic of Korea null #ja.wikipedia 1 null +Republic of Korea null #ko.wikipedia 1 null +Republic of Korea Jeonju #ko.wikipedia 2 null +Republic of Korea Seongnam-si #ko.wikipedia 3 Jeonju +Republic of Korea Seoul #ko.wikipedia 4 Seongnam-si +Republic of Korea Suwon-si #ko.wikipedia 5 Seoul +Republic of Korea Yongsan-dong #ko.wikipedia 6 Suwon-si diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_different_partition_column.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_different_partition_column.q new file mode 100644 index 00000000000..16819458691 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_different_partition_column.q @@ -0,0 +1,7 @@ +select +countryName, cityName, channel, +row_number() over (PARTITION BY channel) as c1, +lag(cityName) over (PARTITION BY countryName) as c2 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_1.e new file mode 100644 index 00000000000..fd2bc5ed2aa --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_1.e @@ -0,0 +1,9 @@ +Austria null 1 null +Austria Horsching 1 null +Austria Vienna 1 null +Republic of Korea null 1 null +Republic of Korea Jeonju 1 null +Republic of Korea Seongnam-si 1 null +Republic of Korea Seoul 1 null +Republic of Korea Suwon-si 1 null +Republic of Korea Yongsan-dong 1 null diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_1.q new file mode 100644 index 00000000000..25180612054 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_1.q @@ -0,0 +1,7 @@ +select +countryName, cityName, +row_number() over (PARTITION BY countryName, cityName) as c1, +lag(cityName) over (PARTITION BY cityName, countryName) as c2 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_2.e new file mode 100644 index 00000000000..e4005d233de --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_2.e @@ -0,0 +1,74 @@ +Austria null #de.wikipedia 0 1 null +Austria Horsching #de.wikipedia 0 1 null +Austria Vienna #de.wikipedia 0 1 Horsching +Austria null #de.wikipedia 14 2 null +Austria Vienna #de.wikipedia 72 2 null +Austria null #de.wikipedia 94 3 null +Austria null #de.wikipedia 272 4 null +Austria null #de.wikipedia 4685 5 null +Austria null #de.wikipedia 6979 6 null +Republic of Korea null #en.wikipedia 0 1 null +Republic of Korea null #en.wikipedia 5 2 null +Republic of Korea null #en.wikipedia 75 3 null +Austria Vienna #es.wikipedia 0 3 null +Republic of Korea null #ja.wikipedia 0 4 null +Republic of Korea null #ja.wikipedia 43 5 null +Republic of Korea null #ja.wikipedia 46 6 null +Republic of Korea null #ja.wikipedia 356 7 null +Republic of Korea null #ko.wikipedia 0 8 null +Republic of Korea Seoul #ko.wikipedia 0 1 null +Republic of Korea Yongsan-dong #ko.wikipedia 0 1 Seoul +Republic of Korea null #ko.wikipedia 1 9 null +Republic of Korea Jeonju #ko.wikipedia 1 1 null +Republic of Korea null #ko.wikipedia 2 10 null +Republic of Korea Seoul #ko.wikipedia 2 2 null +Republic of Korea null #ko.wikipedia 3 11 null +Republic of Korea null #ko.wikipedia 4 12 null +Republic of Korea Seoul #ko.wikipedia 4 3 null +Republic of Korea Seoul #ko.wikipedia 5 4 null +Republic of Korea Seoul #ko.wikipedia 6 5 null +Republic of Korea null #ko.wikipedia 8 13 null +Republic of Korea null #ko.wikipedia 11 14 null +Republic of Korea Seoul #ko.wikipedia 12 6 null +Republic of Korea null #ko.wikipedia 13 15 null +Republic of Korea null #ko.wikipedia 14 16 null +Republic of Korea Seoul #ko.wikipedia 15 7 null +Republic of Korea null #ko.wikipedia 19 17 null +Republic of Korea Seoul #ko.wikipedia 19 8 null +Republic of Korea null #ko.wikipedia 22 18 null +Republic of Korea Seoul #ko.wikipedia 22 9 null +Republic of Korea null #ko.wikipedia 23 19 null +Republic of Korea null #ko.wikipedia 24 20 null +Republic of Korea Seoul #ko.wikipedia 24 10 null +Republic of Korea null #ko.wikipedia 26 21 null +Republic of Korea null #ko.wikipedia 29 22 null +Republic of Korea null #ko.wikipedia 30 23 null +Republic of Korea null #ko.wikipedia 34 24 null +Republic of Korea Suwon-si #ko.wikipedia 40 1 null +Republic of Korea null #ko.wikipedia 41 25 null +Republic of Korea null #ko.wikipedia 42 26 null +Republic of Korea null #ko.wikipedia 51 27 null +Republic of Korea null #ko.wikipedia 52 28 null +Republic of Korea null #ko.wikipedia 56 29 null +Republic of Korea null #ko.wikipedia 59 30 null +Republic of Korea null #ko.wikipedia 62 31 null +Republic of Korea null #ko.wikipedia 65 32 null +Republic of Korea null #ko.wikipedia 73 33 null +Republic of Korea null #ko.wikipedia 159 34 null +Republic of Korea Seoul #ko.wikipedia 243 11 null +Republic of Korea Seongnam-si #ko.wikipedia 254 1 null +Republic of Korea null #ko.wikipedia 357 35 null +Republic of Korea null #ko.wikipedia 390 36 null +Republic of Korea null #ko.wikipedia 400 37 null +Republic of Korea null #ko.wikipedia 495 38 null +Republic of Korea null #ko.wikipedia 524 39 null +Republic of Korea null #ko.wikipedia 566 40 null +Republic of Korea Seoul #ko.wikipedia 663 12 null +Republic of Korea null #ko.wikipedia 757 41 null +Republic of Korea null #ko.wikipedia 827 42 null +Republic of Korea null #ko.wikipedia 832 43 null +Republic of Korea null #ko.wikipedia 1006 44 null +Republic of Korea Seoul #ko.wikipedia 1036 13 null +Republic of Korea null #ko.wikipedia 2539 45 null +Republic of Korea Suwon-si #ko.wikipedia 3234 2 null +Austria Vienna #tr.wikipedia 93 4 null diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_2.q new file mode 100644 index 00000000000..39d97a83de2 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_2.q @@ -0,0 +1,7 @@ +select +countryName, cityName, channel, added, +row_number() over (PARTITION BY countryName, cityName) as c1, +lag(cityName) over (PARTITION BY channel, added) as c2 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel, added diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_same_partition_column.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_same_partition_column.e new file mode 100644 index 00000000000..be82d150428 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_same_partition_column.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia 1 null +Austria Horsching #de.wikipedia 2 null +Austria Vienna #de.wikipedia 3 Horsching +Republic of Korea null #en.wikipedia 1 null +Austria Vienna #es.wikipedia 1 null +Republic of Korea null #ja.wikipedia 1 null +Republic of Korea null #ko.wikipedia 1 null +Republic of Korea Jeonju #ko.wikipedia 2 null +Republic of Korea Seongnam-si #ko.wikipedia 3 Jeonju +Republic of Korea Seoul #ko.wikipedia 4 Seongnam-si +Republic of Korea Suwon-si #ko.wikipedia 5 Seoul +Republic of Korea Yongsan-dong #ko.wikipedia 6 Suwon-si +Austria Vienna #tr.wikipedia 1 null diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_same_partition_column.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_same_partition_column.q new file mode 100644 index 00000000000..c325e6c6c7f --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/multiple_over_same_partition_column.q @@ -0,0 +1,7 @@ +select +countryName, cityName, channel, +row_number() over (PARTITION BY channel) as c1, +lag(cityName) over (PARTITION BY channel) as c2 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_1.e new file mode 100644 index 00000000000..e0e8a666879 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_1.e @@ -0,0 +1,13 @@ +Austria null 1 +Republic of Korea null 2 +Republic of Korea null 3 +Republic of Korea null 4 +Austria Horsching 1 +Republic of Korea Jeonju 1 +Republic of Korea Seongnam-si 1 +Republic of Korea Seoul 1 +Republic of Korea Suwon-si 1 +Austria Vienna 1 +Austria Vienna 2 +Austria Vienna 3 +Republic of Korea Yongsan-dong 1 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_1.q new file mode 100644 index 00000000000..e7450711256 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_1.q @@ -0,0 +1,6 @@ +select +countryName, cityName, +row_number() over (PARTITION BY cityName) as c1 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_2.e new file mode 100644 index 00000000000..790533db147 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_2.e @@ -0,0 +1,13 @@ +Austria null null +Republic of Korea null null +Republic of Korea null null +Republic of Korea null null +Austria Horsching null +Republic of Korea Jeonju null +Republic of Korea Seongnam-si null +Republic of Korea Seoul null +Republic of Korea Suwon-si null +Austria Vienna null +Austria Vienna Vienna +Austria Vienna Vienna +Republic of Korea Yongsan-dong null diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_2.q new file mode 100644 index 00000000000..3287be32cf5 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_2.q @@ -0,0 +1,6 @@ +select +countryName, cityName, +lag(cityName) over (PARTITION BY cityName) as c1 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_multiple_partition_columns.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_multiple_partition_columns.e new file mode 100644 index 00000000000..1d8ef5cdc53 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_multiple_partition_columns.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia null +Austria Horsching #de.wikipedia null +Austria Vienna #de.wikipedia Horsching +Austria Vienna #es.wikipedia null +Austria Vienna #tr.wikipedia null +Republic of Korea null #en.wikipedia null +Republic of Korea null #ja.wikipedia null +Republic of Korea null #ko.wikipedia null +Republic of Korea Jeonju #ko.wikipedia null +Republic of Korea Seongnam-si #ko.wikipedia Jeonju +Republic of Korea Seoul #ko.wikipedia Seongnam-si +Republic of Korea Suwon-si #ko.wikipedia Seoul +Republic of Korea Yongsan-dong #ko.wikipedia Suwon-si diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_multiple_partition_columns.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_multiple_partition_columns.q new file mode 100644 index 00000000000..8824c3e1760 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_partitioning/single_over_multiple_partition_columns.q @@ -0,0 +1,6 @@ +select +countryName, cityName, channel, +lag(cityName) over (PARTITION BY channel, countryName) as c1 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_different_sort_column.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_different_sort_column.e new file mode 100644 index 00000000000..151090f3be2 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_different_sort_column.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia 1 null +Austria Horsching #de.wikipedia 2 null +Austria Vienna #de.wikipedia 3 Horsching +Austria Vienna #es.wikipedia 5 Vienna +Austria Vienna #tr.wikipedia 13 Vienna +Republic of Korea null #en.wikipedia 4 Vienna +Republic of Korea null #ja.wikipedia 6 null +Republic of Korea null #ko.wikipedia 7 null +Republic of Korea Jeonju #ko.wikipedia 8 null +Republic of Korea Seongnam-si #ko.wikipedia 9 Jeonju +Republic of Korea Seoul #ko.wikipedia 10 Seongnam-si +Republic of Korea Suwon-si #ko.wikipedia 11 Seoul +Republic of Korea Yongsan-dong #ko.wikipedia 12 Suwon-si diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_different_sort_column.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_different_sort_column.q new file mode 100644 index 00000000000..886bc0ac11f --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_different_sort_column.q @@ -0,0 +1,7 @@ +select +countryName, cityName, channel, +row_number() over (order by channel) as c1, +lag(cityName) over (order by countryName) as c2 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_1.e new file mode 100644 index 00000000000..52148287c42 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_1.e @@ -0,0 +1,9 @@ +Austria null 1 null +Republic of Korea null 4 null +Austria Horsching 2 null +Republic of Korea Jeonju 5 Horsching +Republic of Korea Seongnam-si 6 Jeonju +Republic of Korea Seoul 7 Seongnam-si +Republic of Korea Suwon-si 8 Seoul +Austria Vienna 3 Suwon-si +Republic of Korea Yongsan-dong 9 Vienna diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_1.q new file mode 100644 index 00000000000..f0a07b6666a --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_1.q @@ -0,0 +1,7 @@ +select +countryName, cityName, +row_number() over (order by countryName, cityName) as c1, +lag(cityName) over (order by cityName, countryName) as c2 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_2.e new file mode 100644 index 00000000000..6710f17c2a9 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_2.e @@ -0,0 +1,74 @@ +Austria null #de.wikipedia 0 1 null +Austria Horsching #de.wikipedia 0 7 null +Austria Vienna #de.wikipedia 0 8 Horsching +Austria null #de.wikipedia 14 2 Vienna +Austria Vienna #de.wikipedia 72 9 null +Austria null #de.wikipedia 94 3 Vienna +Austria null #de.wikipedia 272 4 null +Austria null #de.wikipedia 4685 5 null +Austria null #de.wikipedia 6979 6 null +Republic of Korea null #en.wikipedia 0 12 null +Republic of Korea null #en.wikipedia 5 13 null +Republic of Korea null #en.wikipedia 75 14 null +Austria Vienna #es.wikipedia 0 10 null +Republic of Korea null #ja.wikipedia 0 15 Vienna +Republic of Korea null #ja.wikipedia 43 16 null +Republic of Korea null #ja.wikipedia 46 17 null +Republic of Korea null #ja.wikipedia 356 18 null +Republic of Korea null #ko.wikipedia 0 19 null +Republic of Korea Seoul #ko.wikipedia 0 59 null +Republic of Korea Yongsan-dong #ko.wikipedia 0 74 Seoul +Republic of Korea null #ko.wikipedia 1 20 Yongsan-dong +Republic of Korea Jeonju #ko.wikipedia 1 57 null +Republic of Korea null #ko.wikipedia 2 21 Jeonju +Republic of Korea Seoul #ko.wikipedia 2 60 null +Republic of Korea null #ko.wikipedia 3 22 Seoul +Republic of Korea null #ko.wikipedia 4 23 null +Republic of Korea Seoul #ko.wikipedia 4 61 null +Republic of Korea Seoul #ko.wikipedia 5 62 Seoul +Republic of Korea Seoul #ko.wikipedia 6 63 Seoul +Republic of Korea null #ko.wikipedia 8 24 Seoul +Republic of Korea null #ko.wikipedia 11 25 null +Republic of Korea Seoul #ko.wikipedia 12 64 null +Republic of Korea null #ko.wikipedia 13 26 Seoul +Republic of Korea null #ko.wikipedia 14 27 null +Republic of Korea Seoul #ko.wikipedia 15 65 null +Republic of Korea null #ko.wikipedia 19 28 Seoul +Republic of Korea Seoul #ko.wikipedia 19 66 null +Republic of Korea null #ko.wikipedia 22 29 Seoul +Republic of Korea Seoul #ko.wikipedia 22 67 null +Republic of Korea null #ko.wikipedia 23 30 Seoul +Republic of Korea null #ko.wikipedia 24 31 null +Republic of Korea Seoul #ko.wikipedia 24 68 null +Republic of Korea null #ko.wikipedia 26 32 Seoul +Republic of Korea null #ko.wikipedia 29 33 null +Republic of Korea null #ko.wikipedia 30 34 null +Republic of Korea null #ko.wikipedia 34 35 null +Republic of Korea Suwon-si #ko.wikipedia 40 72 null +Republic of Korea null #ko.wikipedia 41 36 Suwon-si +Republic of Korea null #ko.wikipedia 42 37 null +Republic of Korea null #ko.wikipedia 51 38 null +Republic of Korea null #ko.wikipedia 52 39 null +Republic of Korea null #ko.wikipedia 56 40 null +Republic of Korea null #ko.wikipedia 59 41 null +Republic of Korea null #ko.wikipedia 62 42 null +Republic of Korea null #ko.wikipedia 65 43 null +Republic of Korea null #ko.wikipedia 73 44 null +Republic of Korea null #ko.wikipedia 159 45 null +Republic of Korea Seoul #ko.wikipedia 243 69 null +Republic of Korea Seongnam-si #ko.wikipedia 254 58 Seoul +Republic of Korea null #ko.wikipedia 357 46 Seongnam-si +Republic of Korea null #ko.wikipedia 390 47 null +Republic of Korea null #ko.wikipedia 400 48 null +Republic of Korea null #ko.wikipedia 495 49 null +Republic of Korea null #ko.wikipedia 524 50 null +Republic of Korea null #ko.wikipedia 566 51 null +Republic of Korea Seoul #ko.wikipedia 663 70 null +Republic of Korea null #ko.wikipedia 757 52 Seoul +Republic of Korea null #ko.wikipedia 827 53 null +Republic of Korea null #ko.wikipedia 832 54 null +Republic of Korea null #ko.wikipedia 1006 55 null +Republic of Korea Seoul #ko.wikipedia 1036 71 null +Republic of Korea null #ko.wikipedia 2539 56 Seoul +Republic of Korea Suwon-si #ko.wikipedia 3234 73 null +Austria Vienna #tr.wikipedia 93 11 Suwon-si diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_2.q new file mode 100644 index 00000000000..e2608974f8d --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_2.q @@ -0,0 +1,7 @@ +select +countryName, cityName, channel, added, +row_number() over (order by countryName, cityName) as c1, +lag(cityName) over (order by channel, added) as c2 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel, added diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_same_sort_column.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_same_sort_column.e new file mode 100644 index 00000000000..bfe949f40f5 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_same_sort_column.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia 1 null +Austria Horsching #de.wikipedia 2 null +Austria Vienna #de.wikipedia 3 Horsching +Republic of Korea null #en.wikipedia 4 Vienna +Austria Vienna #es.wikipedia 5 null +Republic of Korea null #ja.wikipedia 6 Vienna +Republic of Korea null #ko.wikipedia 7 null +Republic of Korea Jeonju #ko.wikipedia 8 null +Republic of Korea Seongnam-si #ko.wikipedia 9 Jeonju +Republic of Korea Seoul #ko.wikipedia 10 Seongnam-si +Republic of Korea Suwon-si #ko.wikipedia 11 Seoul +Republic of Korea Yongsan-dong #ko.wikipedia 12 Suwon-si +Austria Vienna #tr.wikipedia 13 Yongsan-dong diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_same_sort_column.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_same_sort_column.q new file mode 100644 index 00000000000..723e0ea2963 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/multiple_over_same_sort_column.q @@ -0,0 +1,7 @@ +select +countryName, cityName, channel, +row_number() over (order by channel) as c1, +lag(cityName) over (order by channel) as c2 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_1.e new file mode 100644 index 00000000000..3b1046264ef --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_1.e @@ -0,0 +1,13 @@ +Austria null 1 +Republic of Korea null 2 +Republic of Korea null 3 +Republic of Korea null 4 +Austria Horsching 5 +Republic of Korea Jeonju 6 +Republic of Korea Seongnam-si 7 +Republic of Korea Seoul 8 +Republic of Korea Suwon-si 9 +Austria Vienna 10 +Austria Vienna 11 +Austria Vienna 12 +Republic of Korea Yongsan-dong 13 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_1.q new file mode 100644 index 00000000000..10b81e433b1 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_1.q @@ -0,0 +1,6 @@ +select +countryName, cityName, +row_number() over (order by cityName) as c1 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_2.e new file mode 100644 index 00000000000..244392e74b1 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_2.e @@ -0,0 +1,13 @@ +Austria null null +Republic of Korea null null +Republic of Korea null null +Republic of Korea null null +Austria Horsching null +Republic of Korea Jeonju Horsching +Republic of Korea Seongnam-si Jeonju +Republic of Korea Seoul Seongnam-si +Republic of Korea Suwon-si Seoul +Austria Vienna Suwon-si +Austria Vienna Vienna +Austria Vienna Vienna +Republic of Korea Yongsan-dong Vienna diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_2.q new file mode 100644 index 00000000000..deeb5113026 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_2.q @@ -0,0 +1,6 @@ +select +countryName, cityName, +lag(cityName) over (order by cityName) as c1 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_multiple_sort_columns.e b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_multiple_sort_columns.e new file mode 100644 index 00000000000..6ff49b69fb6 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_multiple_sort_columns.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia null +Austria Horsching #de.wikipedia null +Austria Vienna #de.wikipedia Horsching +Republic of Korea null #en.wikipedia Vienna +Austria Vienna #es.wikipedia null +Republic of Korea null #ja.wikipedia Vienna +Republic of Korea null #ko.wikipedia null +Republic of Korea Jeonju #ko.wikipedia null +Republic of Korea Seongnam-si #ko.wikipedia Jeonju +Republic of Korea Seoul #ko.wikipedia Seongnam-si +Republic of Korea Suwon-si #ko.wikipedia Seoul +Republic of Korea Yongsan-dong #ko.wikipedia Suwon-si +Austria Vienna #tr.wikipedia Yongsan-dong diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_multiple_sort_columns.q b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_multiple_sort_columns.q new file mode 100644 index 00000000000..edce01210da --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/over_clause_only_sorting/single_over_multiple_sort_columns.q @@ -0,0 +1,6 @@ +select +countryName, cityName, channel, +lag(cityName) over (order by channel, countryName) as c1 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.e new file mode 100644 index 00000000000..26c251a35fb --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia 1 +Republic of Korea null #en.wikipedia 2 +Republic of Korea null #ja.wikipedia 3 +Republic of Korea null #ko.wikipedia 4 +Republic of Korea Seoul #ko.wikipedia 1 +Austria Vienna #de.wikipedia 1 +Austria Vienna #es.wikipedia 2 +Austria Vienna #tr.wikipedia 3 +Republic of Korea Jeonju #ko.wikipedia 4 +Republic of Korea Suwon-si #ko.wikipedia 1 +Austria Horsching #de.wikipedia 1 +Republic of Korea Seongnam-si #ko.wikipedia 1 +Republic of Korea Yongsan-dong #ko.wikipedia 1 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.q new file mode 100644 index 00000000000..b10b52af389 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.q @@ -0,0 +1,6 @@ +select +countryName, cityName, channel, +row_number() over (partition by array[1,2,length(cityName)] order by countryName) as c +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.e new file mode 100644 index 00000000000..a1b116035c1 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia 1 +Austria Horsching #de.wikipedia 2 +Austria Vienna #de.wikipedia 3 +Austria Vienna #es.wikipedia 4 +Austria Vienna #tr.wikipedia 5 +Republic of Korea null #en.wikipedia 6 +Republic of Korea null #ja.wikipedia 7 +Republic of Korea null #ko.wikipedia 8 +Republic of Korea Jeonju #ko.wikipedia 9 +Republic of Korea Seongnam-si #ko.wikipedia 10 +Republic of Korea Seoul #ko.wikipedia 11 +Republic of Korea Suwon-si #ko.wikipedia 12 +Republic of Korea Yongsan-dong #ko.wikipedia 13 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.q new file mode 100644 index 00000000000..99245d7f953 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.q @@ -0,0 +1,6 @@ +select +countryName, cityName, channel, +row_number() over (partition by array[1,2,3] order by countryName) as c +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.e b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.e new file mode 100644 index 00000000000..ebd91f9f893 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia 1 +Austria Vienna #de.wikipedia 1 +Austria Vienna #es.wikipedia 2 +Austria Vienna #tr.wikipedia 3 +Austria Horsching #de.wikipedia 1 +Republic of Korea null #en.wikipedia 1 +Republic of Korea null #ja.wikipedia 2 +Republic of Korea null #ko.wikipedia 3 +Republic of Korea Seoul #ko.wikipedia 1 +Republic of Korea Jeonju #ko.wikipedia 1 +Republic of Korea Suwon-si #ko.wikipedia 1 +Republic of Korea Seongnam-si #ko.wikipedia 1 +Republic of Korea Yongsan-dong #ko.wikipedia 1 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.q b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.q new file mode 100644 index 00000000000..9241f2ee94e --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.q @@ -0,0 +1,6 @@ +select +countryName, cityName, channel, +row_number() over (partition by array[1,length(countryName),length(cityName)] order by countryName) as c +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/web-console/e2e-tests/component/datasources/overview.ts b/web-console/e2e-tests/component/datasources/overview.ts index 50a5762c15f..f3f7c1a434d 100644 --- a/web-console/e2e-tests/component/datasources/overview.ts +++ b/web-console/e2e-tests/component/datasources/overview.ts @@ -125,13 +125,13 @@ export class DatasourcesOverview { throw new Error(`Could not find datasource: ${datasourceName}`); } - const editActions = await this.page.$$('.action-cell span[icon=more]'); + const editActions = await this.page.$$('.action-cell span.bp5-icon-more'); await editActions[index].click(); await this.waitForPopupMenu(); } private async waitForPopupMenu(): Promise { - await this.page.waitForSelector('ul.bp4-menu'); + await this.page.waitForSelector('ul.bp5-menu'); } async triggerCompaction(): Promise { diff --git a/web-console/e2e-tests/component/load-data/data-loader.ts b/web-console/e2e-tests/component/load-data/data-loader.ts index 4462641cb9a..102ef2d75bf 100644 --- a/web-console/e2e-tests/component/load-data/data-loader.ts +++ b/web-console/e2e-tests/component/load-data/data-loader.ts @@ -66,7 +66,7 @@ export class DataLoader { } private async start() { - const cardSelector = `//*[contains(@class,"bp4-card")][p[contains(text(),"${this.connector.name}")]]`; + const cardSelector = `//*[contains(@class,"bp5-card")][p[contains(text(),"${this.connector.name}")]]`; await this.page.click(cardSelector); await clickButton(this.page, 'Connect data'); } @@ -126,7 +126,7 @@ export class DataLoader { const rollupChecked = await rollupInput!.evaluate(el => (el as HTMLInputElement).checked); if (rollupChecked !== configureSchemaConfig.rollup) { await this.page.click(rollupSelector); - const confirmationDialogSelector = '//*[contains(@class,"bp4-alert-body")]'; + const confirmationDialogSelector = '//*[contains(@class,"bp5-alert-body")]'; await this.page.waitForSelector(confirmationDialogSelector); await clickButton(this.page, 'Yes'); const statusMessageSelector = '.recipe-toaster'; diff --git a/web-console/package-lock.json b/web-console/package-lock.json index 2945955d9d6..412f728d56d 100644 --- a/web-console/package-lock.json +++ b/web-console/package-lock.json @@ -9,15 +9,15 @@ "version": "31.0.0", "license": "Apache-2.0", "dependencies": { - "@blueprintjs/core": "^4.20.1", - "@blueprintjs/datetime": "^4.4.36", - "@blueprintjs/datetime2": "^0.9.35", - "@blueprintjs/icons": "^4.16.0", - "@blueprintjs/popover2": "^1.14.9", - "@blueprintjs/select": "^4.9.24", + "@blueprintjs/core": "^5.10.5", + "@blueprintjs/datetime": "^5.3.6", + "@blueprintjs/datetime2": "^2.3.7", + "@blueprintjs/icons": "^5.10.0", + "@blueprintjs/select": "^5.2.1", "@druid-toolkit/query": "^0.22.20", "@druid-toolkit/visuals-core": "^0.3.3", "@druid-toolkit/visuals-react": "^0.3.3", + "@fontsource/open-sans": "^5.0.28", "ace-builds": "~1.4.14", "axios": "^1.6.7", "chronoshift": "^0.10.0", @@ -29,9 +29,9 @@ "d3-dsv": "^2.0.0", "d3-scale": "^3.3.0", "d3-selection": "^2.0.0", + "date-fns": "^2.28.0", "echarts": "^5.4.3", "file-saver": "^2.0.5", - "fontsource-open-sans": "^3.0.9", "hjson": "^3.2.2", "json-bigint-native": "^1.2.0", "lodash.debounce": "^4.0.8", @@ -816,36 +816,35 @@ "dev": true }, "node_modules/@blueprintjs/colors": { - "version": "4.2.1", - "resolved": "https://registry.npmjs.org/@blueprintjs/colors/-/colors-4.2.1.tgz", - "integrity": "sha512-Cx7J2YnUuxn+fi+y5XtXnBB7+cFHN4xBrRkaAetp78i3VTCXjUk+d1omrOr8TqbRucUXTdrhbZOUHpzRLFcJpQ==", + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/@blueprintjs/colors/-/colors-5.1.1.tgz", + "integrity": "sha512-Mni/GgCYLaMf5U5zsCN42skOG49w3U0QmUFuJgFZ/1pv+3cHF/9xR4L4dXcj5DtgJoHkmUbK36PR5mdFB65WEA==", "dependencies": { - "tslib": "~2.5.0" + "tslib": "~2.6.2" } }, "node_modules/@blueprintjs/core": { - "version": "4.20.2", - "resolved": "https://registry.npmjs.org/@blueprintjs/core/-/core-4.20.2.tgz", - "integrity": "sha512-5v4Nr0jozfAjiOkjY4zvt1XSpt4ldnrSaxtwo506S2cxJYfwFeMTmDshXNPFcc8L1fjZMxi0IWI2WABXzZXS6w==", + "version": "5.10.5", + "resolved": "https://registry.npmjs.org/@blueprintjs/core/-/core-5.10.5.tgz", + "integrity": "sha512-mvTB+Ox31nt9830hEgVOi/A53QI71hpOzV/7i3ZgzzrYAaEGgFMcwa7X0kn9PSOGfxw87nWL9VpYSirnzpC6lw==", "dependencies": { - "@blueprintjs/colors": "^4.2.1", - "@blueprintjs/icons": "^4.16.0", - "@juggle/resize-observer": "^3.4.0", - "@types/dom4": "^2.0.2", + "@blueprintjs/colors": "^5.1.1", + "@blueprintjs/icons": "^5.10.0", + "@popperjs/core": "^2.11.8", "classnames": "^2.3.1", - "dom4": "^2.1.5", "normalize.css": "^8.0.1", - "popper.js": "^1.16.1", - "react-popper": "^1.3.11", + "react-popper": "^2.3.0", "react-transition-group": "^4.4.5", - "tslib": "~2.5.0" + "react-uid": "^2.3.3", + "tslib": "~2.6.2", + "use-sync-external-store": "^1.2.0" }, "bin": { "upgrade-blueprint-2.0.0-rename": "scripts/upgrade-blueprint-2.0.0-rename.sh", "upgrade-blueprint-3.0.0-rename": "scripts/upgrade-blueprint-3.0.0-rename.sh" }, "peerDependencies": { - "@types/react": "^16.14.32 || 17 || 18", + "@types/react": "^16.14.41 || 17 || 18", "react": "^16.8 || 17 || 18", "react-dom": "^16.8 || 17 || 18" }, @@ -855,35 +854,22 @@ } } }, - "node_modules/@blueprintjs/core/node_modules/react-popper": { - "version": "1.3.11", - "resolved": "https://registry.npmjs.org/react-popper/-/react-popper-1.3.11.tgz", - "integrity": "sha512-VSA/bS+pSndSF2fiasHK/PTEEAyOpX60+H5EPAjoArr8JGm+oihu4UbrqcEBpQibJxBVCpYyjAX7abJ+7DoYVg==", - "dependencies": { - "@babel/runtime": "^7.1.2", - "@hypnosphi/create-react-context": "^0.3.1", - "deep-equal": "^1.1.1", - "popper.js": "^1.14.4", - "prop-types": "^15.6.1", - "typed-styles": "^0.0.7", - "warning": "^4.0.2" - }, - "peerDependencies": { - "react": "0.14.x || ^15.0.0 || ^16.0.0 || ^17.0.0" - } - }, "node_modules/@blueprintjs/datetime": { - "version": "4.4.36", - "resolved": "https://registry.npmjs.org/@blueprintjs/datetime/-/datetime-4.4.36.tgz", - "integrity": "sha512-+0zMGorGNjR/5lEB2t7sjVX0jWPVgmFkHTIezhAIQgJAj0upcxF3SuvnrpVjJKA3Ug6/0wZ+76eLPbpNz9CVzA==", + "version": "5.3.7", + "resolved": "https://registry.npmjs.org/@blueprintjs/datetime/-/datetime-5.3.7.tgz", + "integrity": "sha512-6rDMqSU8jeiAlMxS30MYAQc4X1sfMGWwFjfs3IoKxJfcfieN+ytus5G+if6zjsSKUjdhL9+qBTgKRo3VJJEQ3g==", "dependencies": { - "@blueprintjs/core": "^4.20.1", + "@blueprintjs/core": "^5.10.5", + "@blueprintjs/icons": "^5.10.0", + "@blueprintjs/select": "^5.2.1", "classnames": "^2.3.1", + "date-fns": "^2.28.0", + "date-fns-tz": "^2.0.0", "react-day-picker": "7.4.9", - "tslib": "~2.5.0" + "tslib": "~2.6.2" }, "peerDependencies": { - "@types/react": "^16.14.32 || 17", + "@types/react": "^16.14.41 || 17", "react": "^16.8 || 17", "react-dom": "^16.8 || 17" }, @@ -894,24 +880,23 @@ } }, "node_modules/@blueprintjs/datetime2": { - "version": "0.9.35", - "resolved": "https://registry.npmjs.org/@blueprintjs/datetime2/-/datetime2-0.9.35.tgz", - "integrity": "sha512-9vuhKtVj8GwtB3fx4xnzQWc1g4kH6NB3QqotDI1azJrzTOVVEqzYs+RVGwLw60/WAR2PD+L/WJkthRIS6/wKAw==", + "version": "2.3.7", + "resolved": "https://registry.npmjs.org/@blueprintjs/datetime2/-/datetime2-2.3.7.tgz", + "integrity": "sha512-Q6IBo5hcvoDtH9FsFaysZhadXNS6iyhuyaRY+rJD3y3qLaUt6wiYDhw5LN6L4wuasbPO1g0lVDwY6bkGuIG4jg==", "dependencies": { - "@blueprintjs/core": "^4.20.1", - "@blueprintjs/datetime": "^4.4.36", - "@blueprintjs/popover2": "^1.14.9", - "@blueprintjs/select": "^4.9.22", + "@blueprintjs/core": "^5.10.5", + "@blueprintjs/datetime": "^5.3.7", + "@blueprintjs/icons": "^5.10.0", "classnames": "^2.3.1", "date-fns": "^2.28.0", - "date-fns-tz": "^1.3.7", - "lodash": "^4.17.21", - "tslib": "~2.5.0" + "react-day-picker": "^8.10.0", + "react-innertext": "^1.1.5", + "tslib": "~2.6.2" }, "peerDependencies": { - "@types/react": "^16.14.32 || 17", - "react": "^16.8 || 17", - "react-dom": "^16.8 || 17" + "@types/react": "^16.14.41 || 17 || 18", + "react": "^16.8 || 17 || 18", + "react-dom": "^16.8 || 17 || 18" }, "peerDependenciesMeta": { "@types/react": { @@ -919,31 +904,30 @@ } } }, + "node_modules/@blueprintjs/datetime2/node_modules/react-day-picker": { + "version": "8.10.1", + "resolved": "https://registry.npmjs.org/react-day-picker/-/react-day-picker-8.10.1.tgz", + "integrity": "sha512-TMx7fNbhLk15eqcMt+7Z7S2KF7mfTId/XJDjKE8f+IUcFn0l08/kI4FiYTL/0yuOLmEcbR4Fwe3GJf/NiiMnPA==", + "funding": { + "type": "individual", + "url": "https://github.com/sponsors/gpbl" + }, + "peerDependencies": { + "date-fns": "^2.28.0 || ^3.0.0", + "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + } + }, "node_modules/@blueprintjs/icons": { - "version": "4.16.0", - "resolved": "https://registry.npmjs.org/@blueprintjs/icons/-/icons-4.16.0.tgz", - "integrity": "sha512-cyfgjUZcZCtQrXWUV8FwqYTFEzduV4a0N7yhOU38jY+cBRCLu/sDrD0Osvfk4DGRvNe4YjY7pohVLFSxpg68Uw==", + "version": "5.10.0", + "resolved": "https://registry.npmjs.org/@blueprintjs/icons/-/icons-5.10.0.tgz", + "integrity": "sha512-Qp9H2U2/m6+p0Lh9ACuF2O6o0uVINMFimo+EgasgGaLx8SkJsl7ApB3hYQ+iRSYlQWotnoqd4Vtzj4c7sKukHQ==", "dependencies": { "change-case": "^4.1.2", "classnames": "^2.3.1", - "tslib": "~2.5.0" - } - }, - "node_modules/@blueprintjs/popover2": { - "version": "1.14.11", - "resolved": "https://registry.npmjs.org/@blueprintjs/popover2/-/popover2-1.14.11.tgz", - "integrity": "sha512-5XAjeb2mlWjYXC0pqrNDLzHSsX85Zaiv8jixxUN9abarMUUFKGATgGF8MRsWTLAW94Gli6CB1lzVkrYkRHHf6Q==", - "dependencies": { - "@blueprintjs/core": "^4.20.2", - "@juggle/resize-observer": "^3.4.0", - "@popperjs/core": "^2.11.7", - "classnames": "^2.3.1", - "dom4": "^2.1.5", - "react-popper": "^2.3.0", - "tslib": "~2.5.0" + "tslib": "~2.6.2" }, "peerDependencies": { - "@types/react": "^16.14.32 || 17 || 18", + "@types/react": "^16.14.41 || 17 || 18", "react": "^16.8 || 17 || 18", "react-dom": "^16.8 || 17 || 18" }, @@ -954,17 +938,17 @@ } }, "node_modules/@blueprintjs/select": { - "version": "4.9.24", - "resolved": "https://registry.npmjs.org/@blueprintjs/select/-/select-4.9.24.tgz", - "integrity": "sha512-OTjesxH/7UZvM7yAdHJ5u3sIjX1N8Rs4CQQ22AfqNl82SIROqkuXI31XEl6YNX1GsYfmAMiw0K7XohEKOMXR5g==", + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/@blueprintjs/select/-/select-5.2.1.tgz", + "integrity": "sha512-rXM9ik2mLd394g+sxwbHv2bGfrLbcdY9gFftqmJmm3vu7XJ2kRrsC1oOlNSC4dNBpUdHnyAzBTHEu81B1LkRbQ==", "dependencies": { - "@blueprintjs/core": "^4.20.2", - "@blueprintjs/popover2": "^1.14.11", + "@blueprintjs/core": "^5.10.5", + "@blueprintjs/icons": "^5.10.0", "classnames": "^2.3.1", - "tslib": "~2.5.0" + "tslib": "~2.6.2" }, "peerDependencies": { - "@types/react": "^16.14.32 || 17 || 18", + "@types/react": "^16.14.41 || 17 || 18", "react": "^16.8 || 17 || 18", "react-dom": "^16.8 || 17 || 18" }, @@ -1235,6 +1219,11 @@ "node": "^12.22.0 || ^14.17.0 || >=16.0.0" } }, + "node_modules/@fontsource/open-sans": { + "version": "5.0.28", + "resolved": "https://registry.npmjs.org/@fontsource/open-sans/-/open-sans-5.0.28.tgz", + "integrity": "sha512-hBvJHY76pJT/JynGUB5EXWhnzjYfLdcMn655J5p1v9lTT9HdQSy+keq2KPVXO2Htlg998BBa3p6u/jlrZ6w0kg==" + }, "node_modules/@humanwhocodes/config-array": { "version": "0.11.8", "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.11.8.tgz", @@ -1291,19 +1280,6 @@ "integrity": "sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA==", "dev": true }, - "node_modules/@hypnosphi/create-react-context": { - "version": "0.3.1", - "resolved": "https://registry.npmjs.org/@hypnosphi/create-react-context/-/create-react-context-0.3.1.tgz", - "integrity": "sha512-V1klUed202XahrWJLLOT3EXNeCpFHCcJntdFGI15ntCwau+jfT386w7OFTMaCqOgXUH1fa0w/I1oZs+i/Rfr0A==", - "dependencies": { - "gud": "^1.0.0", - "warning": "^4.0.3" - }, - "peerDependencies": { - "prop-types": "^15.0.0", - "react": ">=0.14.0" - } - }, "node_modules/@istanbuljs/load-nyc-config": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz", @@ -2539,11 +2515,6 @@ "integrity": "sha512-9MVYlmIgmRR31C5b4FVSWtuMmBHh2mOWQYfl7XAYOa8dsnb7iEmUmRSWSFgXFtkjxO65d7hTUHQC+RhR/9IWFg==", "dev": true }, - "node_modules/@types/dom4": { - "version": "2.0.2", - "resolved": "https://registry.npmjs.org/@types/dom4/-/dom4-2.0.2.tgz", - "integrity": "sha512-Rt4IC1T7xkCWa0OG1oSsPa0iqnxlDeQqKXZAHrQGLb7wFGncWm85MaxKUjAGejOrUynOgWlFi4c6S6IyJwoK4g==" - }, "node_modules/@types/enzyme": { "version": "3.10.17", "resolved": "https://registry.npmjs.org/@types/enzyme/-/enzyme-3.10.17.tgz", @@ -4448,6 +4419,7 @@ "version": "1.0.2", "resolved": "https://registry.npmjs.org/call-bind/-/call-bind-1.0.2.tgz", "integrity": "sha512-7O+FbCihrB5WGbFYesctwmTKae6rOiIzmz1icreWJ+0aA7LJfuqhEso2T9ncpcFtzMQtzXf2QGGueWJGTYsqrA==", + "dev": true, "dependencies": { "function-bind": "^1.1.1", "get-intrinsic": "^1.0.2" @@ -5521,11 +5493,11 @@ } }, "node_modules/date-fns-tz": { - "version": "1.3.8", - "resolved": "https://registry.npmjs.org/date-fns-tz/-/date-fns-tz-1.3.8.tgz", - "integrity": "sha512-qwNXUFtMHTTU6CFSFjoJ80W8Fzzp24LntbjFFBgL/faqds4e5mo9mftoRLgr3Vi1trISsg4awSpYVsOQCRnapQ==", + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/date-fns-tz/-/date-fns-tz-2.0.1.tgz", + "integrity": "sha512-fJCG3Pwx8HUoLhkepdsP7Z5RsucUi+ZBOxyM5d0ZZ6c4SdYustq0VMmOu6Wf7bli+yS/Jwp91TOCqn9jMcVrUA==", "peerDependencies": { - "date-fns": ">=2.0.0" + "date-fns": "2.x" } }, "node_modules/debounce": { @@ -5594,22 +5566,6 @@ } } }, - "node_modules/deep-equal": { - "version": "1.1.1", - "resolved": "https://registry.npmjs.org/deep-equal/-/deep-equal-1.1.1.tgz", - "integrity": "sha512-yd9c5AdiqVcR+JjcwUQb9DkhJc8ngNr0MahEBGvDiJw8puWab2yZlh+nkasOnZP+EGTAP6rRp2JzJhJZzvNF8g==", - "dependencies": { - "is-arguments": "^1.0.4", - "is-date-object": "^1.0.1", - "is-regex": "^1.0.4", - "object-is": "^1.0.1", - "object-keys": "^1.1.1", - "regexp.prototype.flags": "^1.2.0" - }, - "funding": { - "url": "https://github.com/sponsors/ljharb" - } - }, "node_modules/deep-is": { "version": "0.1.4", "resolved": "https://registry.npmjs.org/deep-is/-/deep-is-0.1.4.tgz", @@ -5650,6 +5606,7 @@ "version": "1.2.0", "resolved": "https://registry.npmjs.org/define-properties/-/define-properties-1.2.0.tgz", "integrity": "sha512-xvqAVKGfT1+UAvPwKTVw/njhdQ8ZhXK4lI0bCIuCMrp2up9nPnaDftrLtmpTazqd1o+UY4zgzU+avtMbDP+ldA==", + "dev": true, "dependencies": { "has-property-descriptors": "^1.0.0", "object-keys": "^1.1.1" @@ -5802,11 +5759,6 @@ "entities": "^1.1.1" } }, - "node_modules/dom4": { - "version": "2.1.6", - "resolved": "https://registry.npmjs.org/dom4/-/dom4-2.1.6.tgz", - "integrity": "sha512-JkCVGnN4ofKGbjf5Uvc8mmxaATIErKQKSgACdBXpsQ3fY6DlIpAyWfiBSrGkttATssbDCp3psiAKWXk5gmjycA==" - }, "node_modules/domelementtype": { "version": "1.3.1", "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-1.3.1.tgz", @@ -7508,12 +7460,6 @@ } } }, - "node_modules/fontsource-open-sans": { - "version": "3.0.9", - "resolved": "https://registry.npmjs.org/fontsource-open-sans/-/fontsource-open-sans-3.0.9.tgz", - "integrity": "sha512-NEgZAgtpvz7iYZ/IlojOOy6hRLaLmxnxzEKYXnq0KU/X8lb9cD7d0XImykhfksK1/JZ6xyKQKjYq4PycHQZK9w==", - "deprecated": "Package relocated. Please install and migrate to @fontsource/open-sans." - }, "node_modules/for-each": { "version": "0.3.3", "resolved": "https://registry.npmjs.org/for-each/-/for-each-0.3.3.tgz", @@ -7634,6 +7580,7 @@ "version": "1.2.3", "resolved": "https://registry.npmjs.org/functions-have-names/-/functions-have-names-1.2.3.tgz", "integrity": "sha512-xckBUXyTIqT97tq2x2AMb+g163b5JFysYk0x4qxNFwbfQkmNZoiRHb6sPzI9/QV33WeuvVYBUIiD4NzNIyqaRQ==", + "dev": true, "funding": { "url": "https://github.com/sponsors/ljharb" } @@ -7660,6 +7607,7 @@ "version": "1.2.0", "resolved": "https://registry.npmjs.org/get-intrinsic/-/get-intrinsic-1.2.0.tgz", "integrity": "sha512-L049y6nFOuom5wGyRc3/gdTLO94dySVKRACj1RmJZBQXlbTMhtNIgkWkUHq+jYmZvKf14EW1EoJnnjbmoHij0Q==", + "dev": true, "dependencies": { "function-bind": "^1.1.1", "has": "^1.0.3", @@ -7836,11 +7784,6 @@ "integrity": "sha512-bzh50DW9kTPM00T8y4o8vQg89Di9oLJVLW/KaOGIXJWP/iqCN6WKYkbNOF04vFLJhwcpYUh9ydh/+5vpOqV4YQ==", "dev": true }, - "node_modules/gud": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/gud/-/gud-1.0.0.tgz", - "integrity": "sha512-zGEOVKFM5sVPPrYs7J5/hYEw2Pof8KCyOwyhG8sAF26mCAeUFAcYPu1mwB7hhpIP29zOIBaDqwuHdLp0jvZXjw==" - }, "node_modules/gzip-size": { "version": "6.0.0", "resolved": "https://registry.npmjs.org/gzip-size/-/gzip-size-6.0.0.tgz", @@ -7917,6 +7860,7 @@ "version": "1.0.0", "resolved": "https://registry.npmjs.org/has-property-descriptors/-/has-property-descriptors-1.0.0.tgz", "integrity": "sha512-62DVLZGoiEBDHQyqG4w9xCuZ7eJEwNmJRWw2VY84Oedb7WFcA27fiEVe8oUQx9hAUJ4ekurquucTGwsyO1XGdQ==", + "dev": true, "dependencies": { "get-intrinsic": "^1.1.1" }, @@ -7940,6 +7884,7 @@ "version": "1.0.3", "resolved": "https://registry.npmjs.org/has-symbols/-/has-symbols-1.0.3.tgz", "integrity": "sha512-l3LCuF6MgDNwTDKkdYGEihYjt5pRPbEg46rtlmnSPlUbgmB8LOIrKJbYYFBSbnPaJexMKtiPO8hmeRjRz2Td+A==", + "dev": true, "engines": { "node": ">= 0.4" }, @@ -7951,6 +7896,7 @@ "version": "1.0.0", "resolved": "https://registry.npmjs.org/has-tostringtag/-/has-tostringtag-1.0.0.tgz", "integrity": "sha512-kFjcSNhnlGV1kyoGk7OXKSawH5JOb/LzUc5w9B02hOTO0dfFRjbHQKvg1d6cf3HbeUmtU9VbbV3qzZ2Teh97WQ==", + "dev": true, "dependencies": { "has-symbols": "^1.0.2" }, @@ -8504,6 +8450,7 @@ "version": "1.1.1", "resolved": "https://registry.npmjs.org/is-arguments/-/is-arguments-1.1.1.tgz", "integrity": "sha512-8Q7EARjzEnKpt/PCD7e1cgUS0a6X8u5tdSiMqXhojOdoV9TsMsiO+9VLC5vAmO8N7/GmXn7yjR8qnA6bVAEzfA==", + "dev": true, "dependencies": { "call-bind": "^1.0.2", "has-tostringtag": "^1.0.0" @@ -8636,6 +8583,7 @@ "version": "1.0.5", "resolved": "https://registry.npmjs.org/is-date-object/-/is-date-object-1.0.5.tgz", "integrity": "sha512-9YQaSxsAiSwcvS33MBk3wTCVnWK+HhF8VZR2jRxehM16QcVOdHqPn4VPHmRK4lSr38n9JriurInLcP90xsYNfQ==", + "dev": true, "dependencies": { "has-tostringtag": "^1.0.0" }, @@ -8787,6 +8735,7 @@ "version": "1.1.4", "resolved": "https://registry.npmjs.org/is-regex/-/is-regex-1.1.4.tgz", "integrity": "sha512-kvRdxDsxZjhzUX07ZnLydzS1TU/TJlTUHHY4YLL87e37oUA49DfkLqgy+VjFocowy29cKvcSiu+kIv728jTTVg==", + "dev": true, "dependencies": { "call-bind": "^1.0.2", "has-tostringtag": "^1.0.0" @@ -11512,7 +11461,8 @@ "node_modules/lodash": { "version": "4.17.21", "resolved": "https://registry.npmjs.org/lodash/-/lodash-4.17.21.tgz", - "integrity": "sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==" + "integrity": "sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==", + "dev": true }, "node_modules/lodash.debounce": { "version": "4.0.8", @@ -12329,6 +12279,7 @@ "version": "1.1.5", "resolved": "https://registry.npmjs.org/object-is/-/object-is-1.1.5.tgz", "integrity": "sha512-3cyDsyHgtmi7I7DfSSI2LDp6SK2lwvtbg0p0R1e0RvTqF5ceGx+K2dfSjm1bKDMVCFEDAQvy+o8c6a7VujOddw==", + "dev": true, "dependencies": { "call-bind": "^1.0.2", "define-properties": "^1.1.3" @@ -12344,6 +12295,7 @@ "version": "1.1.1", "resolved": "https://registry.npmjs.org/object-keys/-/object-keys-1.1.1.tgz", "integrity": "sha512-NuAESUOUMrlIXOfHKzD6bpPu3tYt3xvjNdRIQ+FeT0lNb4K8WR70CaDxhuNguS2XG+GjkyMwOzsN5ZktImfhLA==", + "dev": true, "engines": { "node": ">= 0.4" } @@ -12812,16 +12764,6 @@ "node": ">=4" } }, - "node_modules/popper.js": { - "version": "1.16.1", - "resolved": "https://registry.npmjs.org/popper.js/-/popper.js-1.16.1.tgz", - "integrity": "sha512-Wb4p1J4zyFTbM+u6WuO4XstYx4Ky9Cewe4DWrel7B0w6VVICvPwdOpotjzcf6eD8TsckVnIMNONQyPIUFOUbCQ==", - "deprecated": "You can find the new Popper v2 at @popperjs/core, this package is dedicated to the legacy v1", - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/popperjs" - } - }, "node_modules/postcss": { "version": "8.4.31", "resolved": "https://registry.npmjs.org/postcss/-/postcss-8.4.31.tgz", @@ -14745,6 +14687,15 @@ "resolved": "https://registry.npmjs.org/react-fast-compare/-/react-fast-compare-3.2.2.tgz", "integrity": "sha512-nsO+KSNgo1SbJqJEYRE9ERzo7YtYbou/OqjSQKxV7jcKox7+usiUVZOAC+XnDOABXggQTno0Y1CpVnuWEc1boQ==" }, + "node_modules/react-innertext": { + "version": "1.1.5", + "resolved": "https://registry.npmjs.org/react-innertext/-/react-innertext-1.1.5.tgz", + "integrity": "sha512-PWAqdqhxhHIv80dT9znP2KvS+hfkbRovFp4zFYHFFlOoQLRiawIic81gKb3U1wEyJZgMwgs3JoLtwryASRWP3Q==", + "peerDependencies": { + "@types/react": ">=0.0.0 <=99", + "react": ">=0.0.0 <=99" + } + }, "node_modules/react-is": { "version": "16.8.6", "resolved": "https://registry.npmjs.org/react-is/-/react-is-16.8.6.tgz", @@ -14856,6 +14807,26 @@ "react-dom": ">=16.6.0" } }, + "node_modules/react-uid": { + "version": "2.3.3", + "resolved": "https://registry.npmjs.org/react-uid/-/react-uid-2.3.3.tgz", + "integrity": "sha512-iNpDovcb9qBpBTo8iUgqRSQOS8GV3bWoNaTaUptHkXtAooXSo0OWe7vN6TqqB8x3x0bNBbQx96kkmSltQ5h9kQ==", + "dependencies": { + "tslib": "^2.0.0" + }, + "engines": { + "node": ">=10" + }, + "peerDependencies": { + "@types/react": "^16.8.0 || ^17.0.0 || ^18.0.0", + "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + }, + "peerDependenciesMeta": { + "@types/react": { + "optional": true + } + } + }, "node_modules/read-installed": { "version": "4.0.3", "resolved": "https://registry.npmjs.org/read-installed/-/read-installed-4.0.3.tgz", @@ -15028,6 +14999,7 @@ "version": "1.5.0", "resolved": "https://registry.npmjs.org/regexp.prototype.flags/-/regexp.prototype.flags-1.5.0.tgz", "integrity": "sha512-0SutC3pNudRKgquxGoRGIz946MZVHqbNfPjBdxeOhBrdgDKlRoXmYLQN9xRbrR09ZXWeGAdPuif7egofn6v5LA==", + "dev": true, "dependencies": { "call-bind": "^1.0.2", "define-properties": "^1.2.0", @@ -17228,9 +17200,9 @@ } }, "node_modules/tslib": { - "version": "2.5.3", - "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.5.3.tgz", - "integrity": "sha512-mSxlJJwl3BMEQCUNnxXBU9jP4JBktcEGhURcPR6VQVlnP0FdDEsIaz0C35dXNGLyRfrATNofF0F5p2KPxQgB+w==" + "version": "2.6.3", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.6.3.tgz", + "integrity": "sha512-xNvxJEOUiWPGhUuUdQgAJPKOOJfGnIyKySOc09XkKsgdUV/3E2zvwZYdejjmRgPCgcym1juLH3226yA7sEFJKQ==" }, "node_modules/tsutils": { "version": "3.21.0", @@ -17313,11 +17285,6 @@ "url": "https://github.com/sponsors/ljharb" } }, - "node_modules/typed-styles": { - "version": "0.0.7", - "resolved": "https://registry.npmjs.org/typed-styles/-/typed-styles-0.0.7.tgz", - "integrity": "sha512-pzP0PWoZUhsECYjABgCGQlRGL1n7tOHsgwYv3oIiEpJwGhFTuty/YNeduxQYzXXa3Ge5BdT6sHYIQYpl4uJ+5Q==" - }, "node_modules/typedarray-to-buffer": { "version": "3.1.5", "resolved": "https://registry.npmjs.org/typedarray-to-buffer/-/typedarray-to-buffer-3.1.5.tgz", @@ -19022,107 +18989,86 @@ "dev": true }, "@blueprintjs/colors": { - "version": "4.2.1", - "resolved": "https://registry.npmjs.org/@blueprintjs/colors/-/colors-4.2.1.tgz", - "integrity": "sha512-Cx7J2YnUuxn+fi+y5XtXnBB7+cFHN4xBrRkaAetp78i3VTCXjUk+d1omrOr8TqbRucUXTdrhbZOUHpzRLFcJpQ==", + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/@blueprintjs/colors/-/colors-5.1.1.tgz", + "integrity": "sha512-Mni/GgCYLaMf5U5zsCN42skOG49w3U0QmUFuJgFZ/1pv+3cHF/9xR4L4dXcj5DtgJoHkmUbK36PR5mdFB65WEA==", "requires": { - "tslib": "~2.5.0" + "tslib": "~2.6.2" } }, "@blueprintjs/core": { - "version": "4.20.2", - "resolved": "https://registry.npmjs.org/@blueprintjs/core/-/core-4.20.2.tgz", - "integrity": "sha512-5v4Nr0jozfAjiOkjY4zvt1XSpt4ldnrSaxtwo506S2cxJYfwFeMTmDshXNPFcc8L1fjZMxi0IWI2WABXzZXS6w==", + "version": "5.10.5", + "resolved": "https://registry.npmjs.org/@blueprintjs/core/-/core-5.10.5.tgz", + "integrity": "sha512-mvTB+Ox31nt9830hEgVOi/A53QI71hpOzV/7i3ZgzzrYAaEGgFMcwa7X0kn9PSOGfxw87nWL9VpYSirnzpC6lw==", "requires": { - "@blueprintjs/colors": "^4.2.1", - "@blueprintjs/icons": "^4.16.0", - "@juggle/resize-observer": "^3.4.0", - "@types/dom4": "^2.0.2", + "@blueprintjs/colors": "^5.1.1", + "@blueprintjs/icons": "^5.10.0", + "@popperjs/core": "^2.11.8", "classnames": "^2.3.1", - "dom4": "^2.1.5", "normalize.css": "^8.0.1", - "popper.js": "^1.16.1", - "react-popper": "^1.3.11", + "react-popper": "^2.3.0", "react-transition-group": "^4.4.5", - "tslib": "~2.5.0" - }, - "dependencies": { - "react-popper": { - "version": "1.3.11", - "resolved": "https://registry.npmjs.org/react-popper/-/react-popper-1.3.11.tgz", - "integrity": "sha512-VSA/bS+pSndSF2fiasHK/PTEEAyOpX60+H5EPAjoArr8JGm+oihu4UbrqcEBpQibJxBVCpYyjAX7abJ+7DoYVg==", - "requires": { - "@babel/runtime": "^7.1.2", - "@hypnosphi/create-react-context": "^0.3.1", - "deep-equal": "^1.1.1", - "popper.js": "^1.14.4", - "prop-types": "^15.6.1", - "typed-styles": "^0.0.7", - "warning": "^4.0.2" - } - } + "react-uid": "^2.3.3", + "tslib": "~2.6.2", + "use-sync-external-store": "^1.2.0" } }, "@blueprintjs/datetime": { - "version": "4.4.36", - "resolved": "https://registry.npmjs.org/@blueprintjs/datetime/-/datetime-4.4.36.tgz", - "integrity": "sha512-+0zMGorGNjR/5lEB2t7sjVX0jWPVgmFkHTIezhAIQgJAj0upcxF3SuvnrpVjJKA3Ug6/0wZ+76eLPbpNz9CVzA==", + "version": "5.3.7", + "resolved": "https://registry.npmjs.org/@blueprintjs/datetime/-/datetime-5.3.7.tgz", + "integrity": "sha512-6rDMqSU8jeiAlMxS30MYAQc4X1sfMGWwFjfs3IoKxJfcfieN+ytus5G+if6zjsSKUjdhL9+qBTgKRo3VJJEQ3g==", "requires": { - "@blueprintjs/core": "^4.20.1", + "@blueprintjs/core": "^5.10.5", + "@blueprintjs/icons": "^5.10.0", + "@blueprintjs/select": "^5.2.1", "classnames": "^2.3.1", + "date-fns": "^2.28.0", + "date-fns-tz": "^2.0.0", "react-day-picker": "7.4.9", - "tslib": "~2.5.0" + "tslib": "~2.6.2" } }, "@blueprintjs/datetime2": { - "version": "0.9.35", - "resolved": "https://registry.npmjs.org/@blueprintjs/datetime2/-/datetime2-0.9.35.tgz", - "integrity": "sha512-9vuhKtVj8GwtB3fx4xnzQWc1g4kH6NB3QqotDI1azJrzTOVVEqzYs+RVGwLw60/WAR2PD+L/WJkthRIS6/wKAw==", + "version": "2.3.7", + "resolved": "https://registry.npmjs.org/@blueprintjs/datetime2/-/datetime2-2.3.7.tgz", + "integrity": "sha512-Q6IBo5hcvoDtH9FsFaysZhadXNS6iyhuyaRY+rJD3y3qLaUt6wiYDhw5LN6L4wuasbPO1g0lVDwY6bkGuIG4jg==", "requires": { - "@blueprintjs/core": "^4.20.1", - "@blueprintjs/datetime": "^4.4.36", - "@blueprintjs/popover2": "^1.14.9", - "@blueprintjs/select": "^4.9.22", + "@blueprintjs/core": "^5.10.5", + "@blueprintjs/datetime": "^5.3.7", + "@blueprintjs/icons": "^5.10.0", "classnames": "^2.3.1", "date-fns": "^2.28.0", - "date-fns-tz": "^1.3.7", - "lodash": "^4.17.21", - "tslib": "~2.5.0" + "react-day-picker": "^8.10.0", + "react-innertext": "^1.1.5", + "tslib": "~2.6.2" + }, + "dependencies": { + "react-day-picker": { + "version": "8.10.1", + "resolved": "https://registry.npmjs.org/react-day-picker/-/react-day-picker-8.10.1.tgz", + "integrity": "sha512-TMx7fNbhLk15eqcMt+7Z7S2KF7mfTId/XJDjKE8f+IUcFn0l08/kI4FiYTL/0yuOLmEcbR4Fwe3GJf/NiiMnPA==" + } } }, "@blueprintjs/icons": { - "version": "4.16.0", - "resolved": "https://registry.npmjs.org/@blueprintjs/icons/-/icons-4.16.0.tgz", - "integrity": "sha512-cyfgjUZcZCtQrXWUV8FwqYTFEzduV4a0N7yhOU38jY+cBRCLu/sDrD0Osvfk4DGRvNe4YjY7pohVLFSxpg68Uw==", + "version": "5.10.0", + "resolved": "https://registry.npmjs.org/@blueprintjs/icons/-/icons-5.10.0.tgz", + "integrity": "sha512-Qp9H2U2/m6+p0Lh9ACuF2O6o0uVINMFimo+EgasgGaLx8SkJsl7ApB3hYQ+iRSYlQWotnoqd4Vtzj4c7sKukHQ==", "requires": { "change-case": "^4.1.2", "classnames": "^2.3.1", - "tslib": "~2.5.0" - } - }, - "@blueprintjs/popover2": { - "version": "1.14.11", - "resolved": "https://registry.npmjs.org/@blueprintjs/popover2/-/popover2-1.14.11.tgz", - "integrity": "sha512-5XAjeb2mlWjYXC0pqrNDLzHSsX85Zaiv8jixxUN9abarMUUFKGATgGF8MRsWTLAW94Gli6CB1lzVkrYkRHHf6Q==", - "requires": { - "@blueprintjs/core": "^4.20.2", - "@juggle/resize-observer": "^3.4.0", - "@popperjs/core": "^2.11.7", - "classnames": "^2.3.1", - "dom4": "^2.1.5", - "react-popper": "^2.3.0", - "tslib": "~2.5.0" + "tslib": "~2.6.2" } }, "@blueprintjs/select": { - "version": "4.9.24", - "resolved": "https://registry.npmjs.org/@blueprintjs/select/-/select-4.9.24.tgz", - "integrity": "sha512-OTjesxH/7UZvM7yAdHJ5u3sIjX1N8Rs4CQQ22AfqNl82SIROqkuXI31XEl6YNX1GsYfmAMiw0K7XohEKOMXR5g==", + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/@blueprintjs/select/-/select-5.2.1.tgz", + "integrity": "sha512-rXM9ik2mLd394g+sxwbHv2bGfrLbcdY9gFftqmJmm3vu7XJ2kRrsC1oOlNSC4dNBpUdHnyAzBTHEu81B1LkRbQ==", "requires": { - "@blueprintjs/core": "^4.20.2", - "@blueprintjs/popover2": "^1.14.11", + "@blueprintjs/core": "^5.10.5", + "@blueprintjs/icons": "^5.10.0", "classnames": "^2.3.1", - "tslib": "~2.5.0" + "tslib": "~2.6.2" } }, "@cspotcode/source-map-support": { @@ -19334,6 +19280,11 @@ "integrity": "sha512-lxJ9R5ygVm8ZWgYdUweoq5ownDlJ4upvoWmO4eLxBYHdMo+vZ/Rx0EN6MbKWDJOSUGrqJy2Gt+Dyv/VKml0fjg==", "dev": true }, + "@fontsource/open-sans": { + "version": "5.0.28", + "resolved": "https://registry.npmjs.org/@fontsource/open-sans/-/open-sans-5.0.28.tgz", + "integrity": "sha512-hBvJHY76pJT/JynGUB5EXWhnzjYfLdcMn655J5p1v9lTT9HdQSy+keq2KPVXO2Htlg998BBa3p6u/jlrZ6w0kg==" + }, "@humanwhocodes/config-array": { "version": "0.11.8", "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.11.8.tgz", @@ -19374,15 +19325,6 @@ "integrity": "sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA==", "dev": true }, - "@hypnosphi/create-react-context": { - "version": "0.3.1", - "resolved": "https://registry.npmjs.org/@hypnosphi/create-react-context/-/create-react-context-0.3.1.tgz", - "integrity": "sha512-V1klUed202XahrWJLLOT3EXNeCpFHCcJntdFGI15ntCwau+jfT386w7OFTMaCqOgXUH1fa0w/I1oZs+i/Rfr0A==", - "requires": { - "gud": "^1.0.0", - "warning": "^4.0.3" - } - }, "@istanbuljs/load-nyc-config": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz", @@ -20379,11 +20321,6 @@ "integrity": "sha512-9MVYlmIgmRR31C5b4FVSWtuMmBHh2mOWQYfl7XAYOa8dsnb7iEmUmRSWSFgXFtkjxO65d7hTUHQC+RhR/9IWFg==", "dev": true }, - "@types/dom4": { - "version": "2.0.2", - "resolved": "https://registry.npmjs.org/@types/dom4/-/dom4-2.0.2.tgz", - "integrity": "sha512-Rt4IC1T7xkCWa0OG1oSsPa0iqnxlDeQqKXZAHrQGLb7wFGncWm85MaxKUjAGejOrUynOgWlFi4c6S6IyJwoK4g==" - }, "@types/enzyme": { "version": "3.10.17", "resolved": "https://registry.npmjs.org/@types/enzyme/-/enzyme-3.10.17.tgz", @@ -21905,6 +21842,7 @@ "version": "1.0.2", "resolved": "https://registry.npmjs.org/call-bind/-/call-bind-1.0.2.tgz", "integrity": "sha512-7O+FbCihrB5WGbFYesctwmTKae6rOiIzmz1icreWJ+0aA7LJfuqhEso2T9ncpcFtzMQtzXf2QGGueWJGTYsqrA==", + "dev": true, "requires": { "function-bind": "^1.1.1", "get-intrinsic": "^1.0.2" @@ -22706,9 +22644,9 @@ } }, "date-fns-tz": { - "version": "1.3.8", - "resolved": "https://registry.npmjs.org/date-fns-tz/-/date-fns-tz-1.3.8.tgz", - "integrity": "sha512-qwNXUFtMHTTU6CFSFjoJ80W8Fzzp24LntbjFFBgL/faqds4e5mo9mftoRLgr3Vi1trISsg4awSpYVsOQCRnapQ==" + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/date-fns-tz/-/date-fns-tz-2.0.1.tgz", + "integrity": "sha512-fJCG3Pwx8HUoLhkepdsP7Z5RsucUi+ZBOxyM5d0ZZ6c4SdYustq0VMmOu6Wf7bli+yS/Jwp91TOCqn9jMcVrUA==" }, "debounce": { "version": "1.2.1", @@ -22759,19 +22697,6 @@ "integrity": "sha512-+LxW+KLWxu3HW3M2w2ympwtqPrqYRzU8fqi6Fhd18fBALe15blJPI/I4+UHveMVG6lJqB4JNd4UG0S5cnVHwIg==", "dev": true }, - "deep-equal": { - "version": "1.1.1", - "resolved": "https://registry.npmjs.org/deep-equal/-/deep-equal-1.1.1.tgz", - "integrity": "sha512-yd9c5AdiqVcR+JjcwUQb9DkhJc8ngNr0MahEBGvDiJw8puWab2yZlh+nkasOnZP+EGTAP6rRp2JzJhJZzvNF8g==", - "requires": { - "is-arguments": "^1.0.4", - "is-date-object": "^1.0.1", - "is-regex": "^1.0.4", - "object-is": "^1.0.1", - "object-keys": "^1.1.1", - "regexp.prototype.flags": "^1.2.0" - } - }, "deep-is": { "version": "0.1.4", "resolved": "https://registry.npmjs.org/deep-is/-/deep-is-0.1.4.tgz", @@ -22803,6 +22728,7 @@ "version": "1.2.0", "resolved": "https://registry.npmjs.org/define-properties/-/define-properties-1.2.0.tgz", "integrity": "sha512-xvqAVKGfT1+UAvPwKTVw/njhdQ8ZhXK4lI0bCIuCMrp2up9nPnaDftrLtmpTazqd1o+UY4zgzU+avtMbDP+ldA==", + "dev": true, "requires": { "has-property-descriptors": "^1.0.0", "object-keys": "^1.1.1" @@ -22921,11 +22847,6 @@ "entities": "^1.1.1" } }, - "dom4": { - "version": "2.1.6", - "resolved": "https://registry.npmjs.org/dom4/-/dom4-2.1.6.tgz", - "integrity": "sha512-JkCVGnN4ofKGbjf5Uvc8mmxaATIErKQKSgACdBXpsQ3fY6DlIpAyWfiBSrGkttATssbDCp3psiAKWXk5gmjycA==" - }, "domelementtype": { "version": "1.3.1", "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-1.3.1.tgz", @@ -24220,11 +24141,6 @@ "resolved": "https://registry.npmjs.org/follow-redirects/-/follow-redirects-1.15.6.tgz", "integrity": "sha512-wWN62YITEaOpSK584EZXJafH1AGpO8RVgElfkuXbTOrPX4fIfOyEpW/CsiNd8JdYrAoOvafRTOEnvsO++qCqFA==" }, - "fontsource-open-sans": { - "version": "3.0.9", - "resolved": "https://registry.npmjs.org/fontsource-open-sans/-/fontsource-open-sans-3.0.9.tgz", - "integrity": "sha512-NEgZAgtpvz7iYZ/IlojOOy6hRLaLmxnxzEKYXnq0KU/X8lb9cD7d0XImykhfksK1/JZ6xyKQKjYq4PycHQZK9w==" - }, "for-each": { "version": "0.3.3", "resolved": "https://registry.npmjs.org/for-each/-/for-each-0.3.3.tgz", @@ -24312,7 +24228,8 @@ "functions-have-names": { "version": "1.2.3", "resolved": "https://registry.npmjs.org/functions-have-names/-/functions-have-names-1.2.3.tgz", - "integrity": "sha512-xckBUXyTIqT97tq2x2AMb+g163b5JFysYk0x4qxNFwbfQkmNZoiRHb6sPzI9/QV33WeuvVYBUIiD4NzNIyqaRQ==" + "integrity": "sha512-xckBUXyTIqT97tq2x2AMb+g163b5JFysYk0x4qxNFwbfQkmNZoiRHb6sPzI9/QV33WeuvVYBUIiD4NzNIyqaRQ==", + "dev": true }, "gensync": { "version": "1.0.0-beta.2", @@ -24330,6 +24247,7 @@ "version": "1.2.0", "resolved": "https://registry.npmjs.org/get-intrinsic/-/get-intrinsic-1.2.0.tgz", "integrity": "sha512-L049y6nFOuom5wGyRc3/gdTLO94dySVKRACj1RmJZBQXlbTMhtNIgkWkUHq+jYmZvKf14EW1EoJnnjbmoHij0Q==", + "dev": true, "requires": { "function-bind": "^1.1.1", "has": "^1.0.3", @@ -24458,11 +24376,6 @@ "integrity": "sha512-bzh50DW9kTPM00T8y4o8vQg89Di9oLJVLW/KaOGIXJWP/iqCN6WKYkbNOF04vFLJhwcpYUh9ydh/+5vpOqV4YQ==", "dev": true }, - "gud": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/gud/-/gud-1.0.0.tgz", - "integrity": "sha512-zGEOVKFM5sVPPrYs7J5/hYEw2Pof8KCyOwyhG8sAF26mCAeUFAcYPu1mwB7hhpIP29zOIBaDqwuHdLp0jvZXjw==" - }, "gzip-size": { "version": "6.0.0", "resolved": "https://registry.npmjs.org/gzip-size/-/gzip-size-6.0.0.tgz", @@ -24518,6 +24431,7 @@ "version": "1.0.0", "resolved": "https://registry.npmjs.org/has-property-descriptors/-/has-property-descriptors-1.0.0.tgz", "integrity": "sha512-62DVLZGoiEBDHQyqG4w9xCuZ7eJEwNmJRWw2VY84Oedb7WFcA27fiEVe8oUQx9hAUJ4ekurquucTGwsyO1XGdQ==", + "dev": true, "requires": { "get-intrinsic": "^1.1.1" } @@ -24531,12 +24445,14 @@ "has-symbols": { "version": "1.0.3", "resolved": "https://registry.npmjs.org/has-symbols/-/has-symbols-1.0.3.tgz", - "integrity": "sha512-l3LCuF6MgDNwTDKkdYGEihYjt5pRPbEg46rtlmnSPlUbgmB8LOIrKJbYYFBSbnPaJexMKtiPO8hmeRjRz2Td+A==" + "integrity": "sha512-l3LCuF6MgDNwTDKkdYGEihYjt5pRPbEg46rtlmnSPlUbgmB8LOIrKJbYYFBSbnPaJexMKtiPO8hmeRjRz2Td+A==", + "dev": true }, "has-tostringtag": { "version": "1.0.0", "resolved": "https://registry.npmjs.org/has-tostringtag/-/has-tostringtag-1.0.0.tgz", "integrity": "sha512-kFjcSNhnlGV1kyoGk7OXKSawH5JOb/LzUc5w9B02hOTO0dfFRjbHQKvg1d6cf3HbeUmtU9VbbV3qzZ2Teh97WQ==", + "dev": true, "requires": { "has-symbols": "^1.0.2" } @@ -24965,6 +24881,7 @@ "version": "1.1.1", "resolved": "https://registry.npmjs.org/is-arguments/-/is-arguments-1.1.1.tgz", "integrity": "sha512-8Q7EARjzEnKpt/PCD7e1cgUS0a6X8u5tdSiMqXhojOdoV9TsMsiO+9VLC5vAmO8N7/GmXn7yjR8qnA6bVAEzfA==", + "dev": true, "requires": { "call-bind": "^1.0.2", "has-tostringtag": "^1.0.0" @@ -25047,6 +24964,7 @@ "version": "1.0.5", "resolved": "https://registry.npmjs.org/is-date-object/-/is-date-object-1.0.5.tgz", "integrity": "sha512-9YQaSxsAiSwcvS33MBk3wTCVnWK+HhF8VZR2jRxehM16QcVOdHqPn4VPHmRK4lSr38n9JriurInLcP90xsYNfQ==", + "dev": true, "requires": { "has-tostringtag": "^1.0.0" } @@ -25142,6 +25060,7 @@ "version": "1.1.4", "resolved": "https://registry.npmjs.org/is-regex/-/is-regex-1.1.4.tgz", "integrity": "sha512-kvRdxDsxZjhzUX07ZnLydzS1TU/TJlTUHHY4YLL87e37oUA49DfkLqgy+VjFocowy29cKvcSiu+kIv728jTTVg==", + "dev": true, "requires": { "call-bind": "^1.0.2", "has-tostringtag": "^1.0.0" @@ -27159,7 +27078,8 @@ "lodash": { "version": "4.17.21", "resolved": "https://registry.npmjs.org/lodash/-/lodash-4.17.21.tgz", - "integrity": "sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==" + "integrity": "sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==", + "dev": true }, "lodash.debounce": { "version": "4.0.8", @@ -27787,6 +27707,7 @@ "version": "1.1.5", "resolved": "https://registry.npmjs.org/object-is/-/object-is-1.1.5.tgz", "integrity": "sha512-3cyDsyHgtmi7I7DfSSI2LDp6SK2lwvtbg0p0R1e0RvTqF5ceGx+K2dfSjm1bKDMVCFEDAQvy+o8c6a7VujOddw==", + "dev": true, "requires": { "call-bind": "^1.0.2", "define-properties": "^1.1.3" @@ -27795,7 +27716,8 @@ "object-keys": { "version": "1.1.1", "resolved": "https://registry.npmjs.org/object-keys/-/object-keys-1.1.1.tgz", - "integrity": "sha512-NuAESUOUMrlIXOfHKzD6bpPu3tYt3xvjNdRIQ+FeT0lNb4K8WR70CaDxhuNguS2XG+GjkyMwOzsN5ZktImfhLA==" + "integrity": "sha512-NuAESUOUMrlIXOfHKzD6bpPu3tYt3xvjNdRIQ+FeT0lNb4K8WR70CaDxhuNguS2XG+GjkyMwOzsN5ZktImfhLA==", + "dev": true }, "object.assign": { "version": "4.1.4", @@ -28136,11 +28058,6 @@ "integrity": "sha512-Nc3IT5yHzflTfbjgqWcCPpo7DaKy4FnpB0l/zCAW0Tc7jxAiuqSxHasntB3D7887LSrA93kDJ9IXovxJYxyLCA==", "dev": true }, - "popper.js": { - "version": "1.16.1", - "resolved": "https://registry.npmjs.org/popper.js/-/popper.js-1.16.1.tgz", - "integrity": "sha512-Wb4p1J4zyFTbM+u6WuO4XstYx4Ky9Cewe4DWrel7B0w6VVICvPwdOpotjzcf6eD8TsckVnIMNONQyPIUFOUbCQ==" - }, "postcss": { "version": "8.4.31", "resolved": "https://registry.npmjs.org/postcss/-/postcss-8.4.31.tgz", @@ -29525,6 +29442,11 @@ "resolved": "https://registry.npmjs.org/react-fast-compare/-/react-fast-compare-3.2.2.tgz", "integrity": "sha512-nsO+KSNgo1SbJqJEYRE9ERzo7YtYbou/OqjSQKxV7jcKox7+usiUVZOAC+XnDOABXggQTno0Y1CpVnuWEc1boQ==" }, + "react-innertext": { + "version": "1.1.5", + "resolved": "https://registry.npmjs.org/react-innertext/-/react-innertext-1.1.5.tgz", + "integrity": "sha512-PWAqdqhxhHIv80dT9znP2KvS+hfkbRovFp4zFYHFFlOoQLRiawIic81gKb3U1wEyJZgMwgs3JoLtwryASRWP3Q==" + }, "react-is": { "version": "16.8.6", "resolved": "https://registry.npmjs.org/react-is/-/react-is-16.8.6.tgz", @@ -29611,6 +29533,14 @@ "prop-types": "^15.6.2" } }, + "react-uid": { + "version": "2.3.3", + "resolved": "https://registry.npmjs.org/react-uid/-/react-uid-2.3.3.tgz", + "integrity": "sha512-iNpDovcb9qBpBTo8iUgqRSQOS8GV3bWoNaTaUptHkXtAooXSo0OWe7vN6TqqB8x3x0bNBbQx96kkmSltQ5h9kQ==", + "requires": { + "tslib": "^2.0.0" + } + }, "read-installed": { "version": "4.0.3", "resolved": "https://registry.npmjs.org/read-installed/-/read-installed-4.0.3.tgz", @@ -29752,6 +29682,7 @@ "version": "1.5.0", "resolved": "https://registry.npmjs.org/regexp.prototype.flags/-/regexp.prototype.flags-1.5.0.tgz", "integrity": "sha512-0SutC3pNudRKgquxGoRGIz946MZVHqbNfPjBdxeOhBrdgDKlRoXmYLQN9xRbrR09ZXWeGAdPuif7egofn6v5LA==", + "dev": true, "requires": { "call-bind": "^1.0.2", "define-properties": "^1.2.0", @@ -31384,9 +31315,9 @@ } }, "tslib": { - "version": "2.5.3", - "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.5.3.tgz", - "integrity": "sha512-mSxlJJwl3BMEQCUNnxXBU9jP4JBktcEGhURcPR6VQVlnP0FdDEsIaz0C35dXNGLyRfrATNofF0F5p2KPxQgB+w==" + "version": "2.6.3", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.6.3.tgz", + "integrity": "sha512-xNvxJEOUiWPGhUuUdQgAJPKOOJfGnIyKySOc09XkKsgdUV/3E2zvwZYdejjmRgPCgcym1juLH3226yA7sEFJKQ==" }, "tsutils": { "version": "3.21.0", @@ -31447,11 +31378,6 @@ "is-typed-array": "^1.1.9" } }, - "typed-styles": { - "version": "0.0.7", - "resolved": "https://registry.npmjs.org/typed-styles/-/typed-styles-0.0.7.tgz", - "integrity": "sha512-pzP0PWoZUhsECYjABgCGQlRGL1n7tOHsgwYv3oIiEpJwGhFTuty/YNeduxQYzXXa3Ge5BdT6sHYIQYpl4uJ+5Q==" - }, "typedarray-to-buffer": { "version": "3.1.5", "resolved": "https://registry.npmjs.org/typedarray-to-buffer/-/typedarray-to-buffer-3.1.5.tgz", diff --git a/web-console/package.json b/web-console/package.json index 6e38ba24684..0c9370f8808 100644 --- a/web-console/package.json +++ b/web-console/package.json @@ -45,7 +45,7 @@ "prettify": "prettier --write '{src,e2e-tests}/**/*.{ts,tsx,scss}' './*.js'", "prettify-check": "prettier --check '{src,e2e-tests}/**/*.{ts,tsx,scss}' './*.js'", "generate-licenses-file": "license-checker --production --json --out licenses.json", - "check-licenses": "license-checker --production --onlyAllow 'Apache-1.1;Apache-2.0;BSD-2-Clause;BSD-3-Clause;0BSD;MIT;ISC;CC0-1.0' --summary", + "check-licenses": "license-checker --production --onlyAllow 'Apache-1.1;Apache-2.0;BSD-2-Clause;BSD-3-Clause;0BSD;MIT;ISC;CC0-1.0;OFL-1.1' --summary", "start": "webpack serve" }, "engines": { @@ -63,15 +63,15 @@ "not ie 11" ], "dependencies": { - "@blueprintjs/core": "^4.20.1", - "@blueprintjs/datetime": "^4.4.36", - "@blueprintjs/datetime2": "^0.9.35", - "@blueprintjs/icons": "^4.16.0", - "@blueprintjs/popover2": "^1.14.9", - "@blueprintjs/select": "^4.9.24", + "@blueprintjs/core": "^5.10.5", + "@blueprintjs/datetime": "^5.3.6", + "@blueprintjs/datetime2": "^2.3.7", + "@blueprintjs/icons": "^5.10.0", + "@blueprintjs/select": "^5.2.1", "@druid-toolkit/query": "^0.22.20", "@druid-toolkit/visuals-core": "^0.3.3", "@druid-toolkit/visuals-react": "^0.3.3", + "@fontsource/open-sans": "^5.0.28", "ace-builds": "~1.4.14", "axios": "^1.6.7", "chronoshift": "^0.10.0", @@ -83,9 +83,9 @@ "d3-dsv": "^2.0.0", "d3-scale": "^3.3.0", "d3-selection": "^2.0.0", + "date-fns": "^2.28.0", "echarts": "^5.4.3", "file-saver": "^2.0.5", - "fontsource-open-sans": "^3.0.9", "hjson": "^3.2.2", "json-bigint-native": "^1.2.0", "lodash.debounce": "^4.0.8", diff --git a/web-console/script/licenses b/web-console/script/licenses index 1dc820b94c1..90f1420282a 100755 --- a/web-console/script/licenses +++ b/web-console/script/licenses @@ -139,6 +139,11 @@ checker.init( licenseExt = '0BSD'; break; + case 'OFL-1.1': + properLicenseName = 'SIL Open Font License 1.1'; + licenseExt = 'OFL'; + break; + default: throw new Error(`Unknown license '${licenses}' in ${p}`); } diff --git a/web-console/src/blueprint-overrides/_index.scss b/web-console/src/blueprint-overrides/_index.scss index c345d815b80..1f5c774dfce 100644 --- a/web-console/src/blueprint-overrides/_index.scss +++ b/web-console/src/blueprint-overrides/_index.scss @@ -25,3 +25,4 @@ @import 'components/navbar/navbar'; @import 'components/card/card'; @import 'special-switch-modes'; +@import 'segmented-control'; diff --git a/web-console/src/components/date-range-selector/date-range-selector.scss b/web-console/src/blueprint-overrides/_segmented-control.scss similarity index 84% rename from web-console/src/components/date-range-selector/date-range-selector.scss rename to web-console/src/blueprint-overrides/_segmented-control.scss index 39df4d75f2a..8382d04a198 100644 --- a/web-console/src/components/date-range-selector/date-range-selector.scss +++ b/web-console/src/blueprint-overrides/_segmented-control.scss @@ -16,14 +16,7 @@ * limitations under the License. */ -@import '../../variables'; - -.date-range-selector { - .#{$bp-ns}-popover-target { - display: block; - } - - * { - cursor: pointer; - } +// Adding div explicitly to make this selector slightly stronger +.#{$ns}-dark div.#{$ns}-segmented-control { + background-color: $dark-gray3-half; } diff --git a/web-console/src/blueprint-overrides/_special-switch-modes.scss b/web-console/src/blueprint-overrides/_special-switch-modes.scss index 267ac4a2726..0d4c3928a27 100644 --- a/web-console/src/blueprint-overrides/_special-switch-modes.scss +++ b/web-console/src/blueprint-overrides/_special-switch-modes.scss @@ -16,23 +16,23 @@ * limitations under the License. */ -.bp4-dark .bp4-switch.bp4-control { +.#{$ns}-dark .#{$ns}-switch.#{$ns}-control { &.legacy-switch { - input:checked ~ .bp4-control-indicator { + input:checked ~ .#{$ns}-control-indicator { background: $orange5; } - &:hover input:checked ~ .bp4-control-indicator { + &:hover input:checked ~ .#{$ns}-control-indicator { background: $orange2; } } &.danger-switch { - input:checked ~ .bp4-control-indicator { + input:checked ~ .#{$ns}-control-indicator { background: $red5; } - &:hover input:checked ~ .bp4-control-indicator { + &:hover input:checked ~ .#{$ns}-control-indicator { background: $red2; } } diff --git a/web-console/src/blueprint-overrides/common/_colors.scss b/web-console/src/blueprint-overrides/common/_colors.scss index ca10be55628..a3286230217 100644 --- a/web-console/src/blueprint-overrides/common/_colors.scss +++ b/web-console/src/blueprint-overrides/common/_colors.scss @@ -23,6 +23,7 @@ $black: #0f1320; // Changed $dark-gray1: #181c2d; // Changed $dark-gray2: #24283b; $dark-gray3: #2f344e; +$dark-gray3-half: #343953; $dark-gray4: #383d57; $dark-gray5: #414765; diff --git a/web-console/src/blueprint-overrides/common/_variables.scss b/web-console/src/blueprint-overrides/common/_variables.scss index 1c3695693d0..d3fa168cbc3 100644 --- a/web-console/src/blueprint-overrides/common/_variables.scss +++ b/web-console/src/blueprint-overrides/common/_variables.scss @@ -20,7 +20,7 @@ // Namespace appended to the beginning of each CSS class: `.#{$ns}-button`. // Do not quote this value, for Less consumers. -$ns: bp4; +$ns: bp5; // easily the most important variable, so it comes up top // (so other variables can use it to define themselves) diff --git a/web-console/src/components/action-cell/__snapshots__/action-cell.spec.tsx.snap b/web-console/src/components/action-cell/__snapshots__/action-cell.spec.tsx.snap index a5bdd46adf1..4af64af7f53 100644 --- a/web-console/src/components/action-cell/__snapshots__/action-cell.spec.tsx.snap +++ b/web-console/src/components/action-cell/__snapshots__/action-cell.spec.tsx.snap @@ -6,8 +6,7 @@ exports[`ActionCell matches snapshot 1`] = ` >