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 b7bfe56ff02c..a33dbc0dcfc3 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 void setup() 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 37f7b665927c..98c27c4b2b8c 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 @@ -141,7 +141,7 @@ public void setup() @Benchmark public void measureNewestSegmentFirstPolicy(Blackhole blackhole) { - final CompactionSegmentIterator iterator = policy.reset(compactionConfigs, dataSources, Collections.emptyMap()); + final CompactionSegmentIterator iterator = policy.createIterator(compactionConfigs, dataSources, Collections.emptyMap()); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); } diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index 1b587cf4bac1..275be9e63dc8 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/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index dbd8fa7d5425..d94be5b0c5fd 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -914,13 +914,10 @@ Host: http://ROUTER_IP:ROUTER_PORT ### Get task segments :::info - This API is deprecated and will be removed in future releases. + This API is not supported anymore and always returns a 404 response. + Use the metric `segment/added/bytes` instead to identify the segment IDs committed by a task. ::: -Retrieves information about segments generated by the task given the task ID. To hit this endpoint, make sure to enable the audit log config on the Overlord with `druid.indexer.auditLog.enabled = true`. - -In addition to enabling audit logs, configure a cleanup strategy to prevent overloading the metadata store with old audit logs which may cause performance issues. To enable automated cleanup of audit logs on the Coordinator, set `druid.coordinator.kill.audit.on`. You may also manually export the audit logs to external storage. For more information, see [Audit records](../operations/clean-metadata-store.md#audit-records). - #### URL `GET` `/druid/indexer/v1/task/{taskId}/segments` @@ -929,12 +926,14 @@ In addition to enabling audit logs, configure a cleanup strategy to prevent over - + -
- -*Successfully retrieved task segments* +```json +{ + "error": "Segment IDs committed by a task action are not persisted anymore. Use the metric 'segment/added/bytes' to identify the segments created by a task." +} +```
diff --git a/docs/configuration/extensions.md b/docs/configuration/extensions.md index d396bc29000d..bc7a05a4ae23 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 c627e9fd7f07..3b3c2711d3b4 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 83a1d0c7e10b..d4e7592ee7f7 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 e0d9337b2ccb..919bf372b844 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 5ff77af07cf4..11c67ddfa2c4 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/native-batch-firehose.md b/docs/ingestion/native-batch-firehose.md index db2b3e8779e6..16e9634ff297 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:



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/clean-metadata-store.md b/docs/operations/clean-metadata-store.md index 49f2555c9d27..80e3494a53d1 100644 --- a/docs/operations/clean-metadata-store.md +++ b/docs/operations/clean-metadata-store.md @@ -44,7 +44,7 @@ This applies to all metadata entities in this topic except compaction configurat You can configure the retention period for each metadata type, when available, through the record's `durationToRetain` property. Certain records may require additional conditions be satisfied before clean up occurs. -See the [example](#example) for how you can customize the automated metadata cleanup for a specific use case. +See the [example](#example-configuration-for-automated-metadata-cleanup) for how you can customize the automated metadata cleanup for a specific use case. ## Automated cleanup strategies @@ -62,13 +62,12 @@ You can configure cleanup for each entity separately, as described in this secti Define the properties in the `coordinator/runtime.properties` file. The cleanup of one entity may depend on the cleanup of another entity as follows: -- You have to configure a [kill task for segment records](#kill-task) before you can configure automated cleanup for [rules](#rules-records) or [compaction configuration](#compaction-configuration-records). +- You have to configure a [kill task for segment records](#segment-records-and-segments-in-deep-storage-kill-task) before you can configure automated cleanup for [rules](#rules-records) or [compaction configuration](#compaction-configuration-records). - You have to schedule the metadata management tasks to run at the same or higher frequency as your most frequent cleanup job. For example, if your most frequent cleanup job is every hour, set the metadata store management period to one hour or less: `druid.coordinator.period.metadataStoreManagementPeriod=P1H`. For details on configuration properties, see [Metadata management](../configuration/index.md#metadata-management). -If you want to skip the details, check out the [example](#example) for configuring automated metadata cleanup. +If you want to skip the details, check out the [example](#example-configuration-for-automated-metadata-cleanup) for configuring automated metadata cleanup. - ### Segment records and segments in deep storage (kill task) :::info @@ -110,7 +109,7 @@ Supervisor cleanup uses the following configuration: ### Rules records -Rule records become eligible for deletion when all segments for the datasource have been killed by the kill task and the `durationToRetain` time has passed since their creation. Automated cleanup for rules requires a [kill task](#kill-task). +Rule records become eligible for deletion when all segments for the datasource have been killed by the kill task and the `durationToRetain` time has passed since their creation. Automated cleanup for rules requires a [kill task](#segment-records-and-segments-in-deep-storage-kill-task). Rule cleanup uses the following configuration: - `druid.coordinator.kill.rule.on`: When `true`, enables cleanup for rules records. @@ -129,7 +128,7 @@ To prevent the configuration from being prematurely removed, wait for the dataso Unlike other metadata records, compaction configuration records do not have a retention period set by `durationToRetain`. Druid deletes compaction configuration records at every cleanup cycle for inactive datasources, which do not have segments either used or unused. -Compaction configuration records in the `druid_config` table become eligible for deletion after all segments for the datasource have been killed by the kill task. Automated cleanup for compaction configuration requires a [kill task](#kill-task). +Compaction configuration records in the `druid_config` table become eligible for deletion after all segments for the datasource have been killed by the kill task. Automated cleanup for compaction configuration requires a [kill task](#segment-records-and-segments-in-deep-storage-kill-task). Compaction configuration cleanup uses the following configuration: - `druid.coordinator.kill.compaction.on`: When `true`, enables cleanup for compaction configuration records. @@ -153,7 +152,7 @@ Datasource cleanup uses the following configuration: You can configure the Overlord to periodically delete indexer task logs and associated metadata. During cleanup, the Overlord removes the following: * Indexer task logs from deep storage. -* Indexer task log metadata from the tasks and tasklogs tables in [metadata storage](../configuration/index.md#metadata-storage) (named `druid_tasks` and `druid_tasklogs` by default). Druid no longer uses the tasklogs table, and the table is always empty. +* Indexer task log metadata from the tasks table in [metadata storage](../configuration/index.md#metadata-storage) (named `druid_tasks` by default). To configure cleanup of task logs by the Overlord, set the following properties in the `overlord/runtime.properties` file. @@ -188,7 +187,6 @@ druid.coordinator.kill.rule.on=false druid.coordinator.kill.datasource.on=false ``` - ## Example configuration for automated metadata cleanup Consider a scenario where you have scripts to create and delete hundreds of datasources and related entities a day. You do not want to fill your metadata store with leftover records. The datasources and related entities tend to persist for only one or two days. Therefore, you want to run a cleanup job that identifies and removes leftover records that are at least four days old after a seven day buffer period in case you want to recover the data. The exception is for audit logs, which you need to retain for 30 days: diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index 1d37169684e1..b5be94ba4ba7 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 f470324b7f4e..540685a717f1 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 3bccfd3ef9aa..6ce65c0421e2 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 279a1327b97d..96389c8a72c7 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/lookups.md b/docs/querying/lookups.md index a22fbf03928c..05176f229b1f 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/scan-query.md b/docs/querying/scan-query.md index d758450715ec..503664633ba3 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 7151c23b9181..b04664ac90e3 100644 --- a/docs/querying/sql-functions.md +++ b/docs/querying/sql-functions.md @@ -27,18 +27,46 @@ 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) + +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 -This page provides a reference of all Druid SQL functions in alphabetical order. -Click the linked function type for documentation on a particular function. +Calculates the absolute value of a numeric expression. -## ABS +* **Syntax:** `ABS()` +* **Function type:** Scalar, numeric -`ABS()` +
Example -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +The following example applies the ABS function to the `ArrDelay` column from the `flight-carriers` datasource. -Calculates the absolute value of a numeric expression. +```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 @@ -64,6 +92,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) @@ -419,11 +448,29 @@ Rounds up a timestamp by a given time unit. ## CEIL (numeric) -`CEIL()` +Calculates the smallest integer value greater than or equal to the numeric expression. +* **Syntax:** `CEIL()` +* **Function type:** Scalar, numeric -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +
Example -Calculates the smallest integer value greater than or equal to the numeric expression. +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 @@ -697,11 +744,26 @@ Returns the value of a numeric or string expression corresponding to the earlies ## EXP -`EXP()` +Calculates _e_ raised to the power of the numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `EXP()` +* **Function type:** Scalar, numeric -Calculates _e_ raised to the power of the numeric expression. +
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 @@ -729,11 +791,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 + +
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: -Calculates the largest integer value less than or equal to the numeric expression. +| `fare_amount` | `floor_fare_amount` | +| -- | -- | +| `21.25` | `21` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## GREATEST @@ -961,19 +1042,57 @@ Returns the length of the expression in UTF-16 encoding. ## LN -`LN(expr)` +Calculates the natural logarithm of the numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `LN()` +* **Function type:** Scalar, numeric -Calculates the natural logarithm of the numeric expression. +
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 + +
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` | +
-Calculates the base-10 of the numeric expression. +[Learn more](sql-scalar.md#numeric-functions) ## LOOKUP @@ -1033,11 +1152,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 + +
Example + +The following calculates 78 MOD 10. + +```sql +SELECT MOD(78, 10) as "modulo" +``` +Returns the following: + +| `modulo` | +| -- | +| `8` | +
-Calculates x modulo y, or the remainder of x divided by y. +[Learn more](sql-scalar.md#numeric-functions) ## MV_APPEND @@ -1217,11 +1351,26 @@ Returns the one-based index position of a substring within an expression, option ## POWER -`POWER(expr, power)` +Calculates a numerical expression raised to the specified power. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `POWER(base, exponent)` +* **Function type:** Scalar, numeric -Calculates a numerical expression raised to the specified power. +
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 @@ -1298,11 +1447,31 @@ Returns the rightmost number of characters from an expression. ## ROUND -`ROUND(expr[, digits])` +Calculates the rounded value for a numerical expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `ROUND(expr[, digits])` +* **Function type:** Scalar, numeric -Calculates the rounded value for a numerical expression. +
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 @@ -1346,11 +1515,26 @@ Calculates the trigonometric sine of an angle expressed in radians. ## SQRT -`SQRT(expr)` +Calculates the square root of a numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `SQRT()` +* **Function type:** Scalar, numeric -Calculates the square root of a numeric expression. +
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 @@ -1620,20 +1804,41 @@ Trims the leading or trailing characters of an expression. ## TRUNC -`TRUNC(expr[, digits])` +Alias for [`TRUNCATE`](#truncate). -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `TRUNC(expr[, digits])` +* **Function type:** Scalar, numeric -Alias for [`TRUNCATE`](#truncate). +[Learn more](sql-scalar.md#numeric-functions) ## TRUNCATE -`TRUNCATE(expr[, digits])` +Truncates a numerical expression to a specific number of decimal digits. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `TRUNCATE(expr[, digits])` +* **Function type:** Scalar, numeric -Truncates a numerical expression to a specific number of decimal digits. +
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 +1888,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 056d63ece57c..e430caa8bf09 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 d64538779f07..7c2c3aef53e9 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 000000000000..71c7f30ffb00 --- /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 92053b83a11c..760b691d143d 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-sql-null.md b/docs/tutorials/tutorial-sql-null.md index 37cf23a71522..b91e8019bc25 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 86b5407e0e7c..e9fdeb158c1e 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/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 6c195ed15151..568f8ed5a117 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 ae7869707fca..885a4e3e59a9 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.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 4c2d01b5218b..bf3082a79b1e 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.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 SelectorBasedPodTemplateSelectStrategy( * @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 b374e0b6ff40..bda7788e3c41 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 @@ 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 TaskTypePodTemplateSelectStrategy() } @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 19cc788b3eea..e8aaf1bbab35 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 Job fromTask(Task task) throws IOException 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 000000000000..eeebd9fdb1fd --- /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 04b332aac850..b3fda99b222e 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 static Task getTask() 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 000000000000..58259606c86b --- /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 a82bb0768553..81589c751283 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.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 void testGetPodTemplate_ForTask_emptySelectorsFallbackToBaseTemplate() 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 void testGetPodTemplate_ForTask_noMatchSelectorsFallbackToBaseTemplateIfN 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 void testGetPodTemplate_ForTask_withMatchSelectors() ); 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 241b4d9fc68f..e2d97accc97f 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.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 void setup() { 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 102565efc35a..f8e7186a0263 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 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 @@ public 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 58993b9a6a00..9308835d967f 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.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 @@ public void setup() { 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 4aad419007e8..ac2aaa705581 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 void test_fromTask_withBasePodTemplateInRuntimeProperites() throws IOExce 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 void test_fromTask_withBasePodTemplateInRuntimeProperites_andTlsEnabled() 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 afc5299927fa..4cf9c3e4cbbc 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 @@ public void setup() { 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 2cef837f3972..ddae7c0567f2 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 000000000000..532c3dd53e82 --- /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 cf16c49c5db1..d6c316dcdde8 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 d72d0ef37b03..90ae99709598 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 a230ac913a60..724054454142 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 000000000000..0e52beac9e32 --- /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 7d59cb63ea5b..01039375259e 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.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 14bd59871253..65e71f626e8e 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.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 38f15a840dc2..ff1759195869 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.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/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json index 7e625c9edb3d..9cb01646f209 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-core/azure-extensions/pom.xml b/extensions-core/azure-extensions/pom.xml index 88a1198b61e1..7f0f767af851 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 833df8ab1a55..7d0bb79c71e1 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 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.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 @@ protected byte getCacheTypeId() @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 Aggregator factorize(final ColumnSelectorFactory columnSelectorFactory) @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 boolean canVectorize(ColumnInspector columnInspector) @Override public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory) { + validateInputs(selectorFactory.getColumnCapabilities(getFieldName())); return new HllSketchMergeVectorAggregator( selectorFactory, getFieldName(), @@ -142,6 +150,34 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFact ); } + /** + * 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 757674d6aa68..7d303d272740 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 @@ 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 d221b72ac1c6..15221c0f6f81 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.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.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 Aggregation toDruidAggregation( 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 Aggregation toDruidAggregation( } 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 @@ protected abstract Aggregation toAggregation( 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 211373e873b0..b24e382ec0aa 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.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.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 SketchAggregatorFactory(String name, String fieldName, Integer size, byte @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 Aggregator factorize(ColumnSelectorFactory metricFactory) @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 AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory) @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 BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) @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 eac77901f1d2..5ecd289c7287 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 @@ 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 bf35cd665ae8..1f45f31496ab 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.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.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 Aggregation toDruidAggregation( 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 Aggregation toDruidAggregation( ); } + 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 101b25b99be0..fcecef62d4aa 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 @@ 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 void setUp() 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 void testWithName() throws Exception 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 2907d6f8bb87..edb7dc5a11f4 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.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.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 @@ @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 SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker( .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 void testApproxCountDistinctHllSketchIsRounded() ); } + @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 1d70ff30f251..23887652a735 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.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 void testWithName() 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 247f924357aa..7afd2710ccd0 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.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.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 SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker( .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 void testAvgDailyCountDistinctThetaSketch() ); } + @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 0c7dff247c64..33450a3efccf 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 @@ private Map tryFetchUserMapFromCoordinator(Strin 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 58f982e71b4f..15b77be307d8 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 @@ protected QueryRunnerFactoryConglomerate makeQueryRunnerConglomerate() .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 da04e3ab0a6d..60d8f686a28c 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.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/kubernetes-extensions/pom.xml b/extensions-core/kubernetes-extensions/pom.xml index 613dfd756edd..e51d12184648 100644 --- a/extensions-core/kubernetes-extensions/pom.xml +++ b/extensions-core/kubernetes-extensions/pom.xml @@ -158,10 +158,12 @@ org/apache/druid/k8s/discovery/K8sDiscoveryModule* + org/apache/druid/k8s/discovery/K8sDruidLeaderSelectorProvider* org/apache/druid/k8s/discovery/DefaultK8sApiClient* org/apache/druid/k8s/discovery/DefaultK8sLeaderElectorFactory* +
diff --git a/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDiscoveryModule.java b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDiscoveryModule.java index 6da6819ff4b5..b18f03d7f5b0 100644 --- a/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDiscoveryModule.java +++ b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDiscoveryModule.java @@ -21,9 +21,7 @@ import com.fasterxml.jackson.databind.Module; import com.google.inject.Binder; -import com.google.inject.Inject; import com.google.inject.Key; -import com.google.inject.Provider; import io.kubernetes.client.openapi.ApiClient; import io.kubernetes.client.util.Config; import org.apache.druid.client.coordinator.Coordinator; @@ -34,9 +32,7 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.PolyBind; -import org.apache.druid.guice.annotations.Self; import org.apache.druid.initialization.DruidModule; -import org.apache.druid.server.DruidNode; import java.io.IOException; import java.util.Collections; @@ -88,65 +84,15 @@ public void configure(Binder binder) PolyBind.optionBinder(binder, Key.get(DruidLeaderSelector.class, Coordinator.class)) .addBinding(K8S_KEY) .toProvider( - new DruidLeaderSelectorProvider(true) + K8sDruidLeaderSelectorProvider.K8sCoordinatorDruidLeaderSelectorProvider.class ) .in(LazySingleton.class); PolyBind.optionBinder(binder, Key.get(DruidLeaderSelector.class, IndexingService.class)) .addBinding(K8S_KEY) .toProvider( - new DruidLeaderSelectorProvider(false) + K8sDruidLeaderSelectorProvider.K8sIndexingServiceDruidLeaderSelectorProvider.class ) .in(LazySingleton.class); } - - private static class DruidLeaderSelectorProvider implements Provider - { - @Inject - @Self - private DruidNode druidNode; - - @Inject - private PodInfo podInfo; - - @Inject - private K8sDiscoveryConfig discoveryConfig; - - @Inject - private Provider k8sApiClientProvider; - - private boolean isCoordinator; - - DruidLeaderSelectorProvider(boolean isCoordinator) - { - this.isCoordinator = isCoordinator; - } - - @Override - public DruidLeaderSelector get() - { - // Note: these can not be setup in the constructor because injected K8sDiscoveryConfig and PodInfo - // are not available at that time. - String lockResourceName; - String lockResourceNamespace; - - if (isCoordinator) { - lockResourceName = discoveryConfig.getClusterIdentifier() + "-leaderelection-coordinator"; - lockResourceNamespace = discoveryConfig.getCoordinatorLeaderElectionConfigMapNamespace() == null ? - podInfo.getPodNamespace() : discoveryConfig.getCoordinatorLeaderElectionConfigMapNamespace(); - } else { - lockResourceName = discoveryConfig.getClusterIdentifier() + "-leaderelection-overlord"; - lockResourceNamespace = discoveryConfig.getOverlordLeaderElectionConfigMapNamespace() == null ? - podInfo.getPodNamespace() : discoveryConfig.getOverlordLeaderElectionConfigMapNamespace(); - } - - return new K8sDruidLeaderSelector( - druidNode, - lockResourceName, - lockResourceNamespace, - discoveryConfig, - new DefaultK8sLeaderElectorFactory(k8sApiClientProvider.get(), discoveryConfig) - ); - } - } } diff --git a/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDruidLeaderSelectorProvider.java b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDruidLeaderSelectorProvider.java new file mode 100644 index 000000000000..cff6a8bdf468 --- /dev/null +++ b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDruidLeaderSelectorProvider.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.discovery; + +import com.google.inject.Inject; +import com.google.inject.Provider; +import io.kubernetes.client.openapi.ApiClient; +import org.apache.druid.discovery.DruidLeaderSelector; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.server.DruidNode; + +public abstract class K8sDruidLeaderSelectorProvider implements Provider +{ + @Inject + @Self + private DruidNode druidNode; + + @Inject + private PodInfo podInfo; + + @Inject + private K8sDiscoveryConfig discoveryConfig; + + @Inject + private Provider k8sApiClientProvider; + + private boolean isCoordinator; + + K8sDruidLeaderSelectorProvider(boolean isCoordinator) + { + this.isCoordinator = isCoordinator; + } + + @Override + public DruidLeaderSelector get() + { + // Note: these can not be setup in the constructor because injected K8sDiscoveryConfig and PodInfo + // are not available at that time. + String lockResourceName; + String lockResourceNamespace; + + if (isCoordinator) { + lockResourceName = discoveryConfig.getClusterIdentifier() + "-leaderelection-coordinator"; + lockResourceNamespace = discoveryConfig.getCoordinatorLeaderElectionConfigMapNamespace() == null + ? + podInfo.getPodNamespace() + : discoveryConfig.getCoordinatorLeaderElectionConfigMapNamespace(); + } else { + lockResourceName = discoveryConfig.getClusterIdentifier() + "-leaderelection-overlord"; + lockResourceNamespace = discoveryConfig.getOverlordLeaderElectionConfigMapNamespace() == null ? + podInfo.getPodNamespace() : discoveryConfig.getOverlordLeaderElectionConfigMapNamespace(); + } + + return new K8sDruidLeaderSelector( + druidNode, + lockResourceName, + lockResourceNamespace, + discoveryConfig, + new DefaultK8sLeaderElectorFactory(k8sApiClientProvider.get(), discoveryConfig) + ); + } + + static class K8sCoordinatorDruidLeaderSelectorProvider extends K8sDruidLeaderSelectorProvider + { + @Inject + public K8sCoordinatorDruidLeaderSelectorProvider() + { + super(true); + } + } + + static class K8sIndexingServiceDruidLeaderSelectorProvider extends K8sDruidLeaderSelectorProvider + { + @Inject + public K8sIndexingServiceDruidLeaderSelectorProvider() + { + super(false); + } + } +} 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 e960a8aea29c..0fef9d32e6d4 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.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 ac43e7c864b8..fb25097e9800 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 @@ -359,7 +359,6 @@ private static Query buildScanQuery(CompactionTask compactionTask, Interval i .virtualColumns(getVirtualColumns(dataSchema, interval)) .columnTypes(rowSignature.getColumnTypes()) .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval))) - .legacy(false) .filters(dataSchema.getTransformSpec().getFilter()) .context(compactionTask.getContext()) .build(); 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 bf3dd4a6bf14..d161b01bd0b6 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 @@ 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 4ef6ab077cac..70d1ab11d380 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.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 3817e63ca499..16f9deff63d0 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 Pair apply(ReadableInput readableInput) // 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 6d8cfdfd2773..5fbfd3119d03 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 @@ -38,7 +38,6 @@ import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.TooManyRowsInAWindowFault; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.operator.NaivePartitioningOperatorFactory; import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.operator.Operator; import org.apache.druid.query.operator.OperatorFactory; @@ -52,6 +51,7 @@ 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; @@ -60,7 +60,6 @@ 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; @@ -70,6 +69,7 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor private final WindowOperatorQuery query; private final List operatorFactoryList; + private final List partitionColumnNames; private final ObjectMapper jsonMapper; private final ArrayList frameRowsAndCols; private final ArrayList resultRowAndCols; @@ -79,13 +79,15 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor private final FrameReader frameReader; private final ArrayList objectsOfASingleRac; private final int maxRowsMaterialized; - List partitionColsIndex; private long currentAllocatorCapacity; // Used for generating FrameRowTooLargeException if needed private Cursor frameCursor = null; 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, @@ -96,8 +98,8 @@ public WindowOperatorQueryFrameProcessor( ObjectMapper jsonMapper, final List operatorFactoryList, final RowSignature rowSignature, - final boolean isOverEmpty, - final int maxRowsMaterializedInWindow + final int maxRowsMaterializedInWindow, + final List partitionColumnNames ) { this.inputChannel = inputChannel; @@ -105,14 +107,18 @@ public WindowOperatorQueryFrameProcessor( 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.partitionColsIndex = 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 ReturnOrAwait runIncrementally(IntSet readableInputs) * * * 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 @@ -177,24 +183,22 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) * * Future thoughts: {@link https://github.com/apache/druid/issues/16126} * - * 1. We are writing 1 partition to each frame in this way. In case of low cardinality data - * we will me making a large number of small frames. We can have a check to keep size of frame to a value + * 1. We are writing 1 partition to each frame in this way. In case of high cardinality data + * we will be making a large number of small frames. We can have a check to keep size of frame to a value * say 20k rows and keep on adding to the same pending frame and not create a new frame * * 2. Current approach with R&C and operators materialize a single R&C for processing. In case of data - * with high cardinality a single R&C might be too big to consume. Same for the case of empty OVER() clause + * with low cardinality a single R&C might be too big to consume. Same for the case of empty OVER() clause * Most of the window operations like SUM(), RANK(), RANGE() etc. can be made with 2 passes of the data. * 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); @@ -218,7 +222,6 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) final Frame frame = inputChannel.read(); frameCursor = FrameProcessors.makeCursor(frame, frameReader); final ColumnSelectorFactory frameColumnSelectorFactory = frameCursor.getColumnSelectorFactory(); - partitionColsIndex = findPartitionColumns(frameReader.signature()); final Supplier[] fieldSuppliers = new Supplier[frameReader.signature().size()]; for (int i = 0; i < fieldSuppliers.length; i++) { final ColumnValueSelector selector = @@ -259,18 +262,17 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) if (outputRow == null) { outputRow = currentRow; objectsOfASingleRac.add(currentRow); - } else if (comparePartitionKeys(outputRow, currentRow, partitionColsIndex)) { + } else if (comparePartitionKeys(outputRow, currentRow, partitionColumnNames)) { // if they have the same partition key // keep adding them after checking // guardrails + objectsOfASingleRac.add(currentRow); if (objectsOfASingleRac.size() > maxRowsMaterialized) { throw new MSQException(new TooManyRowsInAWindowFault( objectsOfASingleRac.size(), maxRowsMaterialized )); } - objectsOfASingleRac.add(currentRow); - } else { // key change noted // create rac from the rows seen before @@ -484,37 +486,32 @@ private void convertRowFrameToRowsAndColumns(Frame frame) frameRowsAndCols.add(ldrc); } - private List findPartitionColumns(RowSignature rowSignature) - { - List indexList = new ArrayList<>(); - for (OperatorFactory of : operatorFactoryList) { - if (of instanceof NaivePartitioningOperatorFactory) { - for (String s : ((NaivePartitioningOperatorFactory) of).getPartitionColumns()) { - indexList.add(rowSignature.indexOf(s)); - } - } - } - return indexList; - } - /** - * - * Compare two rows based only the columns in the partitionIndices - * In case the parition indices is empty or null compare entire row - * + * Compare two rows based on the columns in partitionColumnNames. + * If the partitionColumnNames is empty, the method will end up returning true. + *

+ * For example, say: + *

    + *
  • partitionColumnNames = ["d1", "d2"]
  • + *
  • frameReader's row signature = {d1:STRING, d2:STRING, p0:STRING}
  • + *
  • frameReader.signature.indexOf("d1") = 0
  • + *
  • frameReader.signature.indexOf("d2") = 1
  • + *
  • row1 = [d1_row1, d2_row1, p0_row1]
  • + *
  • row2 = [d1_row2, d2_row2, p0_row2]
  • + *
+ *

+ * Then this method will return true if d1_row1==d1_row2 && d2_row1==d2_row2, false otherwise. + * Returning true would indicate that these 2 rows can be put into the same partition for window function processing. */ - private boolean comparePartitionKeys(ResultRow row1, ResultRow row2, List partitionIndices) + private boolean comparePartitionKeys(ResultRow row1, ResultRow row2, List partitionColumnNames) { - if (partitionIndices == null || partitionIndices.isEmpty()) { - return row1.equals(row2); - } else { - int match = 0; - for (int i : partitionIndices) { - 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 == partitionIndices.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 fbbc0a0fc3e7..9852f4f40988 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.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,23 +60,27 @@ 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; @JsonCreator public WindowOperatorQueryFrameProcessorFactory( @JsonProperty("query") WindowOperatorQuery query, @JsonProperty("operatorList") List operatorFactoryList, @JsonProperty("stageRowSignature") RowSignature stageRowSignature, - @JsonProperty("emptyOver") boolean emptyOver, - @JsonProperty("maxRowsMaterializedInWindow") int maxRowsMaterializedInWindow + @JsonProperty("maxRowsMaterializedInWindow") int maxRowsMaterializedInWindow, + @JsonProperty("partitionColumnNames") List partitionColumnNames ) { 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; } @JsonProperty("query") @@ -90,16 +95,16 @@ public List getOperators() return operatorList; } - @JsonProperty("stageRowSignature") - public RowSignature getSignature() + @JsonProperty("partitionColumnNames") + public List getPartitionColumnNames() { - return stageRowSignature; + return partitionColumnNames; } - @JsonProperty("emptyOver") - public boolean isEmptyOverFound() + @JsonProperty("stageRowSignature") + public RowSignature getSignature() { - return isEmptyOver; + return stageRowSignature; } @JsonProperty("maxRowsMaterializedInWindow") @@ -148,7 +153,6 @@ public ProcessorsAndChannels makeProcessors( readableInput -> { final OutputChannel outputChannel = outputChannels.get(readableInput.getStagePartition().getPartitionNumber()); - return new WindowOperatorQueryFrameProcessor( query, readableInput.getChannel(), @@ -158,8 +162,8 @@ public ProcessorsAndChannels makeProcessors( frameContext.jsonMapper(), operatorList, stageRowSignature, - isEmptyOver, - maxRowsMaterializedInWindow + maxRowsMaterializedInWindow, + partitionColumnNames ); } ); @@ -181,16 +185,16 @@ public boolean equals(Object o) 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) && Objects.equals(stageRowSignature, that.stageRowSignature); } @Override public int hashCode() { - return Objects.hash(query, operatorList, 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 d08d78ef791f..23e13f176d7b 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 @@ -24,9 +24,12 @@ import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.frame.key.KeyColumn; import org.apache.druid.frame.key.KeyOrder; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.Limits; import org.apache.druid.msq.input.stage.StageInputSpec; import org.apache.druid.msq.kernel.HashShuffleSpec; +import org.apache.druid.msq.kernel.MixShuffleSpec; import org.apache.druid.msq.kernel.QueryDefinition; import org.apache.druid.msq.kernel.QueryDefinitionBuilder; import org.apache.druid.msq.kernel.ShuffleSpec; @@ -39,15 +42,18 @@ import org.apache.druid.query.operator.OperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; import org.apache.druid.query.operator.window.WindowOperatorFactory; +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; public class WindowOperatorQueryKit implements QueryKit { + private static final Logger log = new Logger(WindowOperatorQueryKit.class); private final ObjectMapper jsonMapper; public WindowOperatorQueryKit(ObjectMapper jsonMapper) @@ -65,13 +71,22 @@ public QueryDefinition makeQueryDefinition( int minStageNumber ) { - // need to validate query first - // populate the group of operators to be processed as each stage - // the size of the operators is the number of serialized stages - // later we should also check if these can be parallelized - // check there is an empty over clause or not - List> operatorList = new ArrayList<>(); - boolean isEmptyOverFound = ifEmptyOverPresentInWindowOperstors(originalQuery, operatorList); + // Need to validate query first. + // Populate the group of operators to be processed at each stage. + // The size of the operators is the number of serialized stages. + // Later we should also check if these can be parallelized. + // Check if there is an empty OVER() clause or not. + RowSignature rowSignature = originalQuery.getRowSignature(); + log.info("Row signature received for query is [%s].", rowSignature); + + boolean isEmptyOverPresent = originalQuery.getOperators() + .stream() + .filter(of -> of instanceof NaivePartitioningOperatorFactory) + .map(of -> (NaivePartitioningOperatorFactory) of) + .anyMatch(of -> of.getPartitionColumns().isEmpty()); + + List> operatorList = getOperatorListFromQuery(originalQuery); + log.info("Created operatorList with operator factories: [%s]", operatorList); ShuffleSpec nextShuffleSpec = findShuffleSpecForNextWindow(operatorList.get(0), maxWorkerCount); // add this shuffle spec to the last stage of the inner query @@ -102,18 +117,19 @@ public QueryDefinition makeQueryDefinition( final int firstStageNumber = Math.max(minStageNumber, queryDefBuilder.getNextStageNumber()); final WindowOperatorQuery queryToRun = (WindowOperatorQuery) originalQuery.withDataSource(dataSourcePlan.getNewDataSource()); final int maxRowsMaterialized; - RowSignature rowSignature = queryToRun.getRowSignature(); + if (originalQuery.context() != null && originalQuery.context().containsKey(MultiStageQueryContext.MAX_ROWS_MATERIALIZED_IN_WINDOW)) { - maxRowsMaterialized = (int) originalQuery.context() - .get(MultiStageQueryContext.MAX_ROWS_MATERIALIZED_IN_WINDOW); + maxRowsMaterialized = (int) originalQuery.context().get(MultiStageQueryContext.MAX_ROWS_MATERIALIZED_IN_WINDOW); } else { maxRowsMaterialized = Limits.MAX_ROWS_MATERIALIZED_IN_WINDOW; } - - if (isEmptyOverFound) { - // empty over clause found - // moving everything to a single partition + if (isEmptyOverPresent) { + // 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)) @@ -124,29 +140,59 @@ public QueryDefinition makeQueryDefinition( queryToRun, queryToRun.getOperators(), rowSignature, - true, - maxRowsMaterialized + maxRowsMaterialized, + Collections.emptyList() )) ); } else { - // there are multiple windows present in the query - // Create stages for each window in the query - // These stages will be serialized - // the partition by clause of the next window will be the shuffle key for the previous window + // There are multiple windows present in the query. + // Create stages for each window in the query. + // These stages will be serialized. + // The partition by clause of the next window will be the shuffle key for the previous window. RowSignature.Builder bob = RowSignature.builder(); - final int numberOfWindows = operatorList.size(); - final int baseSize = rowSignature.size() - numberOfWindows; - for (int i = 0; i < baseSize; i++) { - bob.add(rowSignature.getColumnName(i), rowSignature.getColumnType(i).get()); + RowSignature signatureFromInput = dataSourcePlan.getSubQueryDefBuilder().get().build().getFinalStageDefinition().getSignature(); + log.info("Row signature received from last stage is [%s].", signatureFromInput); + + for (int i = 0; i < signatureFromInput.getColumnNames().size(); i++) { + bob.add(signatureFromInput.getColumnName(i), signatureFromInput.getColumnType(i).get()); } - for (int i = 0; i < numberOfWindows; i++) { - bob.add(rowSignature.getColumnName(baseSize + i), rowSignature.getColumnType(baseSize + i).get()).build(); + List partitionColumnNames = new ArrayList<>(); + + /* + operatorList is a List>, where each List corresponds to the operator factories + to be used for a different window stage. + + We iterate over operatorList, and add the definition for a window stage to QueryDefinitionBuilder. + */ + for (int i = 0; i < operatorList.size(); i++) { + for (OperatorFactory operatorFactory : operatorList.get(i)) { + if (operatorFactory instanceof WindowOperatorFactory) { + List outputColumnNames = ((WindowOperatorFactory) operatorFactory).getProcessor().getOutputColumnNames(); + + // Need to add column names which are present in outputColumnNames and rowSignature but not in bob, + // since they need to be present in the row signature for this window stage. + for (String columnName : outputColumnNames) { + int indexInRowSignature = rowSignature.indexOf(columnName); + if (indexInRowSignature != -1 && bob.build().indexOf(columnName) == -1) { + ColumnType columnType = rowSignature.getColumnType(indexInRowSignature).get(); + bob.add(columnName, columnType); + log.info("Added column [%s] of type [%s] to row signature for window stage.", columnName, columnType); + } else { + throw new ISE( + "Found unexpected column [%s] already present in row signature [%s].", + columnName, + rowSignature + ); + } + } + } + } + // find the shuffle spec of the next stage // if it is the last stage set the next shuffle spec to single partition - if (i + 1 == numberOfWindows) { - nextShuffleSpec = ShuffleSpecFactories.singlePartition() - .build(ClusterBy.none(), false); + if (i + 1 == operatorList.size()) { + nextShuffleSpec = MixShuffleSpec.instance(); } else { nextShuffleSpec = findShuffleSpecForNextWindow(operatorList.get(i + 1), maxWorkerCount); } @@ -162,6 +208,28 @@ public QueryDefinition makeQueryDefinition( ); } + log.info("Using row signature [%s] for window stage.", stageRowSignature); + + boolean partitionOperatorExists = false; + List currentPartitionColumns = new ArrayList<>(); + for (OperatorFactory of : operatorList.get(i)) { + if (of instanceof NaivePartitioningOperatorFactory) { + for (String s : ((NaivePartitioningOperatorFactory) of).getPartitionColumns()) { + currentPartitionColumns.add(s); + partitionOperatorExists = true; + } + } + } + + if (partitionOperatorExists) { + partitionColumnNames = currentPartitionColumns; + } + + log.info( + "Columns which would be used to define partitioning boundaries for this window stage are [%s]", + partitionColumnNames + ); + queryDefBuilder.add( StageDefinition.builder(firstStageNumber + i) .inputs(new StageInputSpec(firstStageNumber + i - 1)) @@ -172,8 +240,8 @@ public QueryDefinition makeQueryDefinition( queryToRun, operatorList.get(i), stageRowSignature, - false, - maxRowsMaterialized + maxRowsMaterialized, + partitionColumnNames )) ); } @@ -184,37 +252,48 @@ public QueryDefinition makeQueryDefinition( /** * * @param originalQuery - * @param operatorList - * @return true if the operator List has a partitioning operator with an empty OVER clause, false otherwise + * @return A list of list of operator factories, where each list represents the operator factories for a particular + * window stage. */ - private boolean ifEmptyOverPresentInWindowOperstors( - WindowOperatorQuery originalQuery, - List> operatorList - ) + private List> getOperatorListFromQuery(WindowOperatorQuery originalQuery) { + 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 true; + // 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(); } } - return false; + + // 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; } private ShuffleSpec findShuffleSpecForNextWindow(List operatorFactories, int maxWorkerCount) { NaivePartitioningOperatorFactory partition = null; NaiveSortOperatorFactory sort = null; - List keyColsOfWindow = new ArrayList<>(); for (OperatorFactory of : operatorFactories) { if (of instanceof NaivePartitioningOperatorFactory) { partition = (NaivePartitioningOperatorFactory) of; @@ -222,29 +301,31 @@ private ShuffleSpec findShuffleSpecForNextWindow(List operatorF sort = (NaiveSortOperatorFactory) of; } } - Map colMap = new HashMap<>(); + + Map sortColumnsMap = new HashMap<>(); if (sort != null) { for (ColumnWithDirection sortColumn : sort.getSortColumns()) { - colMap.put(sortColumn.getColumn(), sortColumn.getDirection()); + sortColumnsMap.put(sortColumn.getColumn(), sortColumn.getDirection()); } } - assert partition != null; - if (partition.getPartitionColumns().isEmpty()) { + + if (partition == null || partition.getPartitionColumns().isEmpty()) { + // If operatorFactories doesn't have any partitioning factory, then we should keep the shuffle spec from previous stage. + // This indicates that we already have the data partitioned correctly, and hence we don't need to do any shuffling. return null; } + + List keyColsOfWindow = new ArrayList<>(); for (String partitionColumn : partition.getPartitionColumns()) { KeyColumn kc; - if (colMap.containsKey(partitionColumn)) { - if (colMap.get(partitionColumn) == ColumnWithDirection.Direction.ASC) { - kc = new KeyColumn(partitionColumn, KeyOrder.ASCENDING); - } else { - kc = new KeyColumn(partitionColumn, KeyOrder.DESCENDING); - } + if (sortColumnsMap.get(partitionColumn) == ColumnWithDirection.Direction.DESC) { + kc = new KeyColumn(partitionColumn, KeyOrder.DESCENDING); } else { kc = new KeyColumn(partitionColumn, KeyOrder.ASCENDING); } keyColsOfWindow.add(kc); } + return new HashShuffleSpec(new ClusterBy(keyColsOfWindow, 0), maxWorkerCount); } } 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 c570fdc29b91..2104d1d40f41 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 ScanQueryFrameProcessor( @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/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 06408af2a1ba..f04a65f89a84 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 void setUp() .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/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index 2d14e7434974..c6d67f14428f 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 void testSelectUnnestOnFoo(String contextName, Map contex )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(defaultScanQueryContext( context, resultSignature @@ -2668,7 +2667,6 @@ public void testSelectUnnestOnQueryFoo(String contextName, Map c ) .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 void testSelectUnnestOnQueryFoo(String contextName, Map c )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(defaultScanQueryContext( context, resultSignature @@ -2751,7 +2748,6 @@ public void testUnionAllUsingUnionDataSource(String contextName, Map cont .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 void testWindowOnFooWithDim2(String contextName, Map cont .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 void testWindowOnFooWithEmptyOverWithUnnest(String contextName, Map cont .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 void testSelectWithWikipedia(String contextName, Map cont .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 void testSelectWithWikipediaWithPartitionKeyNotInSelect(String contextNam .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 void testSelectWithWikipediaWithPartitionKeyNotInSelect(String contextNam .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 void testSelectWithWikipediaWithPartitionKeyNotInSelect(String contextNam .limit(5) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(outerContextWithRowSignature) - .legacy(false) .build(); testSelectQuery() @@ -2016,7 +1970,7 @@ public void testGroupByWithWikipedia(String contextName, Map con .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/MSQControllerTaskTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java index 9de14610f19c..9df6c38f30e1 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 new file mode 100644 index 000000000000..802a4c52f172 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactoryTest.java @@ -0,0 +1,35 @@ +/* + * 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.msq.querykit; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class WindowOperatorQueryFrameProcessorFactoryTest +{ + @Test + public void testEqualsAndHashcode() + { + EqualsVerifier.forClass(WindowOperatorQueryFrameProcessorFactory.class) + .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 a6844fcabffb..e3272cbcee7e 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 void test_runWithInputChannel() throws Exception .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 6d81da64aab0..3036c2617622 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 @@ -307,7 +307,6 @@ public void testExplain() throws IOException + "\"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\"}}", 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 2da5fd42caf1..a97ee01297fc 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/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 54552e5d5b0b..eaa2a9efe5ae 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -39,6 +39,7 @@ import org.apache.druid.guice.annotations.Self; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.java.util.common.ISE; +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.math.expr.ExprMacroTable; @@ -62,12 +63,14 @@ import org.apache.druid.query.groupby.TestGroupByBuffers; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusher; @@ -91,6 +94,7 @@ import java.io.File; import java.util.List; import java.util.Set; +import java.util.UUID; import java.util.function.Function; import java.util.function.Supplier; @@ -99,6 +103,7 @@ import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE2; import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE3; import static org.apache.druid.sql.calcite.util.CalciteTests.DATASOURCE5; +import static org.apache.druid.sql.calcite.util.CalciteTests.WIKIPEDIA; import static org.apache.druid.sql.calcite.util.TestDataBuilder.INDEX_SCHEMA_LOTS_O_COLUMNS; import static org.apache.druid.sql.calcite.util.TestDataBuilder.INDEX_SCHEMA_NUMERIC_DIMS; import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS1; @@ -205,6 +210,17 @@ private static Supplier> getSupplierForSegment(Function< { final QueryableIndex index; switch (segmentId.getDataSource()) { + case WIKIPEDIA: + try { + final File directory = new File(tempFolderProducer.apply("tmpDir"), StringUtils.format("wikipedia-index-%s", UUID.randomUUID())); + final IncrementalIndex incrementalIndex = TestIndex.makeWikipediaIncrementalIndex(); + TestIndex.INDEX_MERGER.persist(incrementalIndex, directory, IndexSpec.DEFAULT, null); + index = TestIndex.INDEX_IO.loadIndex(directory); + } + catch (Exception e) { + throw new RuntimeException(e); + } + break; case DATASOURCE1: IncrementalIndexSchema foo1Schema = new IncrementalIndexSchema.Builder() .withMetrics( 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 ad05c20b5829..14f6f73b24ab 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.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/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 07f48f36cd39..4647434aba10 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.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 @@ @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 boolean equals(Object o) 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 2cb8cc7dc79a..3f62e1d7eb36 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.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.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 List getJacksonModules() 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 43d96961c1ba..1ed7214aa961 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 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 String getConnectURI() 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 String getConnectURI() JdbcAccessSecurityConfig securityConfig = newSecurityConfigEnforcingAllowList(ImmutableSet.of()); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -145,7 +145,7 @@ public String getConnectURI() JdbcAccessSecurityConfig securityConfig = newSecurityConfigEnforcingAllowList(ImmutableSet.of("user")); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -170,7 +170,7 @@ public String getConnectURI() 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 String getConnectURI() ImmutableSet.of("user", "password", "keyonly", "etc") ); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -218,7 +218,7 @@ public String getConnectURI() ImmutableSet.of("user", "password", "keyonly", "etc") ); - new MySQLFirehoseDatabaseConnector( + new MySQLInputSourceDatabaseConnector( connectorConfig, null, securityConfig, @@ -244,7 +244,7 @@ public String getConnectURI() 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 String getConnectURI() 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 String getConnectURI() 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 boolean isEnforceAllowedProperties() } }; - 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 d9880d7acf2f..9812b213f59f 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.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 @@ @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 boolean equals(Object o) 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 int hashCode() @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 9506eddd04bf..31777d0f4314 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.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.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 List getJacksonModules() 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 9b93f0102c45..0d67d72aa848 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 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 String getConnectURI() 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 String getConnectURI() @Test public void testEqualsAndHashcode() { - EqualsVerifier.forClass(PostgresqlFirehoseDatabaseConnector.class) + EqualsVerifier.forClass(PostgresqlInputSourceDatabaseConnector.class) .usingGetClass() .withNonnullFields("connectorConfig") .withIgnoredFields("dbi") @@ -95,7 +95,7 @@ public String getConnectURI() JdbcAccessSecurityConfig securityConfig = newSecurityConfigEnforcingAllowList(ImmutableSet.of()); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -115,7 +115,7 @@ public String getConnectURI() JdbcAccessSecurityConfig securityConfig = newSecurityConfigEnforcingAllowList(ImmutableSet.of("user")); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -138,7 +138,7 @@ public String getConnectURI() expectedException.expectMessage("is not in the allowed list"); expectedException.expect(IllegalArgumentException.class); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -160,7 +160,7 @@ public String getConnectURI() ImmutableSet.of("user", "password", "keyonly", "etc") ); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -183,7 +183,7 @@ public String getConnectURI() expectedException.expectMessage("is not in the allowed list"); expectedException.expect(IllegalArgumentException.class); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -206,7 +206,7 @@ public String getConnectURI() expectedException.expectMessage("is not in the allowed list"); expectedException.expect(IllegalArgumentException.class); - new PostgresqlFirehoseDatabaseConnector( + new PostgresqlInputSourceDatabaseConnector( connectorConfig, securityConfig ); @@ -239,7 +239,7 @@ public boolean isEnforceAllowedProperties() } }; - 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 687e26e9ff27..9c62e2969ca6 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.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.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 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 void testOverWindow() )) .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 43c2fb557a11..528d82bd9656 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 void testSleepFunction() .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 1e5da8af8ed6..40a67f1236e0 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 WindowedStorageAdapter apply(WindowedDataSegment segment) } ); - firehose = new IngestSegmentFirehose( + segmentReader = new SegmentReader( adapters, spec.getTransformSpec(), spec.getDimensions(), @@ -120,8 +146,8 @@ public WindowedStorageAdapter apply(WindowedDataSegment segment) @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 float getProgress() @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 void close() throws IOException 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 ed8b8c0bb093..6516b0a0e00d 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.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.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 @@ private void testIngestion( 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 @@ private void testIngestion( ); 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 44f24f879a22..042aa5e9550f 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.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 @@ /** */ @RunWith(Parameterized.class) -public class IngestSegmentFirehoseTest +public class DatasourceRecordReaderSegmentReaderTest { private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( ImmutableList.of( @@ -107,7 +108,7 @@ public static Collection constructorFeeder() 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 void testReadFromIndexAndWriteAnotherIndex() throws Exception ) { 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 void testReadFromIndexAndWriteAnotherIndex() throws Exception ); 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 void testReadFromIndexAndWriteAnotherIndex() throws Exception 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/guice/IndexingServiceFirehoseModule.java b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java deleted file mode 100644 index 8e4952210997..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java +++ /dev/null @@ -1,49 +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.common.collect.ImmutableList; -import com.google.inject.Binder; -import org.apache.druid.initialization.DruidModule; -import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; - -import java.util.List; - -public class IndexingServiceFirehoseModule implements DruidModule -{ - @Override - public List getJacksonModules() - { - return ImmutableList.of( - new SimpleModule("IndexingServiceFirehoseModule") - .registerSubtypes( - new NamedType(EventReceiverFirehoseFactory.class, "receiver") - ) - ); - } - - @Override - public void configure(Binder binder) - { - } -} 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 46de3064f033..9732d72eaf5e 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.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 669a30dc5de9..7190c38849f2 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.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/actions/CheckPointDataSourceMetadataAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CheckPointDataSourceMetadataAction.java index 276a9fe22708..e319dc242660 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CheckPointDataSourceMetadataAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CheckPointDataSourceMetadataAction.java @@ -121,12 +121,6 @@ public Boolean perform(Task task, TaskActionToolbox toolbox) ); } - @Override - public boolean isAudited() - { - return true; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java index bd0be9f74114..1d0059335ed1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java @@ -22,8 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.TaskStorage; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; @@ -37,45 +35,21 @@ public class LocalTaskActionClient implements TaskActionClient private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class); private final Task task; - private final TaskStorage storage; private final TaskActionToolbox toolbox; - private final TaskAuditLogConfig auditLogConfig; public LocalTaskActionClient( Task task, - TaskStorage storage, - TaskActionToolbox toolbox, - TaskAuditLogConfig auditLogConfig + TaskActionToolbox toolbox ) { this.task = task; - this.storage = storage; this.toolbox = toolbox; - this.auditLogConfig = auditLogConfig; } @Override public RetType submit(TaskAction taskAction) { log.debug("Performing action for task[%s]: %s", task.getId(), taskAction); - - if (auditLogConfig.isEnabled() && taskAction.isAudited()) { - // Add audit log - try { - final long auditLogStartTime = System.currentTimeMillis(); - storage.addAuditLog(task, taskAction); - emitTimerMetric("task/action/log/time", taskAction, System.currentTimeMillis() - auditLogStartTime); - } - catch (Exception e) { - final String actionClass = taskAction.getClass().getName(); - log.makeAlert(e, "Failed to record action in audit log") - .addData("task", task.getId()) - .addData("actionClass", actionClass) - .emit(); - throw new ISE(e, "Failed to record action [%s] in audit log", actionClass); - } - } - final long performStartTime = System.currentTimeMillis(); final RetType result = performAction(taskAction); emitTimerMetric("task/action/run/time", taskAction, System.currentTimeMillis() - performStartTime); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientFactory.java index 60fc01a2390d..c485b1660380 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClientFactory.java @@ -21,27 +21,22 @@ import com.google.inject.Inject; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.TaskStorage; /** */ public class LocalTaskActionClientFactory implements TaskActionClientFactory { - private final TaskStorage storage; private final TaskActionToolbox toolbox; - private final TaskAuditLogConfig auditLogConfig; @Inject - public LocalTaskActionClientFactory(TaskStorage storage, TaskActionToolbox toolbox, TaskAuditLogConfig auditLogConfig) + public LocalTaskActionClientFactory(TaskActionToolbox toolbox) { - this.storage = storage; this.toolbox = toolbox; - this.auditLogConfig = auditLogConfig; } @Override public TaskActionClient create(Task task) { - return new LocalTaskActionClient(task, storage, toolbox, auditLogConfig); + return new LocalTaskActionClient(task, toolbox); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockListAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockListAction.java index 22af74f26b54..c60c57407984 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockListAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockListAction.java @@ -39,12 +39,6 @@ public List perform(Task task, TaskActionToolbox toolbox) return toolbox.getTaskLockbox().findLocksForTask(task); } - @Override - public boolean isAudited() - { - return false; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockReleaseAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockReleaseAction.java index fa8e1d6bea24..32974117cc7b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockReleaseAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockReleaseAction.java @@ -56,12 +56,6 @@ public Void perform(Task task, TaskActionToolbox toolbox) return null; } - @Override - public boolean isAudited() - { - return false; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentsAsUnusedAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentsAsUnusedAction.java index 93cb75280fac..35a8ed3e35c0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentsAsUnusedAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentsAsUnusedAction.java @@ -67,9 +67,4 @@ public Integer perform(Task task, TaskActionToolbox toolbox) .markSegmentsAsUnusedWithinInterval(dataSource, interval); } - @Override - public boolean isAudited() - { - return true; - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/ResetDataSourceMetadataAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/ResetDataSourceMetadataAction.java index c853a00c58da..42259b381830 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/ResetDataSourceMetadataAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/ResetDataSourceMetadataAction.java @@ -64,12 +64,6 @@ public Boolean perform(Task task, TaskActionToolbox toolbox) return toolbox.getSupervisorManager().resetSupervisor(dataSource, resetMetadata); } - @Override - public boolean isAudited() - { - return true; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsByIdAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsByIdAction.java index 88d3703f4b09..a4ca90ac62ed 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsByIdAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsByIdAction.java @@ -74,12 +74,6 @@ public Set perform(Task task, TaskActionToolbox toolbox) .retrieveSegmentsById(dataSource, segmentIds); } - @Override - public boolean isAudited() - { - return false; - } - @Override public boolean equals(Object o) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java index 3f8d4725835a..fb58328b3d76 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java @@ -101,12 +101,6 @@ public List perform(Task task, TaskActionToolbox toolbox) .retrieveUnusedSegmentsForInterval(dataSource, interval, versions, limit, maxUsedStatusLastUpdatedTime); } - @Override - public boolean isAudited() - { - return false; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedFromSegmentIdsAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedFromSegmentIdsAction.java index 67f7ae6e1317..4f54131dee6f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedFromSegmentIdsAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedFromSegmentIdsAction.java @@ -73,12 +73,6 @@ public UpgradedFromSegmentsResponse perform(Task task, TaskActionToolbox toolbox ); } - @Override - public boolean isAudited() - { - return false; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedToSegmentIdsAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedToSegmentIdsAction.java index 412c9604d114..256a663a1a47 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedToSegmentIdsAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedToSegmentIdsAction.java @@ -78,12 +78,6 @@ public UpgradedToSegmentsResponse perform(Task task, TaskActionToolbox toolbox) ); } - @Override - public boolean isAudited() - { - return false; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsAction.java index a107795864ce..473976efd1ea 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsAction.java @@ -186,12 +186,6 @@ private Set retrieveUsedSegments(TaskActionToolbox toolbox) .retrieveUsedSegmentsForIntervals(dataSource, intervals, visibility); } - @Override - public boolean isAudited() - { - return false; - } - @Override public boolean equals(Object o) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java index d0308516e04b..902dad5dd879 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java @@ -386,12 +386,6 @@ private SegmentIdWithShardSpec tryAllocate( } } - @Override - public boolean isAudited() - { - return false; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java deleted file mode 100644 index 478e0b89d3d9..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java +++ /dev/null @@ -1,101 +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.actions; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.collect.ImmutableSet; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.segment.SegmentSchemaMapping; -import org.apache.druid.segment.SegmentUtils; -import org.apache.druid.timeline.DataSegment; - -import javax.annotation.Nullable; -import java.util.Set; - -/** - * Word of warning: Very large "segments" sets can cause oversized audit log entries, which is bad because it means - * that the task cannot actually complete. Callers should avoid this by avoiding inserting too many segments in the - * same action. - */ -public class SegmentInsertAction implements TaskAction> -{ - private final Set segments; - - @Nullable - private final SegmentSchemaMapping segmentSchemaMapping; - - @JsonCreator - public SegmentInsertAction( - @JsonProperty("segments") Set segments, - @JsonProperty("segmentSchemaMapping") @Nullable SegmentSchemaMapping segmentSchemaMapping - ) - { - this.segments = ImmutableSet.copyOf(segments); - this.segmentSchemaMapping = segmentSchemaMapping; - } - - @JsonProperty - public Set getSegments() - { - return segments; - } - - @JsonProperty - @Nullable - public SegmentSchemaMapping getSegmentSchemaMapping() - { - return segmentSchemaMapping; - } - - @Override - public TypeReference> getReturnTypeReference() - { - return new TypeReference>() - { - }; - } - - /** - * Behaves similarly to - * {@link org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator#commitSegments}, - * with startMetadata and endMetadata both null. - */ - @Override - public Set perform(Task task, TaskActionToolbox toolbox) - { - return SegmentTransactionalInsertAction.appendAction(segments, null, null, segmentSchemaMapping).perform(task, toolbox).getSegments(); - } - - @Override - public boolean isAudited() - { - return true; - } - - @Override - public String toString() - { - return "SegmentInsertAction{" + - "segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + - '}'; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java index 70c81225b2f5..e17af3847ca8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java @@ -119,12 +119,6 @@ public LockResult perform(Task task, TaskActionToolbox toolbox) } } - @Override - public boolean isAudited() - { - return false; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java index 7728574c0756..c5084eb61cda 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java @@ -102,12 +102,6 @@ public List perform(Task task, TaskActionToolbox toolbox) .collect(Collectors.toList()); } - @Override - public boolean isAudited() - { - return false; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java index cbad8bfcc5c2..b2a5ee5843be 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java @@ -99,12 +99,6 @@ public Void perform(Task task, TaskActionToolbox toolbox) return null; } - @Override - public boolean isAudited() - { - return true; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java index 2856f161e0e6..326907c8f0a5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java @@ -101,12 +101,6 @@ public Void perform(Task task, TaskActionToolbox toolbox) return null; } - @Override - public boolean isAudited() - { - return true; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index 4871e65e162c..2326176885b3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -44,7 +44,6 @@ import java.util.stream.Collectors; /** - * * Append segments to metadata storage. The segment versions must all be less than or equal to a lock held by * your task for the segment intervals. * @@ -209,12 +208,6 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) return retVal; } - @Override - public boolean isAudited() - { - return true; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java index c2f542b096e1..e8dd472cf31d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java @@ -304,12 +304,6 @@ private static Map> groupSegmentsByIntervalAndSort(S return segmentsMap; } - @Override - public boolean isAudited() - { - return true; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index df188ac81533..572f4aa3f284 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -187,12 +187,6 @@ private void registerUpgradedPendingSegmentsOnSupervisor( ); } - @Override - public boolean isAudited() - { - return true; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SurrogateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SurrogateAction.java index 700220bde3a7..2dbc383c3259 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SurrogateAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SurrogateAction.java @@ -73,12 +73,6 @@ public ReturnType perform(Task task, TaskActionToolbox toolbox) } } - @Override - public boolean isAudited() - { - return taskAction.isAudited(); - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java index 973a83ecee43..8b1d863530a9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java @@ -34,7 +34,6 @@ @JsonSubTypes.Type(name = "segmentLockAcquire", value = SegmentLockAcquireAction.class), @JsonSubTypes.Type(name = "lockList", value = LockListAction.class), @JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class), - @JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class), @JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class), @@ -61,8 +60,6 @@ public interface TaskAction RetType perform(Task task, TaskActionToolbox toolbox); - boolean isAudited(); - default boolean canPerformAsync(Task task, TaskActionToolbox toolbox) { return false; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java index 305a0d7b2843..42f4e40d0e64 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java @@ -97,12 +97,6 @@ public TaskLock perform(Task task, TaskActionToolbox toolbox) } } - @Override - public boolean isAudited() - { - return false; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java index 07bcfa5c1011..b2dc6e3d5e62 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java @@ -81,12 +81,6 @@ public TaskLock perform(Task task, TaskActionToolbox toolbox) return result.getTaskLock(); } - @Override - public boolean isAudited() - { - return false; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java index dce9f00c35b4..2c7c265a7f40 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateLocationAction.java @@ -63,12 +63,6 @@ public Void perform(Task task, TaskActionToolbox toolbox) return null; } - @Override - public boolean isAudited() - { - return true; - } - @Override public String toString() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateStatusAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateStatusAction.java index 2ff8375b7f68..d02020acb9d0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateStatusAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpdateStatusAction.java @@ -85,12 +85,6 @@ public Void perform(Task task, TaskActionToolbox toolbox) return null; } - @Override - public boolean isAudited() - { - return true; - } - @Override public String toString() { 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 db48d6f07f7d..b8025e415a89 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.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 enum BatchProcessingMode @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 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 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 @@ private TaskConfig( Period directoryLockTimeout, List shuffleDataLocations, boolean ignoreTimestampSpecForDruidInputSource, - boolean batchMemoryMappedIndex, - BatchProcessingMode batchProcessingMode, boolean storeEmptyColumns, boolean encapsulatedTask, long tmpStorageBytesPerTask @@ -223,8 +183,6 @@ private 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 boolean isIgnoreTimestampSpecForDruidInputSource() 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 TaskConfig withBaseTaskDir(File baseTaskDir) directoryLockTimeout, shuffleDataLocations, ignoreTimestampSpecForDruidInputSource, - batchMemoryMappedIndex, - batchProcessingMode, storeEmptyColumns, encapsulatedTask, tmpStorageBytesPerTask @@ -387,8 +327,6 @@ public TaskConfig withTmpStorageBytesPerTask(long tmpStorageBytesPerTask) 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 087464b48acd..6af4402c35a6 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 @@ 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 static Appenderator newAppenderator( 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/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index bc9d5aca2b65..f1f96d8ca34d 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.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 dc6c07b6b83c..a8f60cca5729 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 @@ 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.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.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.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 String getTaskAllocatorId() @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 IndexTuningConfig getTuningConfig() @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 static class IndexIOConfig implements BatchIOConfig @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 f2eacb8c1c6c..30761b674e54 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 @@ -157,7 +157,6 @@ private static ParallelIndexIOConfig createIoConfig( } 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 4fb4bb7852c0..9b882e2e8d2b 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 @@ default int getPriority() * 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 @@ default Set getInputSourceResources() throws UOE )); } - 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/Tasks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java index b357125dc4df..a8f268a67f13 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java @@ -21,16 +21,9 @@ import org.apache.curator.shaded.com.google.common.base.Verify; import org.apache.druid.indexing.common.TaskLockType; -import org.apache.druid.java.util.common.JodaUtils; -import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.duty.CompactSegments; -import org.joda.time.Interval; -import java.util.ArrayList; -import java.util.List; -import java.util.SortedSet; -import java.util.TreeSet; import java.util.concurrent.TimeUnit; public class Tasks @@ -63,44 +56,19 @@ public class Tasks * Context flag denoting if maximum possible values should be used to estimate * on-heap memory usage while indexing. Refer to OnHeapIncrementalIndex for * more details. - * + *

* The value of this flag is true by default which corresponds to the old method * of estimation. */ public static final String USE_MAX_MEMORY_ESTIMATES = "useMaxMemoryEstimates"; /** - * This context is used in compaction. When it is set in the context, the segments created by the task - * will fill 'lastCompactionState' in its metadata. This will be used to track what segments are compacted or not. - * See {@link org.apache.druid.timeline.DataSegment} and {@link - * org.apache.druid.server.coordinator.compact.NewestSegmentFirstIterator} for more details. + * Context flag to denote if segments published to metadata by a task should + * have the {@code lastCompactionState} field set. */ public static final String STORE_COMPACTION_STATE_KEY = "storeCompactionState"; static { Verify.verify(STORE_COMPACTION_STATE_KEY.equals(CompactSegments.STORE_COMPACTION_STATE_KEY)); } - - public static SortedSet computeCondensedIntervals(SortedSet intervals) - { - final SortedSet condensedIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); - List toBeAccumulated = new ArrayList<>(); - for (Interval interval : intervals) { - if (toBeAccumulated.size() == 0) { - toBeAccumulated.add(interval); - } else { - if (toBeAccumulated.get(toBeAccumulated.size() - 1).abuts(interval)) { - toBeAccumulated.add(interval); - } else { - condensedIntervals.add(JodaUtils.umbrellaInterval(toBeAccumulated)); - toBeAccumulated.clear(); - toBeAccumulated.add(interval); - } - } - } - if (toBeAccumulated.size() > 0) { - condensedIntervals.add(JodaUtils.umbrellaInterval(toBeAccumulated)); - } - return condensedIntervals; - } } 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 286325186b25..e6097678b5ae 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 @@ final int estimateTotalNumSubTasks() throws IOException final SubTaskSpec 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 1daa99994274..27a242885c36 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 String getType() @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 c86446d91303..ff10e556de8f 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 @@ 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 4ca0f1ff80d7..6039a62bc984 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.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 String getType() @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 TaskStatus runTask(TaskToolbox toolbox) throws Exception } 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 6724a2ebb90c..d75304f38c08 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 String getType() @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 cb361a204ea3..4bb395420b3f 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 String getType() @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 383fc7afb2e0..4c224e396d18 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 String getType() @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/PartialRangeSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java index cf7ae15a9a59..933df9ee7785 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 String getType() @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 c53557c66558..620d593467ad 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.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 ce6aee98af35..a2a29b3cdd3e 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 @@ 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 @@ int estimateTotalNumSubTasks() throws IOException @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 @@ SubTaskSpec newTaskSpec(InputSplit split) 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 cd05e50ff1c8..8d49a7f3dbe7 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.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 String getType() @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 89afdbb64d05..3e3d653e3c32 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 dd1998645b34..84c17f1a8fa0 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.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 d048cf4d8b91..42618556d003 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.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 b55510a7e219..8232aba04460 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 000000000000..f99189d35894 --- /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/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index e84976db86c4..5109abe2377d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -327,24 +327,6 @@ public void removeTasksOlderThan(final long timestamp) } } - @Deprecated - @Override - public void addAuditLog(Task task, TaskAction taskAction) - { - synchronized (taskActions) { - taskActions.put(task.getId(), taskAction); - } - } - - @Deprecated - @Override - public List getAuditLogs(String taskid) - { - synchronized (taskActions) { - return ImmutableList.copyOf(taskActions.get(taskid)); - } - } - private static class TaskStuff { final Task task; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java index 6d23fcec327e..c3e498aa18f7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java @@ -25,6 +25,7 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.metadata.TaskLookup; import org.joda.time.Interval; import java.util.Comparator; @@ -32,16 +33,16 @@ public class IndexerMetadataStorageAdapter { - private final TaskStorageQueryAdapter taskStorageQueryAdapter; + private final TaskStorage taskStorage; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; @Inject public IndexerMetadataStorageAdapter( - TaskStorageQueryAdapter taskStorageQueryAdapter, + TaskStorage taskStorage, IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator ) { - this.taskStorageQueryAdapter = taskStorageQueryAdapter; + this.taskStorage = taskStorage; this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; } @@ -49,8 +50,8 @@ public int deletePendingSegments(String dataSource, Interval deleteInterval) { // Find the earliest active task created for the specified datasource; if one exists, // check if its interval overlaps with the delete interval. - final Optional> earliestActiveTaskOptional = taskStorageQueryAdapter - .getActiveTaskInfo(dataSource) + final Optional> earliestActiveTaskOptional = taskStorage + .getTaskInfos(TaskLookup.activeTasksOnly(), dataSource) .stream() .min(Comparator.comparing(TaskInfo::getCreatedTime)); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 15730d48bb1f..3608f0238992 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -75,14 +75,6 @@ public TypeReference getStatusType() }; } - @Override - public TypeReference getLogType() - { - return new TypeReference() - { - }; - } - @Override public TypeReference getLockType() { @@ -319,24 +311,6 @@ public List getLocks(String taskid) ); } - @Deprecated - @Override - public void addAuditLog(final Task task, final TaskAction taskAction) - { - Preconditions.checkNotNull(taskAction, "taskAction"); - - log.info("Logging action for task[%s]: %s", task.getId(), taskAction); - - handler.addLog(task.getId(), taskAction); - } - - @Deprecated - @Override - public List getAuditLogs(final String taskId) - { - return handler.getLogs(taskId); - } - private Map getLocksWithIds(final String taskid) { return handler.getLocks(taskid); 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 5103f9bd87eb..1de953525045 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(); - } - } - }; - } - - /** - * Starts waiting for leadership. Should only be called once throughout the life of the program. - */ - @LifecycleStart - public void start() - { - giant.lock(); - - try { - overlordLeaderSelector.registerListener(leadershipListener); - } - finally { - giant.unlock(); - } - } - - /** - * Stops forever (not just this particular leadership session). Should only be called once throughout the life of - * the program. - */ - @LifecycleStop - public void stop() - { - giant.lock(); - - try { - gracefulStopLeaderLifecycle(); - overlordLeaderSelector.unregisterListener(); - } - finally { - giant.unlock(); - } + this.supervisorManager = supervisorManager; } - /** - * Returns true if it's the leader and all its services have been initialized. - */ - public boolean isLeader() + public void becomeLeader(TaskRunner taskRunner, TaskQueue taskQueue) { - return overlordLeaderSelector.isLeader() && initialized; + this.taskRunner = taskRunner; + this.taskQueue = taskQueue; + isLeader.set(true); } - public String getCurrentLeader() + public void stopBeingLeader() { - return overlordLeaderSelector.getCurrentLeader(); + isLeader.set(false); + this.taskQueue = null; + this.taskRunner = null; } - public Optional getRedirectLocation() + private boolean isLeader() { - 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 CoordinatorRunStats getStats() } } - 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 new file mode 100644 index 000000000000..f5351d7c6e51 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java @@ -0,0 +1,384 @@ +/* + * 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.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.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. + * This class may serve information that is cached in memory in {@link TaskQueue} + * or {@link TaskLockbox}. If not present in memory, then the underlying + * {@link TaskStorage} is queried. + */ +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, + ProvisioningStrategy provisioningStrategy, + JacksonConfigManager configManager + ) + { + this.storage = storage; + this.taskLockbox = taskLockbox; + this.taskMaster = taskMaster; + this.configManager = configManager; + this.provisioningStrategy = provisioningStrategy; + } + + /** + * @param lockFilterPolicies Requests for conflicing lock intervals for various datasources + * @return Map from datasource to intervals locked by tasks that have a conflicting lock type that cannot be revoked + */ + public Map> getLockedIntervals(List lockFilterPolicies) + { + return taskLockbox.getLockedIntervals(lockFilterPolicies); + } + + /** + * 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. + */ + public Map> getLockedIntervals(Map minTaskPriority) + { + return taskLockbox.getLockedIntervals(minTaskPriority); + } + + public List> getActiveTaskInfo(@Nullable String dataSource) + { + return storage.getTaskInfos(TaskLookup.activeTasksOnly(), dataSource); + } + + private List getTaskStatusPlusList( + Map taskLookups, + @Nullable String dataSource + ) + { + return storage.getTaskStatusPlusList(taskLookups, dataSource); + } + + public Optional getTask(final String taskId) + { + final Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + Optional activeTask = taskQueue.get().getActiveTask(taskId); + if (activeTask.isPresent()) { + return activeTask; + } + } + return storage.getTask(taskId); + } + + public Optional getTaskStatus(final String taskId) + { + final Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + return taskQueue.get().getTaskStatus(taskId); + } else { + return storage.getStatus(taskId); + } + } + + @Nullable + public TaskInfo getTaskInfo(String taskId) + { + 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/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java index 235c763e1f71..b231b3f37c28 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java @@ -24,7 +24,6 @@ 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.actions.TaskAction; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TaskLookup.TaskLookupType; @@ -110,26 +109,6 @@ public interface TaskStorage @Nullable TaskInfo getTaskInfo(String taskId); - /** - * Add an action taken by a task to the audit log. - * - * @param task task to record action for - * @param taskAction task action to record - * @param task action return type - */ - @Deprecated - void addAuditLog(Task task, TaskAction taskAction); - - /** - * Returns all actions taken by a task. - * - * @param taskid task ID - * - * @return list of task actions - */ - @Deprecated - List getAuditLogs(String taskid); - /** * Returns a list of currently running or pending tasks as stored in the storage facility. No particular order * is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java deleted file mode 100644 index ba2ca3c7066a..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ /dev/null @@ -1,152 +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.overlord; - -import com.google.common.base.Optional; -import com.google.inject.Inject; -import org.apache.druid.indexer.TaskInfo; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.TaskStatusPlus; -import org.apache.druid.indexing.common.actions.SegmentInsertAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.actions.TaskAction; -import org.apache.druid.indexing.common.task.Task; -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.TaskLookupType; -import org.apache.druid.timeline.DataSegment; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** - * Wraps a {@link TaskStorage}, providing a useful collection of read-only methods. - */ -public class TaskStorageQueryAdapter -{ - private final TaskStorage storage; - private final TaskLockbox taskLockbox; - private final Optional taskQueue; - - @Inject - public TaskStorageQueryAdapter(TaskStorage storage, TaskLockbox taskLockbox, TaskMaster taskMaster) - { - this.storage = storage; - this.taskLockbox = taskLockbox; - this.taskQueue = taskMaster.getTaskQueue(); - } - - public List getActiveTasks() - { - return storage.getActiveTasks(); - } - - /** - * @param lockFilterPolicies Requests for conflicing lock intervals for various datasources - * @return Map from datasource to intervals locked by tasks that have a conflicting lock type that cannot be revoked - */ - public Map> getLockedIntervals(List lockFilterPolicies) - { - return taskLockbox.getLockedIntervals(lockFilterPolicies); - } - - /** - * 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. - */ - public Map> getLockedIntervals(Map minTaskPriority) - { - return taskLockbox.getLockedIntervals(minTaskPriority); - } - - public List> getActiveTaskInfo(@Nullable String dataSource) - { - return storage.getTaskInfos( - ActiveTaskLookup.getInstance(), - dataSource - ); - } - - public List getTaskStatusPlusList( - Map taskLookups, - @Nullable String dataSource - ) - { - return storage.getTaskStatusPlusList(taskLookups, dataSource); - } - - public Optional getTask(final String taskid) - { - if (taskQueue.isPresent()) { - Optional activeTask = taskQueue.get().getActiveTask(taskid); - if (activeTask.isPresent()) { - return activeTask; - } - } - return storage.getTask(taskid); - } - - public Optional getStatus(final String taskid) - { - return storage.getStatus(taskid); - } - - @Nullable - public TaskInfo getTaskInfo(String taskId) - { - return storage.getTaskInfo(taskId); - } - - /** - * Returns all segments created by this task. - * - * This method is useful when you want to figure out all of the things a single task spawned. It does pose issues - * with the result set perhaps growing boundlessly and we do not do anything to protect against that. Use at your - * own risk and know that at some point, we might adjust this to actually enforce some sort of limits. - * - * @param taskid task ID - * @return set of segments created by the specified task - */ - @Deprecated - public Set getInsertedSegments(final String taskid) - { - final Set segments = new HashSet<>(); - for (final TaskAction action : storage.getAuditLogs(taskid)) { - if (action instanceof SegmentInsertAction) { - segments.addAll(((SegmentInsertAction) action).getSegments()); - } else if (action instanceof SegmentTransactionalInsertAction) { - segments.addAll(((SegmentTransactionalInsertAction) action).getSegments()); - } - } - return segments; - } -} 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 4e332f599df4..41e1ef8b7c9a 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 @@ 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 56123a561f6b..54ada7cb2b43 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.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.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.TaskQueue; +import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; -import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter; 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,8 +77,7 @@ import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceType; import org.apache.druid.tasklogs.TaskLogStreamer; -import org.apache.druid.timeline.DataSegment; -import org.joda.time.Duration; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -105,17 +97,12 @@ 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; /** * @@ -126,63 +113,44 @@ public class OverlordResource private static final Logger log = new Logger(OverlordResource.class); private final TaskMaster taskMaster; - private final TaskStorageQueryAdapter taskStorageQueryAdapter; + private final DruidOverlord overlord; + private final TaskQueryTool taskQueryTool; private final IndexerMetadataStorageAdapter indexerMetadataStorageAdapter; private final TaskLogStreamer taskLogStreamer; private final JacksonConfigManager configManager; 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, - TaskStorageQueryAdapter taskStorageQueryAdapter, + TaskQueryTool taskQueryTool, IndexerMetadataStorageAdapter indexerMetadataStorageAdapter, TaskLogStreamer taskLogStreamer, JacksonConfigManager configManager, AuditManager auditManager, AuthorizerMapper authorizerMapper, WorkerTaskRunnerQueryAdapter workerTaskRunnerQueryAdapter, - ProvisioningStrategy provisioningStrategy, AuthConfig authConfig ) { + this.overlord = overlord; this.taskMaster = taskMaster; - this.taskStorageQueryAdapter = taskStorageQueryAdapter; + this.taskQueryTool = taskQueryTool; this.indexerMetadataStorageAdapter = indexerMetadataStorageAdapter; this.taskLogStreamer = taskLogStreamer; this.configManager = configManager; this.auditManager = auditManager; this.authorizerMapper = authorizerMapper; this.workerTaskRunnerQueryAdapter = workerTaskRunnerQueryAdapter; - this.provisioningStrategy = provisioningStrategy; this.authConfig = authConfig; } @@ -253,7 +221,7 @@ public Response taskPost( @Produces(MediaType.APPLICATION_JSON) public Response getLeader() { - return Response.ok(taskMaster.getCurrentLeader()).build(); + return Response.ok(overlord.getCurrentLeader()).build(); } /** @@ -264,7 +232,7 @@ public Response getLeader() @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(); @@ -285,7 +253,7 @@ public Response getDatasourceLockedIntervals(Map minTaskPriorit } // Build the response - return Response.ok(taskStorageQueryAdapter.getLockedIntervals(minTaskPriority)).build(); + return Response.ok(taskQueryTool.getLockedIntervals(minTaskPriority)).build(); } @POST @@ -299,7 +267,7 @@ public Response getDatasourceLockedIntervalsV2(List lockFilter } // Build the response - return Response.ok(taskStorageQueryAdapter.getLockedIntervals(lockFilterPolicies)).build(); + return Response.ok(taskQueryTool.getLockedIntervals(lockFilterPolicies)).build(); } @GET @@ -310,7 +278,7 @@ public Response getTaskPayload(@PathParam("taskid") String taskid) { final TaskPayloadResponse response = new TaskPayloadResponse( taskid, - taskStorageQueryAdapter.getTask(taskid).orNull() + taskQueryTool.getTask(taskid).orNull() ); final Response.Status status = response.getPayload() == null @@ -326,7 +294,7 @@ public Response getTaskPayload(@PathParam("taskid") String taskid) @ResourceFilters(TaskResourceFilter.class) public Response getTaskStatus(@PathParam("taskid") String taskid) { - final TaskInfo taskInfo = taskStorageQueryAdapter.getTaskInfo(taskid); + final TaskInfo taskInfo = taskQueryTool.getTaskInfo(taskid); TaskStatusResponse response = null; if (taskInfo != null) { @@ -374,9 +342,7 @@ public Response getTaskStatus(@PathParam("taskid") String taskid) 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() ) @@ -400,8 +366,12 @@ public Response getTaskStatus(@PathParam("taskid") String taskid) @ResourceFilters(TaskResourceFilter.class) public Response getTaskSegments(@PathParam("taskid") String taskid) { - final Set segments = taskStorageQueryAdapter.getInsertedSegments(taskid); - return Response.ok().entity(segments).build(); + final String errorMsg = + "Segment IDs committed by a task action are not persisted anymore." + + " Use the metric 'segment/added/bytes' to identify the segments created by a task."; + return Response.status(Status.NOT_FOUND) + .entity(Collections.singletonMap("error", errorMsg)) + .build(); } @POST @@ -412,14 +382,9 @@ public Response doShutdown(@PathParam("taskid") final String taskid) { 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(); } ); } @@ -432,20 +397,15 @@ public Response shutdownTasksForDataSource(@PathParam("dataSource") final String { return asLeaderWith( taskMaster.getTaskQueue(), - new Function() - { - @Override - public Response apply(TaskQueue taskQueue) - { - final List> tasks = taskStorageQueryAdapter.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(); } } ); @@ -457,19 +417,13 @@ public Response apply(TaskQueue taskQueue) @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 = taskStorageQueryAdapter.getStatus(taskId); - } + final Optional optional = taskQueryTool.getTaskStatus(taskId); if (optional.isPresent()) { result.put(taskId, optional.get()); } @@ -484,11 +438,7 @@ public Response getMultipleTaskStatuses(Set taskIds) @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(); } /** @@ -500,49 +450,11 @@ public Response getWorkerConfig() @ResourceFilters(ConfigResourceFilter.class) public Response getTotalWorkerCapacity() { - // Calculate current cluster capacity - Optional taskRunnerOptional = taskMaster.getTaskRunner(); - if (!taskRunnerOptional.isPresent()) { - // Cannot serve call as not leader - 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; - } + if (overlord.isLeader()) { + return Response.ok(taskQueryTool.getTotalWorkerCapacity()).build(); } 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.status(Response.Status.SERVICE_UNAVAILABLE).build(); } - return Response.ok(new TotalWorkerCapacityResponse(currentCapacity, maximumCapacity, usedCapacity)).build(); } // default value is used for backwards compatibility @@ -690,9 +602,13 @@ public Response getTasks( //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(); } } @@ -722,8 +638,7 @@ public Response getTasks( taskMaster.getTaskRunner(), taskRunner -> { final List authorizedList = securedTaskStatusPlus( - getTaskStatusPlusList( - taskRunner, + taskQueryTool.getTaskStatusPlusList( TaskStateLookup.fromString(state), dataSource, createdTimeInterval, @@ -738,180 +653,6 @@ public Response getTasks( ); } - 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 = taskStorageQueryAdapter.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) @@ -936,7 +677,7 @@ public Response killPendingSegments( 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(); @@ -967,23 +708,17 @@ public Response getWorkers() { 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/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 e2b776fa4fa5..0236afdd0572 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/http/security/TaskResourceFilter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilter.java index ad1f526eda64..a9f66ce30e72 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilter.java @@ -26,7 +26,7 @@ import com.sun.jersey.spi.container.ContainerRequest; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter; +import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.http.security.AbstractResourceFilter; import org.apache.druid.server.security.Access; @@ -49,16 +49,16 @@ */ public class TaskResourceFilter extends AbstractResourceFilter { - private final TaskStorageQueryAdapter taskStorageQueryAdapter; + private final TaskQueryTool taskQueryTool; @Inject public TaskResourceFilter( - TaskStorageQueryAdapter taskStorageQueryAdapter, + TaskQueryTool taskQueryTool, AuthorizerMapper authorizerMapper ) { super(authorizerMapper); - this.taskStorageQueryAdapter = taskStorageQueryAdapter; + this.taskQueryTool = taskQueryTool; } @Override @@ -76,7 +76,7 @@ public ContainerRequest filter(ContainerRequest request) IdUtils.validateId("taskId", taskId); - Optional taskOptional = taskStorageQueryAdapter.getTask(taskId); + Optional taskOptional = taskQueryTool.getTask(taskId); if (!taskOptional.isPresent()) { throw new WebApplicationException( Response.status(Response.Status.NOT_FOUND) 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 a84cb4e1979c..5018795cbb4f 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 ee4eed572dff..41cd084cd960 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.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 fad7b923ca97..d347fd815038 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.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 e1ac9482436b..78cbb88c3cf4 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.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 void setUp() throws IOException 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 1572373151bf..000000000000 --- 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 c69ac562821b..f6732f68a6c4 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 TestIndexTask( 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 7dda3b8ff62c..ae5ba44f7f09 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.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.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 void setupModule(SetupContext context) } ); 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/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java deleted file mode 100644 index c8999c2f5d46..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java +++ /dev/null @@ -1,154 +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.actions; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import org.apache.druid.indexing.common.TaskLockType; -import org.apache.druid.indexing.common.task.NoopTask; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.CriticalAction; -import org.apache.druid.indexing.overlord.LockResult; -import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.indexing.overlord.TimeChunkLockRequest; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.LinearShardSpec; -import org.assertj.core.api.Assertions; -import org.hamcrest.CoreMatchers; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -import java.util.Collections; -import java.util.Set; - -public class SegmentInsertActionTest -{ - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Rule - public TaskActionTestKit actionTestKit = new TaskActionTestKit(); - - private static final String DATA_SOURCE = "none"; - private static final Interval INTERVAL = Intervals.of("2020/2020T01"); - private static final String PARTY_YEAR = "1999"; - private static final String THE_DISTANT_FUTURE = "3000"; - - private static final DataSegment SEGMENT1 = new DataSegment( - DATA_SOURCE, - INTERVAL, - PARTY_YEAR, - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - new LinearShardSpec(0), - 9, - 1024 - ); - - private static final DataSegment SEGMENT2 = new DataSegment( - DATA_SOURCE, - INTERVAL, - PARTY_YEAR, - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - new LinearShardSpec(1), - 9, - 1024 - ); - - private static final DataSegment SEGMENT3 = new DataSegment( - DATA_SOURCE, - INTERVAL, - THE_DISTANT_FUTURE, - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - new LinearShardSpec(1), - 9, - 1024 - ); - - private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interval interval, long timeoutMs) - throws InterruptedException - { - return actionTestKit.getTaskLockbox().lock(task, new TimeChunkLockRequest(lockType, task, interval, null), timeoutMs); - } - - @Test - public void testSimple() throws Exception - { - final Task task = NoopTask.create(); - final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT1, SEGMENT2), null); - actionTestKit.getTaskLockbox().add(task); - acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); - actionTestKit.getTaskLockbox().doInCriticalSection( - task, - Collections.singleton(INTERVAL), - CriticalAction.builder() - .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) - .onInvalidLocks( - () -> { - Assert.fail(); - return null; - } - ) - .build() - ); - - Assertions.assertThat( - actionTestKit.getMetadataStorageCoordinator() - .retrieveUsedSegmentsForInterval(DATA_SOURCE, INTERVAL, Segments.ONLY_VISIBLE) - ).containsExactlyInAnyOrder(SEGMENT1, SEGMENT2); - } - - @Test - public void testFailBadVersion() throws Exception - { - final Task task = NoopTask.create(); - final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT3), null); - actionTestKit.getTaskLockbox().add(task); - acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); - - thrown.expect(IllegalStateException.class); - thrown.expectMessage(CoreMatchers.containsString("are not covered by locks")); - final Set segments = actionTestKit.getTaskLockbox().doInCriticalSection( - task, - Collections.singleton(INTERVAL), - CriticalAction.>builder() - .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) - .onInvalidLocks( - () -> { - Assert.fail(); - return null; - } - ) - .build() - ); - - Assert.assertEquals(ImmutableSet.of(SEGMENT3), segments); - } -} 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 af920ebbeb73..1213b5525146 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 TaskConfigBuilder setIgnoreTimestampSpecForDruidInputSource(boolean ignor 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 TaskConfig build() 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 f789c1e641f7..000000000000 --- 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 cd942a9a7668..55898357d663 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.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 188ea3cdd071..1b742971eb95 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.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 @@ protected TaskToolbox createTaskToolbox(Task task, TaskActionClient actionClient 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 0c274a019668..7ebde700bee2 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.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 List getLocations() ); 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 25dc84d98b79..f9849b1483df 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.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; @@ -1769,7 +1769,6 @@ public void cleanup(DataSegment segment) }; 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/CountingLocalTaskActionClientForTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CountingLocalTaskActionClientForTest.java index 63380e84fd9e..cad2feb0deb1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CountingLocalTaskActionClientForTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CountingLocalTaskActionClientForTest.java @@ -24,7 +24,6 @@ import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionToolbox; -import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; import org.apache.druid.indexing.overlord.TaskStorage; import java.util.concurrent.ConcurrentHashMap; @@ -42,7 +41,7 @@ public CountingLocalTaskActionClientForTest( TaskActionToolbox toolbox ) { - delegate = new LocalTaskActionClient(task, storage, toolbox, new TaskAuditLogConfig(false)); + delegate = new LocalTaskActionClient(task, toolbox); } @Override 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 ceca135c810d..130258e85d4c 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 TaskStatus runTask(TaskToolbox toolbox) 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 7555c6b1e378..ab953ba954ac 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 void testParserAndInputFormat() null ), new IndexIOConfig( - null, new NoopInputSource(), new NoopInputFormat(), null, @@ -79,7 +78,6 @@ public void testParserAndInputSource() 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 40be38445857..3ec5b8d0aa8a 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.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 void testCorrectInputSourceResources() null ), new IndexIOConfig( - null, new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, false, @@ -295,7 +294,6 @@ public void testIngestNullOnlyColumns() throws Exception null ), new IndexIOConfig( - null, new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, false, @@ -358,7 +356,6 @@ public void testIngestNullOnlyColumns_storeEmptyColumnsOff_shouldNotStoreEmptyCo null ), new IndexIOConfig( - null, new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, false, @@ -2727,7 +2724,6 @@ private static IndexIngestionSpec createIngestionSpec( transformSpec ), new IndexIOConfig( - null, new LocalInputSource(baseDir, "druid*"), inputFormat, appendToExisting, @@ -2755,7 +2751,6 @@ private static IndexIngestionSpec createIngestionSpec( 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 d6687efbf34e..67a0c518f57c 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.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; @@ -43,7 +41,6 @@ import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.actions.SegmentInsertAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionClient; @@ -86,7 +83,7 @@ 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; @@ -301,8 +298,7 @@ public IndexMergerV9Factory getIndexMergerV9Factory() } /** - * 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) { @@ -353,7 +349,8 @@ public TaskActionClient create(Task task) public class TestLocalTaskActionClient extends CountingLocalTaskActionClientForTest { private final Set publishedSegments = new HashSet<>(); - private SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + private final SegmentSchemaMapping segmentSchemaMapping + = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); private TestLocalTaskActionClient(Task task) { @@ -365,11 +362,9 @@ public RetType submit(TaskAction taskAction) { final RetType result = super.submit(taskAction); if (taskAction instanceof SegmentTransactionalInsertAction) { - publishedSegments.addAll(((SegmentTransactionalInsertAction) taskAction).getSegments()); - segmentSchemaMapping.merge(((SegmentTransactionalInsertAction) taskAction).getSegmentSchemaMapping()); - } else if (taskAction instanceof SegmentInsertAction) { - publishedSegments.addAll(((SegmentInsertAction) taskAction).getSegments()); - segmentSchemaMapping.merge(((SegmentInsertAction) taskAction).getSegmentSchemaMapping()); + SegmentTransactionalInsertAction insertAction = (SegmentTransactionalInsertAction) taskAction; + publishedSegments.addAll(insertAction.getSegments()); + segmentSchemaMapping.merge(insertAction.getSegmentSchemaMapping()); } return result; } @@ -448,7 +443,6 @@ public ListenableFuture run(Task task) ); final TaskConfig config = new TaskConfigBuilder() - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); centralizedDatasourceSchemaConfig.setEnabled(true); 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 9cf6c0102fbe..e6ea0e1329ae 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 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 void testIndexTaskSerde() throws Exception ), 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 void testIndexTaskwithResourceSerde() throws Exception ), 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 void testIndexTaskwithResourceSerde() throws Exception 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 void testHadoopIndexTaskSerde() throws Exception 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/TasksTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TasksTest.java deleted file mode 100644 index 248df1db44d3..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TasksTest.java +++ /dev/null @@ -1,94 +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 org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.guava.Comparators; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Iterator; -import java.util.SortedSet; -import java.util.TreeSet; - -public class TasksTest -{ - - @Test - public void testComputeCondensedIntervals() - { - final SortedSet inputIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); - for (int m = 1; m < 13; m++) { - for (int d = 1; d < 10; d++) { - inputIntervals.add(getInterval(m, d, m, d + 1)); - } - - for (int d = 12; d < 20; d++) { - inputIntervals.add(getInterval(m, d, m, d + 1)); - } - - inputIntervals.add(getInterval(m, 22, m, 23)); - - for (int d = 25; d < 28; d++) { - inputIntervals.add(getInterval(m, d, m, d + 1)); - } - - if (m == 1 || m == 3 || m == 5 || m == 7 || m == 8 || m == 10) { - inputIntervals.add(getInterval(m, 31, m + 1, 1)); - } - } - - inputIntervals.add(Intervals.of("2017-12-31/2018-01-01")); - - final SortedSet condensedIntervals = Tasks.computeCondensedIntervals(inputIntervals); - final Iterator condensedIntervalIterator = condensedIntervals.iterator(); - Assert.assertTrue(condensedIntervalIterator.hasNext()); - - Interval condensedInterval = condensedIntervalIterator.next(); - final SortedSet checkedIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); - for (Interval inputInterval : inputIntervals) { - if (!condensedInterval.contains(inputInterval)) { - if (condensedIntervalIterator.hasNext()) { - condensedInterval = condensedIntervalIterator.next(); - Assert.assertTrue(condensedInterval.contains(inputInterval)); - } - } - checkedIntervals.add(inputInterval); - } - - Assert.assertFalse(condensedIntervalIterator.hasNext()); - Assert.assertEquals(inputIntervals, checkedIntervals); - } - - private static Interval getInterval(int startMonth, int startDay, int endMonth, int endDay) - { - return Intervals.of( - StringUtils.format( - "2017-%02d-%02d/2017-%02d-%02d", - startMonth, - startDay, - endMonth, - endDay - ) - ); - } -} 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 e61863291264..417195326fb4 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 Appenderator createRealtimeAppenderatorForTask( } @Override - public Appenderator createOpenSegmentsOfflineAppenderatorForTask( + public Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, @@ -116,71 +116,7 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( 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 be322ad06d38..44b8284f407d 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 @@ protected ParallelIndexSupervisorTask createTask( if (useInputFormatApi) { Preconditions.checkArgument(parseSpec == null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDirectory, filter), inputFormat, appendToExisting, @@ -239,7 +235,6 @@ protected ParallelIndexSupervisorTask createTask( } else { Preconditions.checkArgument(inputFormat == null && parseSpec != null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDirectory, filter), createInputFormatFromParseSpec(parseSpec), appendToExisting, @@ -293,7 +288,6 @@ List querySegment(DataSegment dataSegment, List columns 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 ea6f9d963fef..1cd6a29e10d2 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.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 void setUpAbstractParallelIndexSupervisorTaskTest() throws IOException 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 DataSegmentsWithSchemas getSegmentAndSchemas(Task task) 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 void prepareObjectMapper(ObjectMapper objectMapper, IndexIO indexIO) 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 e71dc6db36d3..b8c59d042a31 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 @@ private ParallelIndexSupervisorTask createTestTask( if (useInputFormatApi) { Preconditions.checkArgument(parseSpec == null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDir, filter), inputFormat, appendToExisting, @@ -265,7 +264,6 @@ private ParallelIndexSupervisorTask createTestTask( } 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 1448a5723f36..0d19cd86e03c 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 void testIngestNullColumn() throws JsonProcessingException null ), new ParallelIndexIOConfig( - null, getInputSource(), JSON_FORMAT, false, @@ -191,7 +190,6 @@ public void testIngestNullColumn_useFieldDiscovery_includeAllDimensions_shouldSt null ), new ParallelIndexIOConfig( - null, getInputSource(), new JsonInputFormat( new JSONPathSpec(true, null), @@ -252,7 +250,6 @@ public void testIngestNullColumn_explicitPathSpec_useFieldDiscovery_includeAllDi null ), new ParallelIndexIOConfig( - null, getInputSource(), new JsonInputFormat( new JSONPathSpec( @@ -321,7 +318,6 @@ public void testIngestNullColumn_storeEmptyColumnsOff_shouldNotStoreEmptyColumns 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 e85df4fe5dd4..cbf711469734 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 void testStopGracefully() throws Exception 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 void testSubTaskFail() throws Exception 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 @@ SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) 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 50939a99d8a1..c9858b80847b 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 void testAPIs() throws Exception 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 @@ SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) 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 cd588d0b6eea..e11fb2ef001c 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 @@ private static class ParallelIndexIngestionSpecBuilder ); 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 699476dbc16d..57dbafa173f9 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 void testFailToConstructWhenBothAppendToExistingAndForceGuaranteedRollupA 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 void testFailToConstructWhenBothInputSourceAndParserAreSet() { final ObjectMapper mapper = new DefaultObjectMapper(); final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new InlineInputSource("test"), null, false, @@ -528,7 +526,6 @@ public void testCompactionTaskDoesntCleanup() throws Exception 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 16976e827d23..2455ce692b95 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 @@ static ParallelIndexIngestionSpec createIngestionSpec( 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 97e4f54c06ef..96494b8ac794 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 @@ protected ParallelIndexSupervisorTask newTask( 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 de060e521a9f..ac8371acaa0d 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 void testRunInParallelIngestNullColumn() null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), DEFAULT_INPUT_FORMAT, false, @@ -460,7 +459,6 @@ public void testRunInParallelIngestNullColumn_storeEmptyColumnsOff_shouldNotStor null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), DEFAULT_INPUT_FORMAT, false, @@ -803,7 +801,6 @@ public void testIngestBothExplicitAndImplicitDims() throws IOException null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, "*.json", true), new JsonInputFormat( new JSONPathSpec(true, null), @@ -887,7 +884,6 @@ public void testIngestBothExplicitAndImplicitDimsSchemaDiscovery() throws IOExce null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, "*.json", true), new JsonInputFormat( new JSONPathSpec(true, null), @@ -967,7 +963,6 @@ private ParallelIndexSupervisorTask newTask( null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, inputSourceFilter, splittableInputSource), DEFAULT_INPUT_FORMAT, appendToExisting, @@ -990,7 +985,6 @@ private ParallelIndexSupervisorTask newTask( 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 b70cc7a99e69..4acc3d3f5912 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 6c6db80c1c72..4fc77934f60e 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.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 0586e98c11ba..add3ccc0a3cb 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 56695e5cf27c..0c4908563633 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 void testJavaOptsAndJavaOptsArrayOverride() throws ExecutionException, In + " \"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 void testInvalidTaskContextJavaOptsArray() throws JsonProcessingException + " \"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 static TaskConfigBuilder makeDefaultTaskConfigBuilder() .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/IndexerMetadataStorageAdapterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java index 633d861410d8..5226c9d735b8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java @@ -28,6 +28,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.metadata.TaskLookup; import org.easymock.EasyMock; import org.hamcrest.MatcherAssert; import org.joda.time.Interval; @@ -39,7 +40,7 @@ public class IndexerMetadataStorageAdapterTest { - private TaskStorageQueryAdapter taskStorageQueryAdapter; + private TaskStorage taskStorage; private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private IndexerMetadataStorageAdapter indexerMetadataStorageAdapter; @@ -47,9 +48,9 @@ public class IndexerMetadataStorageAdapterTest public void setup() { indexerMetadataStorageCoordinator = EasyMock.strictMock(IndexerMetadataStorageCoordinator.class); - taskStorageQueryAdapter = EasyMock.strictMock(TaskStorageQueryAdapter.class); + taskStorage = EasyMock.strictMock(TaskStorage.class); indexerMetadataStorageAdapter = new IndexerMetadataStorageAdapter( - taskStorageQueryAdapter, + taskStorage, indexerMetadataStorageCoordinator ); } @@ -73,7 +74,7 @@ public void testDeletePendingSegments() NoopTask.create() ) ); - EasyMock.expect(taskStorageQueryAdapter.getActiveTaskInfo("dataSource")).andReturn(taskInfos); + EasyMock.expect(taskStorage.getTaskInfos(TaskLookup.activeTasksOnly(), "dataSource")).andReturn(taskInfos); final Interval deleteInterval = Intervals.of("2017-01-01/2017-12-01"); EasyMock @@ -84,7 +85,7 @@ public void testDeletePendingSegments() ) ) .andReturn(10); - EasyMock.replay(taskStorageQueryAdapter, indexerMetadataStorageCoordinator); + EasyMock.replay(taskStorage, indexerMetadataStorageCoordinator); Assert.assertEquals(10, indexerMetadataStorageAdapter.deletePendingSegments("dataSource", deleteInterval)); } @@ -109,7 +110,8 @@ public void testDeletePendingSegmentsOfOneOverlappingRunningTask() ) ); - EasyMock.expect(taskStorageQueryAdapter.getActiveTaskInfo("dataSource")).andReturn(taskInfos); + EasyMock.expect(taskStorage.getTaskInfos(TaskLookup.activeTasksOnly(), "dataSource")) + .andReturn(taskInfos); final Interval deleteInterval = Intervals.of("2017-01-01/2017-12-01"); EasyMock @@ -120,7 +122,7 @@ public void testDeletePendingSegmentsOfOneOverlappingRunningTask() ) ) .andReturn(10); - EasyMock.replay(taskStorageQueryAdapter, indexerMetadataStorageCoordinator); + EasyMock.replay(taskStorage, indexerMetadataStorageCoordinator); MatcherAssert.assertThat( Assert.assertThrows( @@ -155,7 +157,8 @@ public void testDeletePendingSegmentsOfMultipleOverlappingRunningTasks() ) ); - EasyMock.expect(taskStorageQueryAdapter.getActiveTaskInfo("dataSource")).andReturn(taskInfos); + EasyMock.expect(taskStorage.getTaskInfos(TaskLookup.activeTasksOnly(), "dataSource")) + .andReturn(taskInfos); final Interval deleteInterval = Intervals.of("2017-01-01/2018-12-01"); EasyMock @@ -166,7 +169,7 @@ public void testDeletePendingSegmentsOfMultipleOverlappingRunningTasks() ) ) .andReturn(10); - EasyMock.replay(taskStorageQueryAdapter, indexerMetadataStorageCoordinator); + EasyMock.replay(taskStorage, indexerMetadataStorageCoordinator); MatcherAssert.assertThat( Assert.assertThrows( 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 7924dcb6e328..3824cc20f85e 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 void tearDown() throws Exception /** * 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/RealtimeishTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java index 68f0ff77bb9d..02d09b6fc6cc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java @@ -20,14 +20,13 @@ package org.apache.druid.indexing.overlord; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.LockReleaseAction; -import org.apache.druid.indexing.common.actions.SegmentInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; @@ -38,6 +37,7 @@ import org.joda.time.Interval; import org.junit.Assert; +import java.util.Collections; import java.util.List; /** @@ -97,18 +97,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception Assert.assertEquals("locks2", ImmutableList.of(lock1, lock2), locks2); // Push first segment - SegmentInsertAction firstSegmentInsertAction = new SegmentInsertAction( - ImmutableSet.of( - DataSegment.builder() - .dataSource("foo") - .interval(interval1) - .version(lock1.getVersion()) - .size(0) - .build() - ), - null - ); - toolbox.getTaskActionClient().submit(firstSegmentInsertAction); + toolbox.getTaskActionClient().submit(createSegmentInsertAction(interval1, lock1.getVersion())); // Release first lock toolbox.getTaskActionClient().submit(new LockReleaseAction(interval1)); @@ -118,18 +107,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception Assert.assertEquals("locks3", ImmutableList.of(lock2), locks3); // Push second segment - SegmentInsertAction secondSegmentInsertAction = new SegmentInsertAction( - ImmutableSet.of( - DataSegment.builder() - .dataSource("foo") - .interval(interval2) - .version(lock2.getVersion()) - .size(0) - .build() - ), - null - ); - toolbox.getTaskActionClient().submit(secondSegmentInsertAction); + toolbox.getTaskActionClient().submit(createSegmentInsertAction(interval2, lock2.getVersion())); // Release second lock toolbox.getTaskActionClient().submit(new LockReleaseAction(interval2)); @@ -141,4 +119,17 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception // Exit return TaskStatus.success(getId()); } + + private SegmentTransactionalInsertAction createSegmentInsertAction(Interval interval, String version) + { + final DataSegment segmentToInsert + = DataSegment.builder() + .dataSource("foo") + .interval(interval) + .version(version) + .size(0) + .build(); + return SegmentTransactionalInsertAction + .appendAction(Collections.singleton(segmentToInsert), null, null, null); + } } 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 ad34aa10ebe9..3acf9ff6f0e5 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.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 void setup() throws IOException .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 7e421519963f..ddbed6be7c74 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.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.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; @@ -63,11 +60,10 @@ import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.LockListAction; -import org.apache.druid.indexing.common.actions.SegmentInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionToolbox; -import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; @@ -131,9 +127,9 @@ 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; @@ -235,7 +231,7 @@ public int compare(DataSegment dataSegment, DataSegment dataSegment2) private final String taskStorageType; private ObjectMapper mapper; - private TaskStorageQueryAdapter tsqa = null; + private TaskQueryTool tsqa = null; private TaskStorage taskStorage = null; private TaskLockbox taskLockbox = null; private TaskQueue taskQueue = null; @@ -369,37 +365,6 @@ public boolean needsFormat() } } - 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 { @@ -421,17 +386,11 @@ public void setUp() throws Exception // 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); } @@ -453,7 +412,6 @@ private TaskStorage setUpTaskStorage() 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") ); @@ -478,7 +436,7 @@ private TaskStorage setUpTaskStorage() TaskMaster taskMaster = EasyMock.createMock(TaskMaster.class); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.absent()).anyTimes(); EasyMock.replay(taskMaster); - tsqa = new TaskStorageQueryAdapter(taskStorage, taskLockbox, taskMaster); + tsqa = new TaskQueryTool(taskStorage, taskLockbox, taskMaster, null, null); return taskStorage; } @@ -592,7 +550,6 @@ private TaskToolboxFactory setUpTaskToolboxFactory( taskLockbox = new TaskLockbox(taskStorage, mdc); tac = new LocalTaskActionClientFactory( - taskStorage, new TaskActionToolbox( taskLockbox, taskStorage, @@ -600,13 +557,11 @@ private TaskToolboxFactory setUpTaskToolboxFactory( emitter, EasyMock.createMock(SupervisorManager.class), mapper - ), - new TaskAuditLogConfig(true) + ) ); taskConfig = new TaskConfigBuilder() .setBaseDir(temporaryFolder.newFolder().toString()) .setDefaultRowFlushBoundary(50000) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .setTmpStorageBytesPerTask(-1L) .build(); @@ -729,7 +684,7 @@ public void testIndexTask() ), null ), - new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), + new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) .withMaxRowsInMemory(100) @@ -741,18 +696,16 @@ public void testIndexTask() 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); final TaskStatus status = taskStorage.getStatus(indexTask.getId()).get(); final List publishedSegments = BY_INTERVAL_ORDERING.sortedCopy(mdc.getPublished()); - final List loggedSegments = BY_INTERVAL_ORDERING.sortedCopy(tsqa.getInsertedSegments(indexTask.getId())); Assert.assertEquals("statusCode", TaskState.SUCCESS, status.getStatusCode()); Assert.assertEquals(taskLocation, status.getLocation()); Assert.assertEquals("merged statusCode", TaskState.SUCCESS, mergedStatus.getStatusCode()); - Assert.assertEquals("segments logged vs published", loggedSegments, publishedSegments); Assert.assertEquals("num segments published", 2, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); @@ -794,7 +747,7 @@ public void testIndexTaskFailure() 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) @@ -1103,7 +1056,9 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception .size(0) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment), null)); + toolbox.getTaskActionClient().submit( + SegmentTransactionalInsertAction.appendAction(ImmutableSet.of(segment), null, null, null) + ); return TaskStatus.success(getId()); } }; @@ -1144,7 +1099,9 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception .size(0) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment), null)); + toolbox.getTaskActionClient().submit( + SegmentTransactionalInsertAction.appendAction(ImmutableSet.of(segment), null, null, null) + ); return TaskStatus.success(getId()); } }; @@ -1186,7 +1143,9 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception .size(0) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment), null)); + toolbox.getTaskActionClient().submit( + SegmentTransactionalInsertAction.appendAction(ImmutableSet.of(segment), null, null, null) + ); return TaskStatus.success(getId()); } }; @@ -1218,7 +1177,7 @@ public void testResumeTasks() throws Exception ), null ), - new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), + new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) .withMaxRowsInMemory(10) @@ -1234,7 +1193,7 @@ public void testResumeTasks() throws Exception 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()); } @@ -1244,11 +1203,9 @@ public void testResumeTasks() throws Exception final TaskStatus status = taskStorage.getStatus(indexTask.getId()).get(); final List publishedSegments = BY_INTERVAL_ORDERING.sortedCopy(mdc.getPublished()); - final List loggedSegments = BY_INTERVAL_ORDERING.sortedCopy(tsqa.getInsertedSegments(indexTask.getId())); Assert.assertEquals("statusCode", TaskState.SUCCESS, status.getStatusCode()); Assert.assertEquals(taskLocation, status.getLocation()); - Assert.assertEquals("segments logged vs published", loggedSegments, publishedSegments); Assert.assertEquals("num segments published", 2, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); @@ -1308,7 +1265,7 @@ public void testUnifiedAppenderatorsManagerCleanup() throws Exception ), null ), - new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), + new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) .withMaxRowsInMemory(10) @@ -1320,7 +1277,7 @@ public void testUnifiedAppenderatorsManagerCleanup() throws Exception 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); @@ -1410,7 +1367,7 @@ private TaskStatus runTask(final Task task) 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/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java index 8984e73b6b9c..ee90a3335a17 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 void testGetActiveTaskRedactsPassword() throws JsonProcessingException 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 edebc0fe8dc8..19e4bd33fd8e 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.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 46e03206f49d..f9a0105c0676 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 @@ 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 void testGetRedirectURL() 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 void testGetRedirectURLWithEncodedCharacter() throws UnsupportedEncodingE "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 51b0cfe742e6..e6dee0c7e403 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 @@ -39,13 +39,16 @@ 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.TaskStorageQueryAdapter; +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 +61,7 @@ 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.TaskLookup; import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup; import org.apache.druid.metadata.TaskLookup.TaskLookupType; @@ -102,11 +106,14 @@ 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; - private TaskStorageQueryAdapter taskStorageQueryAdapter; + private TaskQueryTool taskQueryTool; private IndexerMetadataStorageAdapter indexerMetadataStorageAdapter; private HttpServletRequest req; private TaskRunner taskRunner; @@ -121,12 +128,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); - taskStorageQueryAdapter = EasyMock.createStrictMock(TaskStorageQueryAdapter.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,15 +186,15 @@ public Access authorize(AuthenticationResult authenticationResult, Resource reso }; overlordResource = new OverlordResource( + overlord, taskMaster, - taskStorageQueryAdapter, + taskQueryTool, indexerMetadataStorageAdapter, null, configManager, auditManager, authMapper, workerTaskRunnerQueryAdapter, - provisioningStrategy, authConfig ); } @@ -189,7 +205,8 @@ public void tearDown() EasyMock.verify( taskRunner, taskMaster, - taskStorageQueryAdapter, + taskStorage, + taskLockbox, indexerMetadataStorageAdapter, req, workerTaskRunnerQueryAdapter, @@ -197,19 +214,30 @@ public void tearDown() ); } - @Test - public void testLeader() + private void replayAll() { - EasyMock.expect(taskMaster.getCurrentLeader()).andReturn("boz").once(); EasyMock.replay( + overlord, taskRunner, + taskQueue, taskMaster, - taskStorageQueryAdapter, + taskStorage, + taskLockbox, indexerMetadataStorageAdapter, req, workerTaskRunnerQueryAdapter, - authConfig + authConfig, + configManager, + auditManager, + provisioningStrategy ); + } + + @Test + public void testLeader() + { + EasyMock.expect(overlord.getCurrentLeader()).andReturn("boz").once(); + replayAll(); final Response response = overlordResource.getLeader(); Assert.assertEquals("boz", response.getEntity()); @@ -219,17 +247,9 @@ public void testLeader() @Test public void testIsLeader() { - EasyMock.expect(taskMaster.isLeader()).andReturn(true).once(); - EasyMock.expect(taskMaster.isLeader()).andReturn(false).once(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + EasyMock.expect(overlord.isLeader()).andReturn(true).once(); + EasyMock.expect(overlord.isLeader()).andReturn(false).once(); + replayAll(); // true final Response response1 = overlordResource.isLeader(); @@ -247,7 +267,7 @@ public void testSecuredGetWaitingTask() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -267,15 +287,7 @@ public void testSecuredGetWaitingTask() ) ); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); List responseObjects = (List) overlordResource.getWaitingTasks(req) .getEntity(); @@ -290,7 +302,7 @@ public void testSecuredGetCompleteTasks() List tasksIds = ImmutableList.of("id_1", "id_2", "id_3"); EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, (Duration) null)), null) ).andStubReturn( ImmutableList.of( @@ -299,15 +311,8 @@ public void testSecuredGetCompleteTasks() createTaskStatusPlus("id_3", TaskState.SUCCESS, "allow") ) ); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); + List responseObjects = (List) overlordResource .getCompleteTasks(null, req).getEntity(); @@ -328,7 +333,7 @@ public void testSecuredGetRunningTasks() ) ); EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -341,15 +346,7 @@ public void testSecuredGetRunningTasks() EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andStubReturn(RunnerTaskState.RUNNING); EasyMock.expect(taskRunner.getRunnerTaskState("id_2")).andStubReturn(RunnerTaskState.RUNNING); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); List responseObjects = (List) overlordResource.getRunningTasks(null, req) .getEntity(); @@ -363,7 +360,7 @@ public void testGetTasks() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance(), @@ -390,15 +387,8 @@ public void testGetTasks() ) ).atLeastOnce(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); + List responseObjects = (List) overlordResource .getTasks(null, null, null, null, null, req) .getEntity(); @@ -411,7 +401,7 @@ public void testGetTasksFilterDataSource() expectAuthorizationTokenCheck(); //completed tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -437,15 +427,7 @@ public void testGetTasksFilterDataSource() new MockTaskRunnerWorkItem("id_4") ) ).atLeastOnce(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); List responseObjects = (List) overlordResource .getTasks(null, "allow", null, null, null, req) @@ -461,7 +443,7 @@ public void testGetTasksFilterWaitingState() expectAuthorizationTokenCheck(); //active tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance() @@ -484,15 +466,8 @@ public void testGetTasksFilterWaitingState() ) ); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); + List responseObjects = (List) overlordResource .getTasks( "waiting", @@ -511,7 +486,7 @@ public void testGetTasksFilterRunningState() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance() @@ -537,15 +512,7 @@ public void testGetTasksFilterRunningState() EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andReturn(RunnerTaskState.RUNNING); EasyMock.expect(taskRunner.getRunnerTaskState("id_2")).andReturn(RunnerTaskState.RUNNING); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); List responseObjects = (List) overlordResource .getTasks("running", "allow", null, null, null, req) @@ -569,7 +536,7 @@ public void testGetTasksFilterPendingState() ) ); EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance()), null ) @@ -587,15 +554,7 @@ public void testGetTasksFilterPendingState() EasyMock.expect(taskRunner.getRunnerTaskState("id_3")).andStubReturn(RunnerTaskState.RUNNING); EasyMock.expect(taskRunner.getRunnerTaskState("id_4")).andStubReturn(RunnerTaskState.RUNNING); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); List responseObjects = (List) overlordResource .getTasks("pending", null, null, null, null, req) @@ -611,7 +570,7 @@ public void testGetTasksFilterCompleteState() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of(TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, (Duration) null)), null ) @@ -622,15 +581,8 @@ public void testGetTasksFilterCompleteState() createTaskStatusPlus("id_3", TaskState.SUCCESS, "allow") ) ); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); + List responseObjects = (List) overlordResource .getTasks("complete", null, null, null, null, req) .getEntity(); @@ -645,7 +597,7 @@ public void testGetTasksFilterCompleteStateWithInterval() expectAuthorizationTokenCheck(); Duration duration = new Period("PT86400S").toStandardDuration(); EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( EasyMock.anyObject(), EasyMock.anyObject() ) @@ -657,15 +609,8 @@ public void testGetTasksFilterCompleteStateWithInterval() ) ); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); + String interval = "2010-01-01_P1D"; List responseObjects = (List) overlordResource .getTasks("complete", null, interval, null, null, req) @@ -684,7 +629,7 @@ public void testGetTasksRequiresDatasourceRead() // Setup mocks to return completed, active, known, pending and running tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -712,16 +657,7 @@ public void testGetTasksRequiresDatasourceRead() EasyMock.expect(taskRunner.getRunnerTaskState("id_4")).andReturn(RunnerTaskState.PENDING); EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andReturn(RunnerTaskState.RUNNING); - // Replay all mocks - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); // Verify that only the tasks of read access datasource are returned List responseObjects = (List) overlordResource @@ -742,7 +678,7 @@ public void testGetTasksFilterByTaskTypeRequiresDatasourceRead() // Setup mocks to return completed, active, known, pending and running tasks EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null), @@ -769,16 +705,7 @@ public void testGetTasksFilterByTaskTypeRequiresDatasourceRead() EasyMock.expect(taskRunner.getRunnerTaskState("id_1")).andReturn(RunnerTaskState.RUNNING); - // Replay all mocks - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); // Verify that only the tasks of read access datasource are returned List responseObjects = (List) overlordResource @@ -797,20 +724,13 @@ public void testGetTasksFilterByDatasourceRequiresReadAccess() // and no access to "buzzfeed" expectAuthorizationTokenCheck(Users.WIKI_READER); - // Replay all mocks - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + 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 @@ -818,7 +738,7 @@ public void testGetCompleteTasksOfAllDatasources() { expectAuthorizationTokenCheck(); EasyMock.expect( - taskStorageQueryAdapter.getTaskStatusPlusList( + taskStorage.getTaskStatusPlusList( ImmutableMap.of( TaskLookupType.COMPLETE, CompleteTaskLookup.of(null, null) @@ -832,15 +752,8 @@ public void testGetCompleteTasksOfAllDatasources() createTaskStatusPlus("id_3", TaskState.SUCCESS, "allow") ) ); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); + List responseObjects = (List) overlordResource .getTasks("complete", null, null, null, null, req) .getEntity(); @@ -852,20 +765,13 @@ public void testGetCompleteTasksOfAllDatasources() @Test public void testGetTasksNegativeState() { - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); + Object responseObject = overlordResource .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() ); } @@ -877,15 +783,8 @@ public void testSecuredTaskPost() expectAuthorizationTokenCheck(); EasyMock.expect(authConfig.isEnableInputSourceSecurity()).andReturn(false); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); + Task task = NoopTask.create(); overlordResource.taskPost(task, req); } @@ -914,17 +813,7 @@ public void testKillTaskIsAudited() auditManager.doAudit(EasyMock.capture(auditEntryCapture)); EasyMock.expectLastCall().once(); - EasyMock.replay( - taskRunner, - taskMaster, - taskQueue, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig, - auditManager - ); + replayAll(); Task task = new KillUnusedSegmentsTask("kill_all", "allow", Intervals.ETERNITY, null, null, 10, null, null); overlordResource.taskPost(task, req); @@ -943,15 +832,7 @@ public void testTaskPostDeniesDatasourceReadUser() expectAuthorizationTokenCheck(Users.WIKI_READER); EasyMock.expect(authConfig.isEnableInputSourceSecurity()).andReturn(false); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); // Verify that taskPost fails for user who has only datasource read access Task task = NoopTask.forDatasource(Datasources.WIKIPEDIA); @@ -965,7 +846,7 @@ public void testKillPendingSegments() { expectAuthorizationTokenCheck(); - EasyMock.expect(taskMaster.isLeader()).andReturn(true); + EasyMock.expect(overlord.isLeader()).andReturn(true); EasyMock .expect( indexerMetadataStorageAdapter.deletePendingSegments( @@ -975,15 +856,7 @@ public void testKillPendingSegments() ) .andReturn(2); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); Response response = overlordResource .killPendingSegments("allow", new Interval(DateTimes.MIN, DateTimes.nowUtc()).toString(), req); @@ -996,7 +869,7 @@ public void testKillPendingSegmentsThrowsInvalidInputDruidException() { expectAuthorizationTokenCheck(); - EasyMock.expect(taskMaster.isLeader()).andReturn(true); + EasyMock.expect(overlord.isLeader()).andReturn(true); final String exceptionMsg = "Some exception msg"; EasyMock .expect( @@ -1008,15 +881,7 @@ public void testKillPendingSegmentsThrowsInvalidInputDruidException() .andThrow(InvalidInput.exception(exceptionMsg)) .once(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); Response response = overlordResource .killPendingSegments("allow", new Interval(DateTimes.MIN, DateTimes.nowUtc()).toString(), req); @@ -1030,7 +895,7 @@ public void testKillPendingSegmentsThrowsDefensiveDruidException() { expectAuthorizationTokenCheck(); - EasyMock.expect(taskMaster.isLeader()).andReturn(true); + EasyMock.expect(overlord.isLeader()).andReturn(true); final String exceptionMsg = "An internal defensive exception"; EasyMock .expect( @@ -1042,15 +907,7 @@ public void testKillPendingSegmentsThrowsDefensiveDruidException() .andThrow(DruidException.defensive(exceptionMsg)) .once(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); Response response = overlordResource .killPendingSegments("allow", new Interval(DateTimes.MIN, DateTimes.nowUtc()).toString(), req); @@ -1064,7 +921,7 @@ public void testKillPendingSegmentsThrowsArbitraryException() { expectAuthorizationTokenCheck(); - EasyMock.expect(taskMaster.isLeader()).andReturn(true); + EasyMock.expect(overlord.isLeader()).andReturn(true); final String exceptionMsg = "An unexpected illegal state exception"; EasyMock .expect( @@ -1076,15 +933,7 @@ public void testKillPendingSegmentsThrowsArbitraryException() .andThrow(new IllegalStateException(exceptionMsg)) .once(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); Response response = overlordResource .killPendingSegments("allow", new Interval(DateTimes.MIN, DateTimes.nowUtc()).toString(), req); @@ -1098,17 +947,9 @@ public void testKillPendingSegmentsToNonLeader() { expectAuthorizationTokenCheck(); - EasyMock.expect(taskMaster.isLeader()).andReturn(false); + EasyMock.expect(overlord.isLeader()).andReturn(false); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); Response response = overlordResource .killPendingSegments("allow", new Interval(DateTimes.MIN, DateTimes.nowUtc()).toString(), req); @@ -1124,22 +965,16 @@ public void testGetTaskPayload() throws Exception // 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(taskStorageQueryAdapter.getTask("mytask")) + EasyMock.expect(taskStorage.getTask("mytask")) .andReturn(Optional.of(task)); - EasyMock.expect(taskStorageQueryAdapter.getTask("othertask")) + EasyMock.expect(taskStorage.getTask("othertask")) .andReturn(Optional.absent()); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); final Response response1 = overlordResource.getTaskPayload("mytask"); final TaskPayloadResponse taskPayloadResponse1 = TestHelper.makeJsonMapper().readValue( @@ -1167,7 +1002,7 @@ public void testGetTaskStatus() throws Exception final String taskId = task.getId(); final TaskStatus status = TaskStatus.running(taskId); - EasyMock.expect(taskStorageQueryAdapter.getTaskInfo(taskId)) + EasyMock.expect(taskQueryTool.getTaskInfo(taskId)) .andReturn(new TaskInfo( task.getId(), DateTimes.of("2018-01-01"), @@ -1176,21 +1011,13 @@ public void testGetTaskStatus() throws Exception task )); - EasyMock.expect(taskStorageQueryAdapter.getTaskInfo("othertask")) + EasyMock.expect(taskQueryTool.getTaskInfo("othertask")) .andReturn(null); EasyMock.>expect(taskRunner.getKnownTasks()) .andReturn(ImmutableList.of()); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); final Response response1 = overlordResource.getTaskStatus(taskId); final TaskStatusResponse taskStatusResponse1 = TestHelper.makeJsonMapper().readValue( @@ -1239,17 +1066,9 @@ public void testGetLockedIntervals() throws Exception ) ); - EasyMock.expect(taskStorageQueryAdapter.getLockedIntervals(minTaskPriority)) + EasyMock.expect(taskLockbox.getLockedIntervals(minTaskPriority)) .andReturn(expectedLockedIntervals); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); final Response response = overlordResource.getDatasourceLockedIntervals(minTaskPriority); Assert.assertEquals(200, response.getStatus()); @@ -1268,15 +1087,7 @@ public void testGetLockedIntervals() throws Exception @Test public void testGetLockedIntervalsWithEmptyBody() { - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); Response response = overlordResource.getDatasourceLockedIntervals(null); Assert.assertEquals(400, response.getStatus()); @@ -1292,26 +1103,14 @@ public void testShutdownTask() // 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(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - mockQueue, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); + EasyMock.replay(mockQueue); final Map response = (Map) overlordResource .doShutdown("id_1") @@ -1326,14 +1125,12 @@ public void testShutdownAllTasks() // 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(taskStorageQueryAdapter.getActiveTaskInfo("datasource")).andStubReturn(ImmutableList.of( + EasyMock.expect( + taskStorage.getTaskInfos(TaskLookup.activeTasksOnly(), "datasource") + ).andStubReturn(ImmutableList.of( new TaskInfo<>( "id_1", DateTime.now(ISOChronology.getInstanceUTC()), @@ -1354,16 +1151,8 @@ public void testShutdownAllTasks() mockQueue.shutdown("id_2", "Shutdown request from user"); EasyMock.expectLastCall(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - mockQueue, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); + EasyMock.replay(mockQueue); final Map response = (Map) overlordResource .shutdownTasksForDataSource("datasource") @@ -1375,18 +1164,11 @@ public void testShutdownAllTasks() 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(taskStorageQueryAdapter.getActiveTaskInfo(EasyMock.anyString())).andReturn(Collections.emptyList()); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + EasyMock.expect(taskStorage.getTaskInfos(EasyMock.anyObject(TaskLookup.class), EasyMock.anyString())) + .andReturn(Collections.emptyList()); + replayAll(); final Response response = overlordResource.shutdownTasksForDataSource("notExisting"); Assert.assertEquals(Status.NOT_FOUND.getStatusCode(), response.getStatus()); @@ -1400,15 +1182,7 @@ public void testEnableWorker() workerTaskRunnerQueryAdapter.enableWorker(host); EasyMock.expectLastCall().once(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); final Response response = overlordResource.enableWorker(host); @@ -1424,15 +1198,7 @@ public void testDisableWorker() workerTaskRunnerQueryAdapter.disableWorker(host); EasyMock.expectLastCall().once(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); final Response response = overlordResource.disableWorker(host); @@ -1448,15 +1214,7 @@ public void testEnableWorkerWhenWorkerAPIRaisesError() workerTaskRunnerQueryAdapter.enableWorker(host); EasyMock.expectLastCall().andThrow(new RE("Worker API returns error!")).once(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); final Response response = overlordResource.enableWorker(host); @@ -1472,15 +1230,7 @@ public void testDisableWorkerWhenWorkerAPIRaisesError() workerTaskRunnerQueryAdapter.disableWorker(host); EasyMock.expectLastCall().andThrow(new RE("Worker API returns error!")).once(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - authConfig - ); + replayAll(); final Response response = overlordResource.disableWorker(host); @@ -1491,20 +1241,8 @@ public void testDisableWorkerWhenWorkerAPIRaisesError() @Test public void testGetTotalWorkerCapacityNotLeader() { - EasyMock.reset(taskMaster); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn( - Optional.absent() - ).anyTimes(); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - configManager, - authConfig - ); + EasyMock.expect(overlord.isLeader()).andReturn(false); + replayAll(); final Response response = overlordResource.getTotalWorkerCapacity(); Assert.assertEquals(HttpResponseStatus.SERVICE_UNAVAILABLE.getCode(), response.getStatus()); } @@ -1513,20 +1251,15 @@ public void testGetTotalWorkerCapacityNotLeader() 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.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - configManager, - authConfig - ); + EasyMock.expect(overlord.isLeader()).andReturn(true); + replayAll(); + final Response response = overlordResource.getTotalWorkerCapacity(); Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus()); Assert.assertEquals(-1, ((TotalWorkerCapacityResponse) response.getEntity()).getCurrentClusterCapacity()); @@ -1541,16 +1274,9 @@ public void testGetTotalWorkerCapacityWithWorkerTaskRunnerButWorkerBehaviorConfi EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference); EasyMock.expect(taskRunner.getTotalCapacity()).andReturn(-1); EasyMock.expect(taskRunner.getUsedCapacity()).andReturn(-1); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - configManager, - authConfig - ); + EasyMock.expect(overlord.isLeader()).andReturn(true); + replayAll(); + final Response response = overlordResource.getTotalWorkerCapacity(); Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus()); Assert.assertEquals(-1, ((TotalWorkerCapacityResponse) response.getEntity()).getCurrentClusterCapacity()); @@ -1566,16 +1292,9 @@ public void testGetTotalWorkerCapacityWithWorkerTaskRunnerButAutoScaleNotConfigu EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference); EasyMock.expect(taskRunner.getTotalCapacity()).andReturn(-1); EasyMock.expect(taskRunner.getUsedCapacity()).andReturn(-1); - EasyMock.replay( - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - configManager, - authConfig - ); + EasyMock.expect(overlord.isLeader()).andReturn(true); + replayAll(); + final Response response = overlordResource.getTotalWorkerCapacity(); Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus()); Assert.assertEquals(-1, ((TotalWorkerCapacityResponse) response.getEntity()).getCurrentClusterCapacity()); @@ -1617,17 +1336,10 @@ public void testGetTotalWorkerCapacityWithAutoScaleConfiguredAndProvisioningStra EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference); EasyMock.replay( workerTaskRunner, - autoScaler, - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - configManager, - provisioningStrategy, - authConfig + autoScaler ); + EasyMock.expect(overlord.isLeader()).andReturn(true); + replayAll(); final Response response = overlordResource.getTotalWorkerCapacity(); Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus()); Assert.assertEquals(expectedWorkerCapacity, ((TotalWorkerCapacityResponse) response.getEntity()).getCurrentClusterCapacity()); @@ -1670,17 +1382,10 @@ public void testGetTotalWorkerCapacityWithAutoScaleConfiguredAndProvisioningStra EasyMock.expect(configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class)).andReturn(workerBehaviorConfigAtomicReference); EasyMock.replay( workerTaskRunner, - autoScaler, - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter, - configManager, - provisioningStrategy, - authConfig + autoScaler ); + EasyMock.expect(overlord.isLeader()).andReturn(true); + replayAll(); final Response response = overlordResource.getTotalWorkerCapacity(); Assert.assertEquals(HttpResponseStatus.OK.getCode(), response.getStatus()); Assert.assertEquals(workerInfos.stream().findFirst().get().getWorker().getCapacity(), ((TotalWorkerCapacityResponse) response.getEntity()).getCurrentClusterCapacity()); @@ -1706,16 +1411,8 @@ public void testResourceActionsForTaskWithInputTypeAndInputSecurityEnabled() Action.READ ))); - EasyMock.replay( - task, - authConfig, - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter - ); + EasyMock.replay(task); + replayAll(); Set expectedResourceActions = ImmutableSet.of( new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.WRITE), @@ -1726,9 +1423,8 @@ public void testResourceActionsForTaskWithInputTypeAndInputSecurityEnabled() } @Test - public void testResourceActionsForTaskWithFirehoseAndInputSecurityEnabled() + public void testResourceActionsForTaskWithInvalidSecurityAndInputSecurityEnabled() { - final String dataSource = "dataSourceTest"; final UOE expectedException = new UOE("unsupported"); Task task = EasyMock.createMock(Task.class); @@ -1739,17 +1435,8 @@ public void testResourceActionsForTaskWithFirehoseAndInputSecurityEnabled() EasyMock.expect(task.getDestinationResource()).andReturn(java.util.Optional.of(new Resource(dataSource, ResourceType.DATASOURCE))); EasyMock.expect(task.getInputSourceResources()).andThrow(expectedException); - EasyMock.replay( - task, - authConfig, - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter - ); - + EasyMock.replay(task); + replayAll(); final UOE e = Assert.assertThrows( UOE.class, @@ -1762,7 +1449,6 @@ public void testResourceActionsForTaskWithFirehoseAndInputSecurityEnabled() @Test public void testResourceActionsForTaskWithInputTypeAndInputSecurityDisabled() { - final String dataSource = "dataSourceTest"; final String inputSourceType = "local"; Task task = EasyMock.createMock(Task.class); @@ -1776,16 +1462,8 @@ public void testResourceActionsForTaskWithInputTypeAndInputSecurityDisabled() Action.READ ))); - EasyMock.replay( - task, - authConfig, - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter - ); + EasyMock.replay(task); + replayAll(); Set expectedResourceActions = ImmutableSet.of( new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.WRITE) @@ -1797,36 +1475,12 @@ public void testResourceActionsForTaskWithInputTypeAndInputSecurityDisabled() @Test public void testGetMultipleTaskStatuses_presentTaskQueue() { - // Needed for teardown - EasyMock.replay( - authConfig, - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter - ); - - 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"))); - EasyMock.replay(taskQueue); - TaskMaster taskMaster = EasyMock.createMock(TaskMaster.class); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)); - EasyMock.replay(taskMaster); - 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); @@ -1835,41 +1489,34 @@ public void testGetMultipleTaskStatuses_presentTaskQueue() @Test public void testGetMultipleTaskStatuses_absentTaskQueue() { - // Needed for teardown - EasyMock.replay( - authConfig, - taskRunner, - taskMaster, - taskStorageQueryAdapter, - indexerMetadataStorageAdapter, - req, - workerTaskRunnerQueryAdapter - ); - - TaskStorageQueryAdapter taskStorageQueryAdapter = EasyMock.createMock(TaskStorageQueryAdapter.class); - EasyMock.expect(taskStorageQueryAdapter.getStatus("task")) + EasyMock.expect(taskStorage.getStatus("task")) .andReturn(Optional.of(TaskStatus.running("task"))); - EasyMock.replay(taskStorageQueryAdapter); - TaskMaster taskMaster = EasyMock.createMock(TaskMaster.class); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.absent()); - EasyMock.replay(taskMaster); - OverlordResource overlordResource = new OverlordResource( - taskMaster, - taskStorageQueryAdapter, - 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); } + @Test + public void testGetTaskSegmentsReturns404() + { + replayAll(); + OverlordResource overlordResource = + new OverlordResource(null, null, null, null, null, null, null, null, null, null); + final Response response = overlordResource.getTaskSegments("taskId"); + Assert.assertEquals(404, response.getStatus()); + Assert.assertEquals( + Collections.singletonMap( + "error", + "Segment IDs committed by a task action are not persisted anymore." + + " Use the metric 'segment/added/bytes' to identify the segments created by a task." + ), + response.getEntity() + ); + } + private void expectAuthorizationTokenCheck() { expectAuthorizationTokenCheck(Users.DRUID); 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 aab3639b2e63..d1c2167b9290 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,17 +47,18 @@ 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; 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.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerFactory; import org.apache.druid.indexing.overlord.TaskRunnerListener; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; -import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter; import org.apache.druid.indexing.overlord.WorkerTaskRunnerQueryAdapter; import org.apache.druid.indexing.overlord.autoscaling.ScalingStats; import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; @@ -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 MockTaskRunner get() 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,29 +267,31 @@ public MockTaskRunner get() 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 TaskStorageQueryAdapter taskStorageQueryAdapter = new TaskStorageQueryAdapter(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, - taskStorageQueryAdapter, - new IndexerMetadataStorageAdapter(taskStorageQueryAdapter, null), + taskQueryTool, + new IndexerMetadataStorageAdapter(taskStorage, null), null, null, auditManager, AuthTestUtils.TEST_AUTHORIZER_MAPPER, workerTaskRunnerQueryAdapter, - null, new AuthConfig() ); Response response = overlordResource.getLeader(); @@ -351,8 +360,8 @@ public void testOverlordRun() throws Exception 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/http/security/OverlordSecurityResourceFilterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java index 3b775ca94915..a25c6aec6215 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java @@ -26,7 +26,7 @@ import com.sun.jersey.spi.container.ResourceFilter; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter; +import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.http.OverlordResource; import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; @@ -60,7 +60,7 @@ public static Collection data() return ImmutableList.copyOf( Iterables.concat( getRequestPaths(OverlordResource.class, ImmutableList.of( - TaskStorageQueryAdapter.class, + TaskQueryTool.class, AuthorizerMapper.class ) ), @@ -84,7 +84,7 @@ public static Collection data() private static boolean mockedOnceTsqa; private static boolean mockedOnceSM; - private TaskStorageQueryAdapter tsqa; + private TaskQueryTool tsqa; private SupervisorManager supervisorManager; public OverlordSecurityResourceFilterTest( @@ -107,7 +107,7 @@ public void setUp() // Since we are creating the mocked tsqa object only once and getting that object from Guice here therefore // if the mockedOnce check is not done then we will call EasyMock.expect and EasyMock.replay on the mocked object // multiple times and it will throw exceptions - tsqa = injector.getInstance(TaskStorageQueryAdapter.class); + tsqa = injector.getInstance(TaskQueryTool.class); EasyMock.expect(tsqa.getTask(EasyMock.anyString())).andReturn(Optional.of(noopTask)).anyTimes(); EasyMock.replay(tsqa); mockedOnceTsqa = true; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilterTest.java index 59abcaacdf80..8a7684b5ccd0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilterTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilterTest.java @@ -21,7 +21,7 @@ import com.google.common.base.Optional; import com.sun.jersey.spi.container.ContainerRequest; -import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter; +import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.server.security.AuthorizerMapper; import org.easymock.EasyMock; @@ -42,7 +42,7 @@ public class TaskResourceFilterTest { private AuthorizerMapper authorizerMapper; - private TaskStorageQueryAdapter taskStorageQueryAdapter; + private TaskQueryTool taskQueryTool; private ContainerRequest containerRequest; private TaskResourceFilter resourceFilter; @@ -50,9 +50,9 @@ public class TaskResourceFilterTest public void setup() { authorizerMapper = EasyMock.createMock(AuthorizerMapper.class); - taskStorageQueryAdapter = EasyMock.createMock(TaskStorageQueryAdapter.class); + taskQueryTool = EasyMock.createMock(TaskQueryTool.class); containerRequest = EasyMock.createMock(ContainerRequest.class); - resourceFilter = new TaskResourceFilter(taskStorageQueryAdapter, authorizerMapper); + resourceFilter = new TaskResourceFilter(taskQueryTool, authorizerMapper); } @Test @@ -68,11 +68,11 @@ public void testTaskNotFound() expect(supervisorSpec.getDataSources()) .andReturn(Collections.singletonList(taskId)) .anyTimes(); - expect(taskStorageQueryAdapter.getTask(taskId)) + expect(taskQueryTool.getTask(taskId)) .andReturn(Optional.absent()) .atLeastOnce(); EasyMock.replay(containerRequest); - EasyMock.replay(taskStorageQueryAdapter); + EasyMock.replay(taskQueryTool); WebApplicationException expected = null; try { @@ -84,7 +84,7 @@ public void testTaskNotFound() Assert.assertNotNull(expected); Assert.assertEquals(expected.getResponse().getStatus(), Response.Status.NOT_FOUND.getStatusCode()); EasyMock.verify(containerRequest); - EasyMock.verify(taskStorageQueryAdapter); + EasyMock.verify(taskQueryTool); } private List getPathSegments(String path) 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 6f92aeef646b..36dfd966e8d2 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.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 IndexTaskSamplerSpecTest() .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 1a9e5a17e1cf..258ebff7b504 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 @@ -62,7 +62,6 @@ import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionToolbox; -import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.common.config.TaskStorageConfig; @@ -118,8 +117,8 @@ 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; @@ -574,7 +573,6 @@ protected void makeToolboxFactory(TestUtils testUtils, ServiceEmitter emitter, b .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(); @@ -631,9 +629,7 @@ public boolean checkPointDataSourceMetadata( objectMapper ); final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( - taskStorage, - taskActionToolbox, - new TaskAuditLogConfig(false) + taskActionToolbox ); final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() { 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 ca3cd6edf8b3..88249509ef97 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 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 8217a12eb837..6b08be3a3c65 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.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 @@ private WorkerTaskManager createWorkerTaskManager() .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 c3191cc068d5..e1822505b977 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.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 @@ private WorkerTaskMonitor createTaskMonitor() 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 f5b0ca5644ee..5505160d989d 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 void setup() throws IOException { 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 fabb1cfb1961..cc14d4505e95 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 void setup() throws IOException 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 ceb74f3631a1..c8c9b7fe0665 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 void setup() throws IOException 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 40e8ad3e276a..69d48417deee 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 Period getIntermediaryPartitionTimeout() }; 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 96e8d500fa6e..20bdd9767a77 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 void testIndexData() throws Exception 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/resources/indexer/export_task.json b/integration-tests-ex/cases/src/test/resources/indexer/export_task.json index e5bfdac4af71..45e59704a85e 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/EventReceiverFirehoseTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java deleted file mode 100644 index 761d9f55dbaf..000000000000 --- 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/guice/DruidTestModuleFactory.java b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java index d08712dea35e..1195795cd970 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.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 @@ private static List getModules() { return ImmutableList.of( new DruidTestModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTuningConfigModule() ); 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 be2ab61a5f59..1312685fe5a1 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 void testIndexData(PartitionsSpec partitionsSpec) throws Exception 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/ITPerfectRollupParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java index 0e8fc904949c..5695156fa9bf 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 void testIndexData(PartitionsSpec partitionsSpec) throws Exception ); - // with DruidInputSource instead of IngestSegmentFirehose + // with DruidInputSource doReindexTest( INDEX_DATASOURCE, INDEX_DRUID_INPUT_SOURCE_DATASOURCE, diff --git a/licenses.yaml b/licenses.yaml index abfc8924898d..dcdac7bd1879 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 000000000000..d5e5ab592f4e --- /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 000000000000..8109ef9ff8a2 --- /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 000000000000..6c48e928b3c1 --- /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 3490fbd0ee7a..ea9c3dcfa179 100644 --- a/pom.xml +++ b/pom.xml @@ -608,7 +608,7 @@ pom - org.hibernate + org.hibernate.validator hibernate-validator ${hibernate-validator.version} diff --git a/processing/pom.xml b/processing/pom.xml index df0ca6fd464d..bfb39cf2f774 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/data/input/Firehose.java b/processing/src/main/java/org/apache/druid/data/input/Firehose.java deleted file mode 100644 index a9ff6f33159c..000000000000 --- 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 6ec8657d435a..000000000000 --- 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 ed59d67285a6..433a6a2f9958 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.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 517235a99f98..ca2708700f08 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 @@ 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 ParallelMergeCombiningSequence( 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 Yielder toYielder(OutType initValue, YieldingAccumulat 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 @@ static Sequence makeOutputSequenceForQueue( return new BaseSequence<>( new BaseSequence.IteratorMaker>() { - private boolean shouldCancelOnCleanup = true; - @Override public Iterator make() { @@ -195,7 +219,7 @@ public boolean hasNext() { 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 boolean hasNext() } } 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 T next() @Override public void cleanup(Iterator iterFromMake) { - if (shouldCancelOnCleanup) { - cancellationGizmo.cancel(new RuntimeException("Already closed")); - } + // nothing to cleanup } } ); @@ -338,7 +359,7 @@ protected void compute() 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 @@ protected void compute() 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 @@ private void spawnParallelTasks(int parallelMergeTasks) 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 @@ private void spawnParallelTasks(int parallelMergeTasks) 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 @@ private static class MergeCombineAction extends RecursiveAction 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 @@ private static class MergeCombineAction extends RecursiveAction private MergeCombineAction( PriorityQueue> pQueue, - QueuePusher> outputQueue, + QueuePusher outputQueue, Ordering orderingFn, BinaryOperator combineFn, T initialValue, @@ -550,6 +571,10 @@ private MergeCombineAction( @Override protected void compute() { + if (cancellationGizmo.isCanceled()) { + cleanup(); + return; + } try { long start = System.nanoTime(); long startCpuNanos = JvmUtils.safeGetThreadCpuTime(); @@ -608,7 +633,7 @@ protected void compute() 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 @@ protected void compute() 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 @@ private static class PrepareMergeCombineInputsAction extends RecursiveAction 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 @@ private static class PrepareMergeCombineInputsAction extends RecursiveAction private PrepareMergeCombineInputsAction( List> partition, - QueuePusher> outputQueue, + QueuePusher outputQueue, Ordering orderingFn, BinaryOperator combineFn, int yieldAfter, @@ -744,7 +776,7 @@ protected void compute() cursor.close(); } } - if (cursors.size() > 0) { + if (!cancellationGizmo.isCanceled() && !cursors.isEmpty()) { getPool().execute(new MergeCombineAction( cursors, outputQueue, @@ -758,14 +790,17 @@ protected void compute() 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 @@ static class QueuePusher implements ForkJoinPool.ManagedBlocker { 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 boolean block() throws InterruptedException 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 boolean isReleasable() return item == null; } - public void offer(E item) + public void offer(ResultBatch item) { try { this.item = item; @@ -828,6 +867,11 @@ public void offer(E item) throw new RuntimeException("Failed to offer result to output queue", e); } } + + public void tryOfferTerminal() + { + this.queue.offer(ResultBatch.terminal()); + } } /** @@ -837,8 +881,10 @@ public void offer(E item) */ 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 @@ private ResultBatch() 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 @@ static class SequenceBatcher implements ForkJoinPool.ManagedBlocker Yielder> getBatchYielder() { try { + batchYielder = null; ForkJoinPool.managedBlock(this); return batchYielder; } @@ -1033,8 +1077,8 @@ static class YielderBatchedResultsCursor extends BatchedResultsCursor @Override public void initialize() { - yielder = batcher.getBatchYielder(); - resultBatch = yielder.get(); + yielder = null; + nextBatch(); } @Override @@ -1059,6 +1103,10 @@ public boolean isDone() @Override public boolean block() { + if (yielder == null) { + yielder = batcher.getBatchYielder(); + resultBatch = yielder.get(); + } if (yielder.isDone()) { return true; } @@ -1073,7 +1121,7 @@ public boolean block() @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 void close() throws IOException 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 @@ static class BlockingQueueuBatchedResultsCursor extends BatchedResultsCursor< { super(ordering); this.queue = blockingQueue; + this.gizmo = cancellationGizmo; this.hasTimeout = hasTimeout; this.timeoutAtNanos = timeoutAtNanos; } @@ -1142,17 +1193,18 @@ public boolean block() throws InterruptedException { 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 boolean isReleasable() } // 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 long getSlowestPartitionInitializedTime() */ static class MergeCombineMetricsAccumulator { - List partitionMetrics; - MergeCombineActionMetricsAccumulator mergeMetrics; + List partitionMetrics = Collections.emptyList(); + MergeCombineActionMetricsAccumulator mergeMetrics = new MergeCombineActionMetricsAccumulator(); private long totalWallTime; @@ -1343,8 +1451,8 @@ MergeCombineMetrics build() // 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/metadata/MetadataStorageActionHandler.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index 45bfde49a8af..4a5a48e8ef78 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -20,6 +20,8 @@ package org.apache.druid.metadata; import com.google.common.base.Optional; +import org.apache.druid.error.DruidException; +import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.indexer.TaskIdentifier; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.metadata.TaskLookup.TaskLookupType; @@ -31,6 +33,7 @@ import java.util.List; import java.util.Map; +@ExtensionPoint public interface MetadataStorageActionHandler { /** @@ -161,21 +164,34 @@ default List> getTaskInfos( void removeTasksOlderThan(long timestamp); /** - * Add a log to the entry with the given id. + * Task logs are not used anymore and this method is never called by Druid code. + * It has been retained only for backwards compatibility with older extensions. + * New extensions must not implement this method. * - * @param entryId entry id - * @param log log to add - * @return true if the log was added + * @throws DruidException of category UNSUPPORTED whenever called. */ - boolean addLog(String entryId, LogType log); + @Deprecated + default boolean addLog(String entryId, LogType log) + { + throw DruidException.defensive() + .ofCategory(DruidException.Category.UNSUPPORTED) + .build("Task actions are not logged anymore."); + } /** - * Returns the logs for the entry with the given id. + * Task logs are not used anymore and this method is never called by Druid code. + * It has been retained only for backwards compatibility with older extensions. + * New extensions must not implement this method. * - * @param entryId entry id - * @return list of logs + * @throws DruidException of category UNSUPPORTED whenever called. */ - List getLogs(String entryId); + @Deprecated + default List getLogs(String entryId) + { + throw DruidException.defensive() + .ofCategory(DruidException.Category.UNSUPPORTED) + .build("Task actions are not logged anymore."); + } /** * Returns the locks for the given entry @@ -188,7 +204,7 @@ default List> getTaskInfos( /** * Returns the lock id for the given entry and the lock. * - * @return lock id if found. Otherwise null. + * @return lock id if found, otherwise null. */ @Nullable Long getLockId(String entryId, LockType lock); diff --git a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerTypes.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerTypes.java index 79098635ca7f..a3597d988c1a 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerTypes.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandlerTypes.java @@ -25,6 +25,5 @@ public interface MetadataStorageActionHandlerTypes getEntryType(); TypeReference getStatusType(); - TypeReference getLogType(); TypeReference getLockType(); } diff --git a/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java b/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java index 75bbb32431c4..6bd4280c5a3d 100644 --- a/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java +++ b/processing/src/main/java/org/apache/druid/metadata/TaskLookup.java @@ -43,6 +43,11 @@ enum TaskLookupType COMPLETE } + static TaskLookup activeTasksOnly() + { + return ActiveTaskLookup.getInstance(); + } + /** * Whether this lookup is guaranteed to not return any tasks. */ 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 7579843636d0..824444410c08 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 static class ScanQueryBuilder 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 ScanQuery build() orderBy, dimFilter, columns, - legacy, context, columnTypes ); @@ -861,7 +859,6 @@ public static ScanQueryBuilder copy(ScanQuery query) .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 ScanQueryBuilder columns(String... c) 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/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index c1c55a826b10..c90b651fffa3 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 @@ 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 Aggregator factorize(ColumnSelectorFactory metricFactory) 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 BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) 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 dad5cfd98b30..ff863f764b7d 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 static Set filterShards( 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/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index d69e09c9ff0b..3f2faf3fc562 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 @@ -828,7 +828,11 @@ public Optional> resultsAsFrames( boolean useNestedForUnknownTypes ) { - RowSignature rowSignature = resultArraySignature(query); + RowSignature rowSignature = query.getResultRowSignature( + query.context().isFinalize(true) + ? RowSignature.Finalization.YES + : RowSignature.Finalization.NO + ); RowSignature modifiedRowSignature = useNestedForUnknownTypes ? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature) : rowSignature; diff --git a/processing/src/main/java/org/apache/druid/query/operator/Operator.java b/processing/src/main/java/org/apache/druid/query/operator/Operator.java index a9a18c36d547..57bc1013fc44 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/Operator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/Operator.java @@ -126,7 +126,7 @@ enum Signal */ STOP, /** - * Inidcates that the downstream processing should pause its pushing of results and instead return a + * Indicates that the downstream processing should pause its pushing of results and instead return a * continuation object that encapsulates whatever state is required to resume processing. When this signal is * received, Operators that are generating data might choose to exert backpressure or otherwise pause their * processing efforts until called again with the returned continuation object. diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ComposingProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/ComposingProcessor.java index a4fa74967f61..0e0fc59498c0 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/ComposingProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/ComposingProcessor.java @@ -23,7 +23,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.query.rowsandcols.RowsAndColumns; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; public class ComposingProcessor implements Processor { @@ -37,6 +39,16 @@ public ComposingProcessor( this.processors = processors; } + @Override + public List getOutputColumnNames() + { + List outputColumnNames = new ArrayList<>(); + for (Processor processor : processors) { + outputColumnNames.addAll(processor.getOutputColumnNames()); + } + return outputColumnNames; + } + @JsonProperty("processors") public Processor[] getProcessors() { diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java b/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java index fe8d125cbdf3..b271d3064efa 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java @@ -31,6 +31,8 @@ import org.apache.druid.query.operator.window.value.WindowOffsetProcessor; import org.apache.druid.query.rowsandcols.RowsAndColumns; +import java.util.List; + /** * A Processor is a bit of logic that processes a single RowsAndColumns object to produce a new RowsAndColumns * object. Generally speaking, it is used to add or alter columns in a batch-oriented fashion. @@ -80,4 +82,9 @@ public interface Processor * @return boolean identifying if these processors should be considered equivalent to each other. */ boolean validateEquivalent(Processor otherProcessor); + + /** + * @return List of output column names for the Processor. + */ + List getOutputColumnNames(); } 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 fca50c25b282..2dd827d323e1 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 @@ 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; - } + @JsonCreator + public OffsetFrame( + @JsonProperty("lowerOffset") Integer lowerOffset, + @JsonProperty("upperOffset") Integer upperOffset) + { + this.lowerOffset = lowerOffset; + this.upperOffset = upperOffset; + } - @JsonProperty("uppUnbounded") - public boolean isUpperUnbounded() - { - return upperUnbounded; - } + @Override + public int hashCode() + { + return Objects.hash(lowerOffset, upperOffset); + } - @JsonProperty("uppOffset") - public int getUpperOffset() - { - return upperOffset; - } + /** + * 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); + } - @JsonProperty("orderBy") - public List getOrderBy() - { - return orderBy; + /** + * 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(); } - @Override - public boolean equals(Object o) + class Rows extends OffsetFrame { - if (this == o) { - return true; + @JsonCreator + public Rows( + @JsonProperty("lowerOffset") Integer lowerOffset, + @JsonProperty("upperOffset") Integer upperOffset) + { + super(lowerOffset, upperOffset); } - if (!(o instanceof WindowFrame)) { - return false; + + @Override + public String toString() + { + return "WindowFrame.Rows [" + + "lowerOffset=" + lowerOffset + + ", upperOffset=" + 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() + class Groups extends OffsetFrame { - return Objects.hash(peerType, lowerUnbounded, lowerOffset, upperUnbounded, upperOffset, orderBy); - } + @JsonProperty + private final ImmutableList orderByColumns; - @Override - public String toString() - { - return "WindowFrame{" + - "peerType=" + peerType + - ", lowerUnbounded=" + lowerUnbounded + - ", lowerOffset=" + lowerOffset + - ", upperUnbounded=" + upperUnbounded + - ", upperOffset=" + upperOffset + - ", orderBy=" + orderBy + - '}'; - } + @JsonCreator + public Groups( + @JsonProperty("lowerOffset") Integer lowerOffset, + @JsonProperty("upperOffset") Integer upperOffset, + @JsonProperty("orderByColumns") List orderByColumns) + { + super(lowerOffset, upperOffset); + this.orderByColumns = ImmutableList.copyOf(orderByColumns); + } - public static WindowFrame forOrderBy(ColumnWithDirection... orderBy) - { - return new WindowFrame(PeerType.RANGE, true, 0, false, 0, Lists.newArrayList(orderBy)); - } + public List getOrderByColumns() + { + return orderByColumns; + } - public List getOrderByColNames() - { - if (orderBy == null) { - return Collections.emptyList(); + @Override + public int hashCode() + { + return Objects.hash(lowerOffset, orderByColumns, upperOffset); } - return orderBy.stream().map(ColumnWithDirection::getColumn).collect(Collectors.toList()); - } - /** - * Calculates the applicable lower offset if the max number of rows is known. - */ - public int getLowerOffsetClamped(int maxRows) - { - if (lowerUnbounded) { - return -maxRows; + @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 + "]"; } - return Math.max(-maxRows, lowerOffset); } - /** - * Calculates the applicable upper offset if the max number of rows is known. - */ - public int getUpperOffsetClamped(int maxRows) + @SuppressWarnings("unchecked") + @Nullable + default T unwrap(Class clazz) { - if (upperUnbounded) { - return maxRows; + if (clazz.isInstance(this)) { + return (T) this; } - return Math.min(maxRows, upperOffset); + return null; } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java index 3545c3740f40..41baced4e611 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java @@ -27,7 +27,9 @@ import org.apache.druid.query.rowsandcols.semantic.FramedOnHeapAggregatable; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.Objects; public class WindowFramedAggregateProcessor implements Processor @@ -45,6 +47,16 @@ private static T[] emptyToNull(T[] arr) private final WindowFrame frame; private final AggregatorFactory[] aggregations; + @Override + public List getOutputColumnNames() + { + List outputColumnNames = new ArrayList<>(); + for (AggregatorFactory aggregation : aggregations) { + outputColumnNames.add(aggregation.getName()); + } + return outputColumnNames; + } + @JsonCreator public WindowFramedAggregateProcessor( @JsonProperty("frame") WindowFrame frame, diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java index 541c1399e36e..b7f77d509694 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java @@ -28,12 +28,20 @@ import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import java.util.Arrays; +import java.util.Collections; +import java.util.List; public class WindowPercentileProcessor implements Processor { private final int numBuckets; private final String outputColumn; + @Override + public List getOutputColumnNames() + { + return Collections.singletonList(outputColumn); + } + @JsonCreator public WindowPercentileProcessor( @JsonProperty("outputColumn") String outputColumn, diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java index fb5bedf9519f..4e026cbdd3db 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java @@ -27,6 +27,7 @@ import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; +import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.function.Function; @@ -124,4 +125,9 @@ public boolean equals(Object obj) return Objects.equals(groupingCols, other.groupingCols) && Objects.equals(outputColumn, other.outputColumn); } + @Override + public List getOutputColumnNames() + { + return Collections.singletonList(outputColumn); + } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java index 7821e3fd53b4..98b09b6f80d1 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java @@ -28,6 +28,9 @@ import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import org.apache.druid.segment.column.ColumnType; +import java.util.Collections; +import java.util.List; + public class WindowRowNumberProcessor implements Processor { private final String outputColumn; @@ -128,4 +131,10 @@ public String toString() "outputColumn='" + outputColumn + '\'' + '}'; } + + @Override + public List getOutputColumnNames() + { + return Collections.singletonList(outputColumn); + } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java b/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java index 2e084ae983a8..93a7ccd9a5bb 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java @@ -26,6 +26,8 @@ import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; +import java.util.Collections; +import java.util.List; import java.util.function.Function; public abstract class WindowValueProcessorBase implements Processor @@ -100,4 +102,10 @@ protected String internalToString() return "inputColumn=" + inputColumn + ", outputColumn='" + outputColumn + '\''; } + + @Override + public List getOutputColumnNames() + { + return Collections.singletonList(outputColumn); + } } 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 04f9eddbff0c..05b9dee5458c 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 T as(Class clazz) @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/DefaultColumnSelectorFactoryMaker.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java index 3c6d3cc08c9b..9cc87be78e7a 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java @@ -41,7 +41,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -209,7 +208,8 @@ public PassThroughColumnValueSelector( myClazz = float.class; break; case ARRAY: - myClazz = List.class; + myClazz = Object[].class; + break; default: throw DruidException.defensive("this class cannot handle type [%s]", columnAccessor.getType()); } 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 839528730506..7130fafd8671 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 @@ 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.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 void appendTo(AppendableRowsAndColumns rac) 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 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 frame) + 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 @@ private static Iterable buildRowIteratorFor(AppendableRowsAndColumn 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 @@ static class GroupIteratorForWindowFrame implements Iterable // 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 ac36b106c642..cde8475383f8 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 static Order fromString(String name) 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 ScanQuery( @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 ScanQuery( ); this.dimFilter = dimFilter; this.columns = columns; - this.legacy = legacy; this.columnTypes = columnTypes; if (columnTypes != null) { @@ -446,21 +443,6 @@ public List getColumnTypes() return columnTypes; } - /** - * 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. - */ - @Nullable - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Boolean isLegacy() - { - return legacy; - } - @Override public Ordering getResultOrdering() { @@ -507,11 +489,6 @@ public ScanQuery withLimit(final long newLimit) 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 +523,6 @@ public boolean equals(Object o) 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 +542,7 @@ public int hashCode() scanRowsLimit, dimFilter, columns, - orderBys, - legacy + orderBys ); } @@ -585,7 +560,6 @@ public String toString() ", dimFilter=" + dimFilter + ", columns=" + columns + (orderBys.isEmpty() ? "" : ", orderBy=" + orderBys) + - (legacy == null ? "" : ", legacy=" + legacy) + ", context=" + getContext() + '}'; } @@ -710,12 +684,6 @@ public boolean equals(Object obj) */ @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 +699,7 @@ public RowSignature getRowSignature(boolean defaultIsLegacy) } 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 b92697282609..aca2f00446f0 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 boolean equals(final Object o) 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 2246e349cbb4..567e07fbdc73 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.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.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 @@ public class ScanQueryEngine { - static final String LEGACY_TIMESTAMP_KEY = "timestamp"; - public Sequence process( final ScanQuery query, final Segment segment, @@ -68,10 +66,6 @@ public Sequence process( @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 Sequence process( 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 Sequence process( } 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 Sequence process( ); allColumns.addAll(availableColumns); - - if (legacy) { - allColumns.remove(ColumnHolder.TIME_COLUMN_NAME); - } } final List intervals = query.getQuerySegmentSpec().getIntervals(); @@ -149,28 +136,15 @@ public Iterator make() { 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 @@ private List> rowsToList() 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 63722db74afb..717a40d1062e 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 mergeResults(final QueryRunner results; @@ -154,8 +148,7 @@ public QueryRunner preMergeQueryDecoration(final QueryRunner 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 67c36fe76030..3ad58270f8b7 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.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 Function, Result> ma @Override public RowSignature resultArraySignature(TimeseriesQuery query) { - RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); - rowSignatureBuilder.addTimeColumn(); - if (StringUtils.isNotEmpty(query.getTimestampResultField())) { - rowSignatureBuilder.add(query.getTimestampResultField(), ColumnType.LONG); - } - rowSignatureBuilder.addAggregators(query.getAggregatorSpecs(), RowSignature.Finalization.UNKNOWN); - rowSignatureBuilder.addPostAggregators(query.getPostAggregatorSpecs()); - return rowSignatureBuilder.build(); + return query.getResultSignature(RowSignature.Finalization.UNKNOWN); } @Override @@ -486,7 +478,10 @@ public Optional> resultsAsFrames( boolean useNestedForUnknownTypes ) { - final RowSignature rowSignature = resultArraySignature(query); + final RowSignature rowSignature = + query.getResultSignature( + query.context().isFinalize(true) ? RowSignature.Finalization.YES : RowSignature.Finalization.NO + ); final Pair 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 349e5a02d163..21729022d616 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.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 void initTopNAlgorithmSelector(TopNAlgorithmSelector selector) 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 21bc336438ac..c5f195615f36 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 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 DimensionAndMetricValueExtractor apply( @Override public RowSignature resultArraySignature(TopNQuery query) { - return RowSignature.builder() - .addTimeColumn() - .addDimensions(Collections.singletonList(query.getDimensionSpec())) - .addAggregators(query.getAggregatorSpecs(), RowSignature.Finalization.UNKNOWN) - .addPostAggregators(query.getPostAggregatorSpecs()) - .build(); + return query.getResultSignature(RowSignature.Finalization.UNKNOWN); } @Override @@ -569,7 +563,10 @@ public Optional> resultsAsFrames( boolean useNestedForUnknownTypes ) { - final RowSignature rowSignature = resultArraySignature(query); + final RowSignature rowSignature = query.getResultSignature( + query.context().isFinalize(true) ? RowSignature.Finalization.YES : RowSignature.Finalization.NO + ); + final Pair cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence( resultsAsArrays(query, resultSequence), rowSignature 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 e6b5b1f65883..c3ac554a0faf 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 @@ 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/nested/DictionaryIdLookup.java b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java index f4176db220cd..6827497f7a6b 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 DictionaryIdLookup( this.arrayDictionaryWriter = arrayDictionaryWriter; } - public int lookupString(@Nullable String value) + @Nullable + public Object getDictionaryValue(int id) { - 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(); + 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) + { + 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 int lookupString(@Nullable String value) 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 int lookupLong(@Nullable Long value) 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 int lookupDouble(@Nullable Double value) 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 @@ private int arrayOffset() 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 aa6a71ae7545..d9f00bb2321f 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 @@ T processValue(int row, Object value) } /** - * 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 void addValue(int row, Object val) throws IOException localId = localDictionary.add(globalId); } intermediateValueWriter.write(localId); - writeValue(value); cursorPosition++; } @@ -168,11 +167,9 @@ public void addValue(int row, Object val) throws IOException */ 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 void writeTo(int finalRowCount, FileSmoosher smoosher) throws IOException 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 void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws I } } - 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 8ccd528715bf..09e8dc121c8e 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 @@ -59,12 +59,22 @@ int lookupGlobalId(Double value) } @Override - public void open() throws IOException + void writeValue(@Nullable Double value) throws IOException + { + if (value == null) { + doublesSerializer.add(0.0); + } else { + doublesSerializer.add(value); + } + } + + @Override + public void openColumnSerializer(SegmentWriteOutMedium medium, int maxId) throws IOException { - super.open(); + super.openColumnSerializer(medium, maxId); doublesSerializer = CompressionFactory.getDoubleSerializer( fieldName, - segmentWriteOutMedium, + medium, StringUtils.format("%s.double_column", fieldName), ByteOrder.nativeOrder(), indexSpec.getDimensionCompression(), @@ -73,16 +83,6 @@ public void open() throws IOException doublesSerializer.open(); } - @Override - void writeValue(@Nullable Double value) throws IOException - { - if (value == null) { - doublesSerializer.add(0.0); - } else { - doublesSerializer.add(value); - } - } - @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 66b5eca18d9f..d9191c4e8050 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 @@ -59,12 +59,22 @@ int lookupGlobalId(Long value) } @Override - public void open() throws IOException + void writeValue(@Nullable Long value) throws IOException + { + if (value == null) { + longsSerializer.add(0L); + } else { + longsSerializer.add(value); + } + } + + @Override + public void openColumnSerializer(SegmentWriteOutMedium medium, int maxId) throws IOException { - super.open(); + super.openColumnSerializer(medium, maxId); longsSerializer = CompressionFactory.getLongSerializer( fieldName, - segmentWriteOutMedium, + medium, StringUtils.format("%s.long_column", fieldName), ByteOrder.nativeOrder(), indexSpec.getLongEncoding(), @@ -74,16 +84,6 @@ public void open() throws IOException longsSerializer.open(); } - @Override - void writeValue(@Nullable Long value) throws IOException - { - if (value == null) { - longsSerializer.add(0L); - } else { - longsSerializer.add(value); - } - } - @Override void writeColumnTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java index ca2cda3e0e14..979d213ee411 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java @@ -475,15 +475,15 @@ static Supplier supplierFromObjectSelector( } final Class clazz = selector.classOfObject(); - if (Number.class.isAssignableFrom(clazz) || String.class.isAssignableFrom(clazz)) { - // Number, String supported as-is. + if (Number.class.isAssignableFrom(clazz) || String.class.isAssignableFrom(clazz) || Object[].class.isAssignableFrom(clazz)) { + // Number, String, Arrays supported as-is. return selector::getObject; } else if (clazz.isAssignableFrom(Number.class) || clazz.isAssignableFrom(String.class)) { // Might be Numbers and Strings. Use a selector that double-checks. return () -> { final Object val = selector.getObject(); if (val instanceof List) { - NonnullPair coerced = ExprEval.coerceListToArray((List) val, homogenizeMultiValue); + NonnullPair coerced = ExprEval.coerceListToArray((List) val, homogenizeMultiValue); if (coerced == null) { return null; } @@ -496,7 +496,7 @@ static Supplier supplierFromObjectSelector( return () -> { final Object val = selector.getObject(); if (val != null) { - NonnullPair coerced = ExprEval.coerceListToArray((List) val, homogenizeMultiValue); + NonnullPair coerced = ExprEval.coerceListToArray((List) val, homogenizeMultiValue); if (coerced == null) { return null; } 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 98588de1d534..77188db58576 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 HashPartitionFunction getPartitionFunction() @Override public 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 52b3069f0b4e..3989340c7b4d 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 NumberedShardSpec convert(int numTotalPartitions) @Override public 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/data/input/impl/InputEntityIteratingReaderTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java index 744c29dba2a1..80602d0508ad 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 void testSampleWithSystemFields() throws IOException @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 @@ protected int getMaxRetries() 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 ca34c364dca8..5b76afb90229 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 void testOrderedResultBatchFromSequenceBacktoYielderOnSequence() throws I 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 void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOE 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 void testTimeoutExceptionDueToStalledInput() } @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 void testTimeoutExceptionDueToStalledReader() 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 @@ private void assertResultWithCustomPool( 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 @@ private void assertResult( 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 @@ private void assertResultWithEarlyClose( } } // 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 @@ private static IntPair makeIntPair(int mergeKey) { 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/metadata/MetadataStorageActionHandlerTest.java b/processing/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java new file mode 100644 index 000000000000..41a0a55b284f --- /dev/null +++ b/processing/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.metadata; + +import com.google.common.base.Optional; +import org.apache.druid.error.DruidException; +import org.apache.druid.indexer.TaskIdentifier; +import org.apache.druid.indexer.TaskInfo; +import org.joda.time.DateTime; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Tests the default methods of the interface {@link MetadataStorageActionHandler}. + * Required only for coverage as these methods are already being tested in + * {@code SQLMetadataStorageActionHandlerTest}. + */ +public class MetadataStorageActionHandlerTest +{ + + private MetadataStorageActionHandler handler; + + @Before + public void setup() + { + this.handler = new MetadataStorageActionHandler() + { + @Override + public void insert( + String id, + DateTime timestamp, + String dataSource, + String entry, + boolean active, + @Nullable String status, + String type, + String groupId + ) + { + + } + + @Override + public boolean setStatus(String entryId, boolean active, String status) + { + return false; + } + + @Override + public Optional getEntry(String entryId) + { + return null; + } + + @Override + public Optional getStatus(String entryId) + { + return null; + } + + @Nullable + @Override + public TaskInfo getTaskInfo(String entryId) + { + return null; + } + + @Override + public List> getTaskInfos( + Map taskLookups, + @Nullable String datasource + ) + { + return Collections.emptyList(); + } + + @Override + public List> getTaskStatusList( + Map taskLookups, + @Nullable String datasource + ) + { + return Collections.emptyList(); + } + + @Override + public boolean addLock(String entryId, String lock) + { + return false; + } + + @Override + public boolean replaceLock(String entryId, long oldLockId, String newLock) + { + return false; + } + + @Override + public void removeLock(long lockId) + { + + } + + @Override + public void removeTasksOlderThan(long timestamp) + { + + } + + @Override + public Map getLocks(String entryId) + { + return Collections.emptyMap(); + } + + @Override + public Long getLockId(String entryId, String lock) + { + return 0L; + } + + @Override + public void populateTaskTypeAndGroupIdAsync() + { + + } + }; + } + + @Test + public void testAddLogThrowsUnsupportedException() + { + Exception exception = Assert.assertThrows( + DruidException.class, + () -> handler.addLog("abcd", "logentry") + ); + Assert.assertEquals( + "Task actions are not logged anymore.", + exception.getMessage() + ); + } + + @Test + public void testGetLogsThrowsUnsupportedException() + { + Exception exception = Assert.assertThrows( + DruidException.class, + () -> handler.getLogs("abcd") + ); + Assert.assertEquals( + "Task actions are not logged anymore.", + exception.getMessage() + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java b/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java index 76d90737a5f5..fe372558b64d 100644 --- a/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java +++ b/processing/src/test/java/org/apache/druid/metadata/TaskLookupTest.java @@ -130,7 +130,7 @@ public void testSingleton() @Test public void testGetType() { - Assert.assertEquals(TaskLookupType.ACTIVE, ActiveTaskLookup.getInstance().getType()); + Assert.assertEquals(TaskLookupType.ACTIVE, TaskLookup.activeTasksOnly().getType()); } } } 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 44d5725ba3d7..03b5fe7e45e3 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 @@ private Druids.ScanQueryBuilder newTestQuery() .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 2ad9f90148a8..e21b185d5cdb 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 static AggregationTestHelper createScanQueryAggregationTestHelper( ObjectMapper mapper = TestHelper.makeJsonMapper(); ScanQueryQueryToolChest toolchest = new ScanQueryQueryToolChest( - new ScanQueryConfig(), DefaultGenericQueryMetricsFactory.instance() ); 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 421a457999d9..c4df70a88de8 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 @@ 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 void testSerde() throws Exception 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/operator/WindowProcessorOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java index 9cce74cb98cc..c11a50cf5cb0 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java @@ -27,6 +27,9 @@ import org.junit.Assert; import org.junit.Test; +import java.util.Collections; +import java.util.List; + public class WindowProcessorOperatorTest { @Test @@ -53,6 +56,12 @@ public boolean validateEquivalent(Processor otherProcessor) { return true; } + + @Override + public List getOutputColumnNames() + { + return Collections.emptyList(); + } }, InlineScanOperator.make(rac) ); diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ComposingProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ComposingProcessorTest.java index 570cba65d92c..d8f4599eb1ac 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/ComposingProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/ComposingProcessorTest.java @@ -23,6 +23,9 @@ import org.junit.Assert; import org.junit.Test; +import java.util.Collections; +import java.util.List; + public class ComposingProcessorTest { @Test @@ -32,6 +35,7 @@ public void testSanity() final ProcessorForTesting secondProcessor = new ProcessorForTesting(); ComposingProcessor proc = new ComposingProcessor(firstProcessor, secondProcessor); + Assert.assertTrue(proc.getOutputColumnNames().isEmpty()); proc.process(null); Assert.assertEquals(1, firstProcessor.processCounter); @@ -70,5 +74,11 @@ public boolean validateEquivalent(Processor otherProcessor) ++validateCounter; return validationResult; } + + @Override + public List getOutputColumnNames() + { + return Collections.emptyList(); + } } } 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 000000000000..855f4694f430 --- /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 88d79c87cdbc..9bae78bc2ccf 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 @@ -19,6 +19,7 @@ package org.apache.druid.query.operator.window; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.common.config.NullHandling; @@ -45,12 +46,13 @@ 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") }; WindowFramedAggregateProcessor proc = new WindowFramedAggregateProcessor(theFrame, theAggs); + Assert.assertEquals(ImmutableList.of("cummMax", "cummSum"), proc.getOutputColumnNames()); final MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(ImmutableMap.of( "yay", new IntArrayColumn(new int[]{1, 2, 3}) @@ -76,7 +78,7 @@ public T as(Class clazz) @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/operator/window/ranking/WindowCumeDistProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowCumeDistProcessorTest.java index f5914e4f5dbe..877c78415496 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowCumeDistProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowCumeDistProcessorTest.java @@ -25,6 +25,7 @@ import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.junit.Assert; import org.junit.Test; import java.util.Collections; @@ -42,6 +43,7 @@ public void testCumeDistProcessing() MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map); Processor processor = new WindowCumeDistProcessor(Collections.singletonList("vals"), "CumeDist"); + Assert.assertEquals(Collections.singletonList("CumeDist"), processor.getOutputColumnNames()); final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper() .expectColumn("vals", new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290}) diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java index e165f46f0746..86580e5bd2fa 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java @@ -25,6 +25,7 @@ import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.junit.Assert; import org.junit.Test; import java.util.Collections; @@ -42,6 +43,7 @@ public void testDenseRankProcessing() MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map); Processor processor = new WindowDenseRankProcessor(Collections.singletonList("vals"), "DenseRank"); + Assert.assertEquals(Collections.singletonList("DenseRank"), processor.getOutputColumnNames()); final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper() .expectColumn("vals", new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290}) diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java index c38cd2a245c1..bf5bb727b0a0 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.query.operator.window.ranking; +import com.google.common.collect.ImmutableList; import org.apache.druid.query.operator.window.ComposingProcessor; import org.apache.druid.query.operator.window.Processor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; @@ -29,6 +30,7 @@ import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; import org.apache.druid.segment.column.ColumnType; +import org.junit.Assert; import org.junit.Test; import java.util.LinkedHashMap; @@ -63,6 +65,11 @@ public void testPercentileProcessing() new WindowPercentileProcessor("10292", 10292) ); + Assert.assertEquals( + ImmutableList.of("1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "10292"), + processor.getOutputColumnNames() + ); + final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper() .expectColumn("intCol", new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9}) .expectColumn("doubleCol", new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9}) diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java index 59c7dd6df363..b7f281c423eb 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.query.operator.window.ranking; +import com.google.common.collect.ImmutableList; import org.apache.druid.query.operator.window.ComposingProcessor; import org.apache.druid.query.operator.window.Processor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; @@ -26,6 +27,7 @@ import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.junit.Assert; import org.junit.Test; import java.util.Collections; @@ -49,6 +51,8 @@ public void testRankProcessing() new WindowRankProcessor(orderingCols, "rankAsPercent", true) ); + Assert.assertEquals(ImmutableList.of("rank", "rankAsPercent"), processor.getOutputColumnNames()); + final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper() .expectColumn("vals", new int[]{7, 18, 18, 30, 120, 121, 122, 122, 8290, 8290}) .expectColumn("rank", new int[]{1, 2, 2, 4, 5, 6, 7, 7, 9, 9}) diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java index 937fea7c3605..f4f9b5bfeee4 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java @@ -28,8 +28,10 @@ import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; import org.apache.druid.segment.column.ColumnType; +import org.junit.Assert; import org.junit.Test; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.Map; @@ -49,6 +51,7 @@ public void testRowNumberProcessing() MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(map); Processor processor = new WindowRowNumberProcessor("rowRow"); + Assert.assertEquals(Collections.singletonList("rowRow"), processor.getOutputColumnNames()); final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper() .expectColumn("intCol", new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9}) diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java index 67242f055033..eb6caa10a0b5 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.query.operator.window.value; +import com.google.common.collect.ImmutableList; import org.apache.druid.query.operator.window.ComposingProcessor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; @@ -28,6 +29,7 @@ import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; import org.apache.druid.segment.column.ColumnType; +import org.junit.Assert; import org.junit.Test; import java.util.LinkedHashMap; @@ -59,6 +61,11 @@ public void testFirstProcessing() new WindowFirstProcessor("nullFirstCol", "NullFirstCol") ); + Assert.assertEquals( + ImmutableList.of("FirstIntCol", "FirstDoubleCol", "FirstObjectCol", "NullFirstCol"), + processor.getOutputColumnNames() + ); + final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper() .expectColumn("intCol", new int[]{88, 1, 2, 3, 4, 5, 6, 7, 8, 9}) .expectColumn("doubleCol", new double[]{0.4728, 1, 2, 3, 4, 5, 6, 7, 8, 9}) diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java index 5aa212b6acb2..1910401f34a7 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.query.operator.window.value; +import com.google.common.collect.ImmutableList; import org.apache.druid.query.operator.window.ComposingProcessor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; @@ -28,6 +29,7 @@ import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; import org.apache.druid.segment.column.ColumnType; +import org.junit.Assert; import org.junit.Test; import java.util.LinkedHashMap; @@ -58,6 +60,10 @@ public void testLastProcessing() new WindowLastProcessor("objectCol", "LastObjectCol"), new WindowLastProcessor("nullLastCol", "NullLastCol") ); + Assert.assertEquals( + ImmutableList.of("LastIntCol", "LastDoubleCol", "LastObjectCol", "NullLastCol"), + processor.getOutputColumnNames() + ); final RowsAndColumnsHelper expectations = new RowsAndColumnsHelper() 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 d5b11f7a612a..41ceb315a04f 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.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 void testWindowedAggregationWindowSmallerThanRowsNoOffsets() 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 void testWindowedAggregationWindowSmallerThanRows() 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 void testWindowedAggregationWindowSmallerThanRowsOnlyUpper() 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 void testWindowedAggregationWindowSmallerThanRowsOnlyLower() 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 void testWindowedAggregationWindowLargerThanRows() 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 void testWindowedAggregationLowerLargerThanRows() 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 void testWindowedAggregationLowerLargerThanRowsNoUpper() 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 void testWindowedAggregationUpperLargerThanRows() 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 void testWindowedAggregationUpperLargerThanRowsNoLower() 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 void testWindowedAggregationWindowLargerThanRowsOnlyUpper() 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 void testWindowedAggregationWindowLargerThanRowsOnlyLower() 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 void testUnboundedWindowedAggregation() 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 void testCumulativeAggregation() 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 void testReverseCumulativeAggregation() 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 void testReverseCumulativeAggregation() @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 void testRangeOrderBy() @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 void testRangeB1() @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 void testRangeA1() @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 void testRangeB1A1() @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 void testRangeB1A1_2() @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 cc276c663287..e3ea3b1ef6d2 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 @@ public class MultiSegmentScanQueryTest extends InitializedNullHandlingTest { private static final ScanQueryQueryToolChest TOOL_CHEST = new ScanQueryQueryToolChest( - new ScanQueryConfig(), DefaultGenericQueryMetricsFactory.instance() ); @@ -193,7 +192,6 @@ private Druids.ScanQueryBuilder newBuilder() .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 7a71c44e28c2..f4214c194567 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 void testSerde() 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 63c5b3819216..76e7f6bdd3ea 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 void test_resultArraySignature_columnsNotSpecifiedLegacyMode() 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 void test_resultArraySignature_columnsSpecified() ); } - @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 fee5b4b9de72..5b155aa4a983 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 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 1c9a6c67ded8..55c23a8d8c99 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 int getMaxSegmentPartitionsOrderedInMemory() }; 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 b003f9580e8f..f2c3a9ea18f5 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.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 Object getResult() 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 Object getResult() 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 @@ private Druids.ScanQueryBuilder newTestQuery() .dataSource(new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE)) .columns(Collections.emptyList()) .eternityInterval() - .limit(3) - .legacy(legacy); + .limit(3); } @Test @@ -270,7 +264,7 @@ public void testSelectWithUnderscoreUnderscoreTime() final List>> expectedEvents = toEvents( new String[]{ - getTimestampName() + ":TIME", + ColumnHolder.TIME_COLUMN_NAME + ":TIME", QueryRunnerTestHelper.MARKET_DIMENSION + ":STRING", null, null, @@ -284,24 +278,12 @@ public void testSelectWithUnderscoreUnderscoreTime() 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 void testSelectWithDimsAndMets() List expectedResults = toExpected( toEvents( new String[]{ - legacy ? getTimestampName() + ":TIME" : null, + null, QueryRunnerTestHelper.MARKET_DIMENSION + ":STRING", null, null, @@ -335,10 +317,9 @@ public void testSelectWithDimsAndMets() 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 void testSelectWithDimsAndMetsAsCompactedList() List expectedResults = toExpected( toEvents( new String[]{ - legacy ? getTimestampName() + ":TIME" : null, + null, QueryRunnerTestHelper.MARKET_DIMENSION + ":STRING", null, null, @@ -373,10 +354,9 @@ public void testSelectWithDimsAndMetsAsCompactedList() 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 void testFullOnSelectWithFilterAndLimit() 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 void testFullOnSelectWithFilterAndLimit() 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 void testSelectWithFilterLookupExtractionFn() 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 void testSelectWithFilterLookupExtractionFn() 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 void testFullSelectNoDimensionAndMetric() 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 void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingListFormat() }; 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 void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingListFormat() 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 void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingCompactedList 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 void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingCompactedLis 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 @@ private List>> toFullEvents(final String[]... valueSet) { 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 @@ private List>> toFullEvents(final String[]... valueSet) "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 static List>> toEvents(final String[] dimSpecs, 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 static List>> toEvents(final String[] dimSpecs, 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 1eaa299927fa..1bc1ff18a58d 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 @@ -67,7 +67,6 @@ public void testSerialization() throws Exception null, Arrays.asList("market", "quality", "index"), null, - null, null ); @@ -102,7 +101,6 @@ public void testSerializationWithTimeOrder() throws Exception null, Arrays.asList("market", "quality", "index", "__time"), null, - null, null ); @@ -141,7 +139,6 @@ public void testSerializationWithOrderBy() throws Exception null, Arrays.asList("market", "quality", "index", "__time"), null, - null, null ); @@ -171,7 +168,6 @@ public void testSerializationLegacyString() throws Exception null, Arrays.asList("market", "quality", "index"), null, - null, null ); 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 8238786998c3..23d7f7a96af8 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.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 @@ private Druids.ScanQueryBuilder newTestUnnestQuery() .dataSource(QueryRunnerTestHelper.UNNEST_DATA_SOURCE) .columns(Collections.emptyList()) .eternityInterval() - .limit(3) - .legacy(legacy); + .limit(3); } private Druids.ScanQueryBuilder newTestUnnestQueryWithFilterDataSource() @@ -99,8 +73,7 @@ private Druids.ScanQueryBuilder newTestUnnestQueryWithFilterDataSource() .dataSource(QueryRunnerTestHelper.UNNEST_FILTER_DATA_SOURCE) .columns(Collections.emptyList()) .eternityInterval() - .limit(3) - .legacy(legacy); + .limit(3); } @Test @@ -123,38 +96,19 @@ public void testScanOnUnnest() ); 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 void testScanOnUnnestFilterDataSource() ); 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 void testUnnestRunnerVirtualColumnsUsingSingleColumn() )) .columns(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST) .eternityInterval() - .legacy(legacy) .limit(3) .build(); @@ -251,38 +185,19 @@ public void testUnnestRunnerVirtualColumnsUsingSingleColumn() "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 void testUnnestRunnerVirtualColumnsUsingMultipleColumn() )) .columns(QueryRunnerTestHelper.MARKET_DIMENSION, QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST) .eternityInterval() - .legacy(legacy) .limit(4) .build(); @@ -322,46 +236,21 @@ public void testUnnestRunnerVirtualColumnsUsingMultipleColumn() ); 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 void testUnnestRunnerWithFilter() ); 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 void testUnnestRunnerWithOrdering() ); 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 void testUnnestRunnerWithOrdering() 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/virtual/ExpressionSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java index 8224f24e8954..b6b9713682a8 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java @@ -591,7 +591,22 @@ public void test_supplierFromObjectSelector_onList() settableSupplier.set(ImmutableList.of("1", "2", "3")); Assert.assertArrayEquals(new String[]{"1", "2", "3"}, (Object[]) supplier.get()); + } + @Test + public void test_supplierFromObjectSelector_onArray() + { + final SettableSupplier settableSupplier = new SettableSupplier<>(); + final Supplier supplier = ExpressionSelectors.supplierFromObjectSelector( + objectSelectorFromSupplier(settableSupplier, Object[].class), + true + ); + + Assert.assertNotNull(supplier); + Assert.assertEquals(null, supplier.get()); + + settableSupplier.set(new String[]{"1", "2", "3"}); + Assert.assertArrayEquals(new String[]{"1", "2", "3"}, (Object[]) supplier.get()); } @Test 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 5fa34d6699d8..e4027bcd3574 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 @@ private Sequence merge(List> sequencesByInterval) 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 @@ private Sequence merge(List> sequencesByInterval) } } ); + scheduler.registerQueryFuture(query, parallelSequence.getCancellationFuture()); + return parallelSequence; } else { return Sequences .simple(sequencesByInterval) diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryTuningConfig.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryTuningConfig.java index 7b1a7c54682b..55fe7d0114f9 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryTuningConfig.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryTuningConfig.java @@ -28,6 +28,7 @@ import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.joda.time.Duration; @@ -79,6 +80,17 @@ public class ClientCompactionTaskQueryTuningConfig @Nullable private final AppendableIndexSpec appendableIndexSpec; + public static ClientCompactionTaskQueryTuningConfig from( + DataSourceCompactionConfig compactionConfig + ) + { + if (compactionConfig == null) { + return from(null, null, null); + } else { + return from(compactionConfig.getTuningConfig(), compactionConfig.getMaxRowsPerSegment(), null); + } + } + public static ClientCompactionTaskQueryTuningConfig from( @Nullable UserCompactionTaskQueryTuningConfig userCompactionTaskQueryTuningConfig, @Nullable Integer maxRowsPerSegment, 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 fe6461bf2158..000000000000 --- 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 59516f8d4c49..e07ac5ed1155 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.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 CoreInjectorBuilder forServer() 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 6ddfd378fb43..cff9308a6971 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/DerbyMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java index fa39d506b4cc..c3265f388294 100644 --- a/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/DerbyMetadataStorageActionHandler.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import org.apache.druid.java.util.common.StringUtils; public class DerbyMetadataStorageActionHandler extends SQLMetadataStorageActionHandler @@ -46,12 +45,4 @@ protected String decorateSqlWithLimit(String sql) return sql + " FETCH FIRST :n ROWS ONLY"; } - @Deprecated - @Override - public String getSqlRemoveLogsOlderThan() - { - return StringUtils.format("DELETE FROM %s WHERE %s_id in (" - + " SELECT id FROM %s WHERE created_date < :date_time and active = false)", - getLogTable(), getEntryTypeName(), getEntryTable()); - } } diff --git a/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java index b91427bcd662..6bed60b89b41 100644 --- a/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/PostgreSQLMetadataStorageActionHandler.java @@ -20,7 +20,6 @@ package org.apache.druid.metadata; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.java.util.common.StringUtils; public class PostgreSQLMetadataStorageActionHandler extends SQLMetadataStorageActionHandler @@ -44,13 +43,4 @@ protected String decorateSqlWithLimit(String sql) return sql + " LIMIT :n"; } - @Deprecated - @Override - public String getSqlRemoveLogsOlderThan() - { - return StringUtils.format("DELETE FROM %s USING %s " - + "WHERE %s_id = %s.id AND created_date < :date_time and active = false", - getLogTable(), getEntryTable(), getEntryTypeName(), getEntryTable()); - } - } 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 11d467323f43..02c2d4269c2c 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.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/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index dc87b9fc2fd4..e383e6ad87c8 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -518,25 +518,6 @@ private void alterPendingSegmentsTable(final String tableName) ); } - public void createLogTable(final String tableName, final String entryTypeName) - { - createTable( - tableName, - ImmutableList.of( - StringUtils.format( - "CREATE TABLE %1$s (\n" - + " id %2$s NOT NULL,\n" - + " %4$s_id VARCHAR(255) DEFAULT NULL,\n" - + " log_payload %3$s,\n" - + " PRIMARY KEY (id)\n" - + ")", - tableName, getSerialType(), getPayloadType(), entryTypeName - ), - StringUtils.format("CREATE INDEX idx_%1$s_%2$s_id ON %1$s(%2$s_id)", tableName, entryTypeName) - ) - ); - } - public void createLockTable(final String tableName, final String entryTypeName) { createTable( @@ -814,7 +795,6 @@ public void createTaskTables() final MetadataStorageTablesConfig tablesConfig = tablesConfigSupplier.get(); final String entryType = tablesConfig.getTaskEntryType(); prepareTaskEntryTable(tablesConfig.getEntryTable(entryType)); - createLogTable(tablesConfig.getLogTable(entryType), entryType); createLockTable(tablesConfig.getLockTable(entryType), entryType); } } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index 8b003340bed0..6cbefc84eeb4 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -75,12 +75,10 @@ public abstract class SQLMetadataStorageActionHandler entryType; private final TypeReference statusType; - private final TypeReference logType; private final TypeReference lockType; private final String entryTypeName; private final String entryTable; - private final String logTable; private final String lockTable; private final TaskInfoMapper taskInfoMapper; @@ -90,7 +88,11 @@ public abstract class SQLMetadataStorageActionHandler taskMigrationCompleteFuture; - @SuppressWarnings("PMD.UnnecessaryFullyQualifiedName") + /** + * @deprecated Use the other constructor without {@code logTable} argument + * since this argument is now unused. + */ + @Deprecated public SQLMetadataStorageActionHandler( final SQLMetadataConnector connector, final ObjectMapper jsonMapper, @@ -100,6 +102,19 @@ public SQLMetadataStorageActionHandler( final String logTable, final String lockTable ) + { + this(connector, jsonMapper, types, entryTypeName, entryTable, lockTable); + } + + @SuppressWarnings("PMD.UnnecessaryFullyQualifiedName") + public SQLMetadataStorageActionHandler( + final SQLMetadataConnector connector, + final ObjectMapper jsonMapper, + final MetadataStorageActionHandlerTypes types, + final String entryTypeName, + final String entryTable, + final String lockTable + ) { this.connector = connector; //fully qualified references required below due to identical package names across project modules. @@ -108,11 +123,9 @@ public SQLMetadataStorageActionHandler( org.apache.druid.metadata.PasswordProviderRedactionMixIn.class); this.entryType = types.getEntryType(); this.statusType = types.getStatusType(); - this.logType = types.getLogType(); this.lockType = types.getLockType(); this.entryTypeName = entryTypeName; this.entryTable = entryTable; - this.logTable = logTable; this.lockTable = lockTable; this.taskInfoMapper = new TaskInfoMapper<>(jsonMapper, entryType, statusType); this.taskStatusMapper = new TaskStatusMapper(jsonMapper); @@ -142,7 +155,7 @@ protected String getEntryTable() protected String getLogTable() { - return logTable; + throw new UnsupportedOperationException("'tasklogs' table is not used anymore"); } protected String getEntryTypeName() @@ -430,7 +443,7 @@ List> getTaskStatusList( } /** - * Wraps the given error in a user friendly DruidException. + * Wraps the given error in a user-friendly DruidException. */ private DruidException wrapInDruidException(String taskId, Throwable t) { @@ -855,21 +868,13 @@ public void removeTasksOlderThan(final long timestamp) { DateTime dateTime = DateTimes.utc(timestamp); connector.retryWithHandle( - handle -> { - handle.createStatement(getSqlRemoveLogsOlderThan()) - .bind("date_time", dateTime.toString()) - .execute(); + handle -> handle.createStatement( StringUtils.format( "DELETE FROM %s WHERE created_date < :date_time AND active = false", entryTable ) - ) - .bind("date_time", dateTime.toString()) - .execute(); - - return null; - } + ).bind("date_time", dateTime.toString()).execute() ); } @@ -880,78 +885,6 @@ private int removeLock(Handle handle, long lockId) .execute(); } - @Override - public boolean addLog(final String entryId, final LogType log) - { - return connector.retryWithHandle( - new HandleCallback() - { - @Override - public Boolean withHandle(Handle handle) throws Exception - { - return handle.createStatement( - StringUtils.format( - "INSERT INTO %1$s (%2$s_id, log_payload) VALUES (:entryId, :payload)", - logTable, entryTypeName - ) - ) - .bind("entryId", entryId) - .bind("payload", jsonMapper.writeValueAsBytes(log)) - .execute() == 1; - } - } - ); - } - - @Override - public List getLogs(final String entryId) - { - return connector.retryWithHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) - { - return handle - .createQuery( - StringUtils.format( - "SELECT log_payload FROM %1$s WHERE %2$s_id = :entryId", - logTable, entryTypeName - ) - ) - .bind("entryId", entryId) - .map(ByteArrayMapper.FIRST) - .fold( - new ArrayList<>(), - (List list, byte[] bytes, FoldController control, StatementContext ctx) -> { - try { - list.add(jsonMapper.readValue(bytes, logType)); - return list; - } - catch (IOException e) { - log.makeAlert(e, "Failed to deserialize log") - .addData("entryId", entryId) - .addData("payload", StringUtils.fromUtf8(bytes)) - .emit(); - throw new SQLException(e); - } - } - ); - } - } - ); - } - - @Deprecated - public String getSqlRemoveLogsOlderThan() - { - return StringUtils.format( - "DELETE a FROM %s a INNER JOIN %s b ON a.%s_id = b.id " - + "WHERE b.created_date < :date_time and b.active = false", - logTable, entryTable, entryTypeName - ); - } - @Override public Map getLocks(final String entryId) { 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 abc64baae5a6..0d55e0ed7b59 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.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 InputStream open() 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 CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws I * 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 CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws I public static CleanableFile openCleanableFile( String sql, - SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector, + SQLInputSourceDatabaseConnector sqlInputSourceDatabaseConnector, ObjectMapper objectMapper, boolean foldCase, File tempFile @@ -124,7 +124,7 @@ public static CleanableFile openCleanableFile( // 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 static CleanableFile openCleanableFile( 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 8d886b058fca..27147babfa5d 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.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 SqlInputSource( 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 boolean isFoldCase() } @JsonProperty("database") - public SQLFirehoseDatabaseConnector getSQLFirehoseDatabaseConnector() + public SQLInputSourceDatabaseConnector getSQLInputSourceDatabaseConnector() { - return sqlFirehoseDatabaseConnector; + return sqlInputSourceDatabaseConnector; } @Override @@ -118,7 +118,7 @@ public SplittableInputSource withSplit(InputSplit split) return new SqlInputSource( Collections.singletonList(split.get()), foldCase, - sqlFirehoseDatabaseConnector, + sqlInputSourceDatabaseConnector, objectMapper ); } @@ -131,7 +131,8 @@ protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nu 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 boolean equals(Object o) 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 36d84f2dc4e7..057201b8564c 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 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/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 004b6c04d480..56f8330fdcbb 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 3af0d5c37f00..f19e25f37561 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 1cd579d27340..4423a0d0f0fa 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 4f6e99217d51..4d971db81e08 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 5f1a88f2ea97..b051fa72724d 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 FireHydrant(Segment adapter, int count) 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 5c49b9ac292e..9480bdf2bc7e 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 802be54cc115..da3975e4545a 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 26d0b41013da..9b4bd1e4ea7f 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 734abc6ed542..000000000000 --- 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 56af5f3f62c9..28b4379f7b98 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.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 static Appenderator createRealtime( ); } - public static Appenderator createOffline( + public static Appenderator createBatch( String id, DataSchema schema, AppenderatorConfig config, @@ -130,77 +129,4 @@ public static Appenderator createOffline( 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 3bbf364656ea..ec328c3b3cd4 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 @@ 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 @@ Appenderator createRealtimeAppenderatorForTask( ); /** - * 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 b544d33705cc..979d8cb92b3d 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 @@ private void initializeExecutors() } 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 f44fffe20e14..d613f3ff59ce 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 Appenderator createRealtimeAppenderatorForTask( } @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 52f75f72e478..998f674daf7c 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 Appenderator createRealtimeAppenderatorForTask( } @Override - public Appenderator createOfflineAppenderatorForTask( + public Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, @@ -137,7 +137,7 @@ public Appenderator createOfflineAppenderatorForTask( 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 Appenderator createOfflineAppenderatorForTask( } } - @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 4d1253591e08..a25d6b7acd77 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 @@ private void initializeExecutors() } 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 ffdfb8d1eb07..0088e33ca7a8 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 Appenderator createRealtimeAppenderatorForTask( } @Override - public Appenderator createOfflineAppenderatorForTask( + public Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, @@ -226,89 +226,7 @@ public Appenderator createOfflineAppenderatorForTask( 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 @@ private IndexMerger wrapIndexMerger(IndexMerger baseMerger) /** * 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 9ca595ce5c28..000000000000 --- 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 4fad62a6d462..000000000000 --- 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 72a48b33a45a..000000000000 --- 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 c0064a25f64c..000000000000 --- 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 d6aadf07adc6..000000000000 --- 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 9bfda42c8aac..000000000000 --- 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 e4a06999f363..2f80f8a1a06c 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/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java index fe46eabb4261..d52d4e9eba0d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.server.coordinator.compact.CompactionStatistics; import javax.validation.constraints.NotNull; import java.util.Objects; @@ -193,15 +194,9 @@ public static class Builder private final String dataSource; private final AutoCompactionScheduleStatus scheduleStatus; - private long bytesAwaitingCompaction; - private long bytesCompacted; - private long bytesSkipped; - private long segmentCountAwaitingCompaction; - private long segmentCountCompacted; - private long segmentCountSkipped; - private long intervalCountAwaitingCompaction; - private long intervalCountCompacted; - private long intervalCountSkipped; + private final CompactionStatistics compactedStats = new CompactionStatistics(); + private final CompactionStatistics skippedStats = new CompactionStatistics(); + private final CompactionStatistics waitingStats = new CompactionStatistics(); private Builder( @NotNull String dataSource, @@ -217,69 +212,21 @@ private Builder( this.dataSource = dataSource; this.scheduleStatus = scheduleStatus; - this.bytesAwaitingCompaction = 0; - this.bytesCompacted = 0; - this.bytesSkipped = 0; - this.segmentCountAwaitingCompaction = 0; - this.segmentCountCompacted = 0; - this.segmentCountSkipped = 0; - this.intervalCountAwaitingCompaction = 0; - this.intervalCountCompacted = 0; - this.intervalCountSkipped = 0; } - public Builder incrementBytesAwaitingCompaction(long incrementValue) + public void incrementWaitingStats(CompactionStatistics entry) { - this.bytesAwaitingCompaction = this.bytesAwaitingCompaction + incrementValue; - return this; + waitingStats.increment(entry); } - public Builder incrementBytesCompacted(long incrementValue) + public void incrementCompactedStats(CompactionStatistics entry) { - this.bytesCompacted = this.bytesCompacted + incrementValue; - return this; + compactedStats.increment(entry); } - public Builder incrementSegmentCountAwaitingCompaction(long incrementValue) + public void incrementSkippedStats(CompactionStatistics entry) { - this.segmentCountAwaitingCompaction = this.segmentCountAwaitingCompaction + incrementValue; - return this; - } - - public Builder incrementSegmentCountCompacted(long incrementValue) - { - this.segmentCountCompacted = this.segmentCountCompacted + incrementValue; - return this; - } - - public Builder incrementIntervalCountAwaitingCompaction(long incrementValue) - { - this.intervalCountAwaitingCompaction = this.intervalCountAwaitingCompaction + incrementValue; - return this; - } - - public Builder incrementIntervalCountCompacted(long incrementValue) - { - this.intervalCountCompacted = this.intervalCountCompacted + incrementValue; - return this; - } - - public Builder incrementBytesSkipped(long incrementValue) - { - this.bytesSkipped = this.bytesSkipped + incrementValue; - return this; - } - - public Builder incrementSegmentCountSkipped(long incrementValue) - { - this.segmentCountSkipped = this.segmentCountSkipped + incrementValue; - return this; - } - - public Builder incrementIntervalCountSkipped(long incrementValue) - { - this.intervalCountSkipped = this.intervalCountSkipped + incrementValue; - return this; + skippedStats.increment(entry); } public AutoCompactionSnapshot build() @@ -287,15 +234,15 @@ public AutoCompactionSnapshot build() return new AutoCompactionSnapshot( dataSource, scheduleStatus, - bytesAwaitingCompaction, - bytesCompacted, - bytesSkipped, - segmentCountAwaitingCompaction, - segmentCountCompacted, - segmentCountSkipped, - intervalCountAwaitingCompaction, - intervalCountCompacted, - intervalCountSkipped + waitingStats.getTotalBytes(), + compactedStats.getTotalBytes(), + skippedStats.getTotalBytes(), + waitingStats.getNumSegments(), + compactedStats.getNumSegments(), + skippedStats.getNumSegments(), + waitingStats.getNumIntervals(), + compactedStats.getNumIntervals(), + skippedStats.getNumIntervals() ); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentSearchPolicy.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentSearchPolicy.java index 5a006908c318..cc5f4f59d856 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentSearchPolicy.java @@ -33,9 +33,9 @@ public interface CompactionSegmentSearchPolicy { /** - * Reset the current states of this policy. This method should be called whenever iterating starts. + * Creates an iterator that returns compactible segments. */ - CompactionSegmentIterator reset( + CompactionSegmentIterator createIterator( Map compactionConfigs, Map dataSources, Map> skipIntervals diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatistics.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatistics.java index dd672ce4480b..6997dec47c01 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatistics.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatistics.java @@ -28,9 +28,13 @@ public class CompactionStatistics private long numSegments; private long numIntervals; - public static CompactionStatistics create() + public static CompactionStatistics create(long bytes, long numSegments, long numIntervals) { - return new CompactionStatistics(); + final CompactionStatistics stats = new CompactionStatistics(); + stats.totalBytes = bytes; + stats.numIntervals = numIntervals; + stats.numSegments = numSegments; + return stats; } public long getTotalBytes() @@ -48,10 +52,10 @@ public long getNumIntervals() return numIntervals; } - public void addFrom(SegmentsToCompact segments) + public void increment(CompactionStatistics other) { - totalBytes += segments.getTotalBytes(); - numIntervals += segments.getNumIntervals(); - numSegments += segments.size(); + totalBytes += other.getTotalBytes(); + numIntervals += other.getNumIntervals(); + numSegments += other.getNumSegments(); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java index 862f2e7c5b4c..fa053fb8d6ad 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java @@ -167,12 +167,7 @@ private Evaluator( this.objectMapper = objectMapper; this.lastCompactionState = candidateSegments.getFirst().getLastCompactionState(); this.compactionConfig = compactionConfig; - this.tuningConfig = ClientCompactionTaskQueryTuningConfig.from( - compactionConfig.getTuningConfig(), - compactionConfig.getMaxRowsPerSegment(), - null - ); - + this.tuningConfig = ClientCompactionTaskQueryTuningConfig.from(compactionConfig); this.configuredGranularitySpec = compactionConfig.getGranularitySpec(); if (lastCompactionState == null) { this.existingGranularitySpec = null; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstIterator.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java similarity index 82% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstIterator.java rename to server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java index c4ae771f808c..c086be3112bd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstIterator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java @@ -23,9 +23,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.granularity.Granularity; @@ -59,44 +57,46 @@ import java.util.stream.Collectors; /** - * This class iterates all segments of the dataSources configured for compaction from the newest to the oldest. + * Iterator over compactible segments of a datasource in order of specified priority. */ -public class NewestSegmentFirstIterator implements CompactionSegmentIterator +public class DataSourceCompactibleSegmentIterator implements Iterator { - private static final Logger log = new Logger(NewestSegmentFirstIterator.class); + private static final Logger log = new Logger(DataSourceCompactibleSegmentIterator.class); + private final String dataSource; private final ObjectMapper objectMapper; - private final Map compactionConfigs; - private final Map compactedSegmentStats = new HashMap<>(); - private final Map skippedSegmentStats = new HashMap<>(); - - private final Map timelineIterators; + private final DataSourceCompactionConfig config; + private final CompactionStatistics compactedSegmentStats = new CompactionStatistics(); + private final CompactionStatistics skippedSegmentStats = new CompactionStatistics(); // This is needed for datasource that has segmentGranularity configured // If configured segmentGranularity in config is finer than current segmentGranularity, the same set of segments // can belong to multiple intervals in the timeline. We keep track of the compacted intervals between each // run of the compaction job and skip any interval that was already previously compacted. - private final Map> intervalCompactedForDatasource = new HashMap<>(); + private final Set compactedIntervals = new HashSet<>(); - private final PriorityQueue queue = new PriorityQueue<>( - (o1, o2) -> Comparators.intervalsByStartThenEnd().compare(o2.getUmbrellaInterval(), o1.getUmbrellaInterval()) - ); + private final PriorityQueue queue; - NewestSegmentFirstIterator( - ObjectMapper objectMapper, - Map compactionConfigs, - Map dataSources, - Map> skipIntervals + public DataSourceCompactibleSegmentIterator( + DataSourceCompactionConfig config, + SegmentTimeline timeline, + List skipIntervals, + Comparator segmentPriority, + ObjectMapper objectMapper ) { this.objectMapper = objectMapper; - this.compactionConfigs = compactionConfigs; - this.timelineIterators = Maps.newHashMapWithExpectedSize(dataSources.size()); + this.config = config; + this.dataSource = config.getDataSource(); + this.queue = new PriorityQueue<>(segmentPriority); + populateQueue(timeline, skipIntervals); + } - dataSources.forEach((dataSource, timeline) -> { - final DataSourceCompactionConfig config = compactionConfigs.get(dataSource); + private void populateQueue(SegmentTimeline timeline, List skipIntervals) + { + if (timeline != null) { Granularity configuredSegmentGranularity = null; - if (config != null && !timeline.isEmpty()) { + if (!timeline.isEmpty()) { SegmentTimeline originalTimeline = null; if (config.getGranularitySpec() != null && config.getGranularitySpec().getSegmentGranularity() != null) { String temporaryVersion = DateTimes.nowUtc().toString(); @@ -154,33 +154,25 @@ public class NewestSegmentFirstIterator implements CompactionSegmentIterator timeline, config.getSkipOffsetFromLatest(), configuredSegmentGranularity, - skipIntervals.get(dataSource) + skipIntervals ); if (!searchIntervals.isEmpty()) { - timelineIterators.put( - dataSource, + findAndEnqueueSegmentsToCompact( new CompactibleSegmentIterator(timeline, searchIntervals, originalTimeline) ); + } else { + log.warn("Skipping compaction for datasource[%s] as it has no compactible segments.", dataSource); } } - }); - - compactionConfigs.forEach((dataSourceName, config) -> { - if (config == null) { - throw new ISE("Unknown dataSource[%s]", dataSourceName); - } - updateQueue(dataSourceName, config); - }); + } } - @Override - public Map totalCompactedStatistics() + public CompactionStatistics totalCompactedStatistics() { return compactedSegmentStats; } - @Override - public Map totalSkippedStatistics() + public CompactionStatistics totalSkippedStatistics() { return skippedSegmentStats; } @@ -206,25 +198,9 @@ public SegmentsToCompact next() final List resultSegments = entry.getSegments(); Preconditions.checkState(!resultSegments.isEmpty(), "Queue entry must not be empty"); - final String dataSource = resultSegments.get(0).getDataSource(); - updateQueue(dataSource, compactionConfigs.get(dataSource)); - return entry; } - /** - * Find the next segments to compact for the given dataSource and add them to the queue. - * {@link #timelineIterators} is updated according to the found segments. That is, the found segments are removed from - * the timeline of the given dataSource. - */ - private void updateQueue(String dataSourceName, DataSourceCompactionConfig config) - { - final SegmentsToCompact segmentsToCompact = findSegmentsToCompact(dataSourceName, config); - if (!segmentsToCompact.isEmpty()) { - queue.add(segmentsToCompact); - } - } - /** * Iterates compactible segments in a {@link SegmentTimeline}. */ @@ -315,27 +291,12 @@ public List next() } /** - * Finds segments to compact together for the given datasource. - * - * @return An empty {@link SegmentsToCompact} if there are no eligible candidates. + * Finds segments to compact together for the given datasource and adds them to + * the priority queue. */ - private SegmentsToCompact findSegmentsToCompact( - final String dataSourceName, - final DataSourceCompactionConfig config - ) + private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compactibleSegmentIterator) { - final CompactibleSegmentIterator compactibleSegmentIterator - = timelineIterators.get(dataSourceName); - if (compactibleSegmentIterator == null) { - log.warn( - "Skipping compaction for datasource[%s] as there is no compactible segment in its timeline.", - dataSourceName - ); - return SegmentsToCompact.empty(); - } - final long inputSegmentSize = config.getInputSegmentSizeBytes(); - while (compactibleSegmentIterator.hasNext()) { List segments = compactibleSegmentIterator.next(); @@ -352,47 +313,33 @@ private SegmentsToCompact findSegmentsToCompact( if (!compactionStatus.isComplete()) { log.debug( "Datasource[%s], interval[%s] has [%d] segments that need to be compacted because [%s].", - dataSourceName, interval, candidates.size(), compactionStatus.getReasonToCompact() + dataSource, interval, candidates.size(), compactionStatus.getReasonToCompact() ); } if (compactionStatus.isComplete()) { - addSegmentStatsTo(compactedSegmentStats, dataSourceName, candidates); + compactedSegmentStats.increment(candidates.getStats()); } else if (candidates.getTotalBytes() > inputSegmentSize) { - addSegmentStatsTo(skippedSegmentStats, dataSourceName, candidates); + skippedSegmentStats.increment(candidates.getStats()); log.warn( "Skipping compaction for datasource[%s], interval[%s] as total segment size[%d]" + " is larger than allowed inputSegmentSize[%d].", - dataSourceName, interval, candidates.getTotalBytes(), inputSegmentSize + dataSource, interval, candidates.getTotalBytes(), inputSegmentSize ); } else if (config.getGranularitySpec() != null && config.getGranularitySpec().getSegmentGranularity() != null) { - Set compactedIntervals = intervalCompactedForDatasource - .computeIfAbsent(dataSourceName, k -> new HashSet<>()); - if (compactedIntervals.contains(interval)) { // Skip these candidate segments as we have already compacted this interval } else { compactedIntervals.add(interval); - return candidates; + queue.add(candidates); } } else { - return candidates; + queue.add(candidates); } } - log.debug("No more segments to compact for datasource[%s].", dataSourceName); - return SegmentsToCompact.empty(); - } - - private void addSegmentStatsTo( - Map statisticsMap, - String dataSourceName, - SegmentsToCompact segments - ) - { - statisticsMap.computeIfAbsent(dataSourceName, v -> CompactionStatistics.create()) - .addFrom(segments); + log.debug("No more segments to compact for datasource[%s].", dataSource); } /** @@ -428,7 +375,7 @@ private List findInitialSearchInterval( final List segments = new ArrayList<>( timeline.findNonOvershadowedObjectsInInterval(skipInterval, Partitions.ONLY_COMPLETE) ); - addSegmentStatsTo(skippedSegmentStats, dataSourceName, SegmentsToCompact.from(segments)); + skippedSegmentStats.increment(SegmentsToCompact.from(segments).getStats()); } final Interval totalInterval = new Interval(first.getInterval().getStart(), last.getInterval().getEnd()); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java index 20f6d920441a..bc923da4f805 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; +import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Interval; @@ -29,7 +30,7 @@ import java.util.Map; /** - * This policy searches segments for compaction from the newest one to oldest one. + * This policy searches segments for compaction from newest to oldest. */ public class NewestSegmentFirstPolicy implements CompactionSegmentSearchPolicy { @@ -42,12 +43,20 @@ public NewestSegmentFirstPolicy(ObjectMapper objectMapper) } @Override - public CompactionSegmentIterator reset( + public CompactionSegmentIterator createIterator( Map compactionConfigs, Map dataSources, Map> skipIntervals ) { - return new NewestSegmentFirstIterator(objectMapper, compactionConfigs, dataSources, skipIntervals); + return new PriorityBasedCompactionSegmentIterator( + compactionConfigs, + dataSources, + skipIntervals, + (o1, o2) -> Comparators.intervalsByStartThenEnd() + .compare(o2.getUmbrellaInterval(), o1.getUmbrellaInterval()), + objectMapper + ); } + } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java new file mode 100644 index 000000000000..33aea2a0451c --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.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.server.coordinator.compact; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.timeline.SegmentTimeline; +import org.apache.druid.utils.CollectionUtils; +import org.joda.time.Interval; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.PriorityQueue; + +/** + * Implementation of {@link CompactionSegmentIterator} that returns segments in + * order of their priority. + */ +public class PriorityBasedCompactionSegmentIterator implements CompactionSegmentIterator +{ + private static final Logger log = new Logger(PriorityBasedCompactionSegmentIterator.class); + + private final PriorityQueue queue; + private final Map datasourceIterators; + + public PriorityBasedCompactionSegmentIterator( + Map compactionConfigs, + Map datasourceToTimeline, + Map> skipIntervals, + Comparator segmentPriority, + ObjectMapper objectMapper + ) + { + this.queue = new PriorityQueue<>(segmentPriority); + this.datasourceIterators = Maps.newHashMapWithExpectedSize(datasourceToTimeline.size()); + compactionConfigs.forEach((datasource, config) -> { + if (config == null) { + throw DruidException.defensive("Invalid null compaction config for dataSource[%s].", datasource); + } + final SegmentTimeline timeline = datasourceToTimeline.get(datasource); + if (timeline == null) { + log.warn("Skipping compaction for datasource[%s] as it has no timeline.", datasource); + return; + } + + datasourceIterators.put( + datasource, + new DataSourceCompactibleSegmentIterator( + compactionConfigs.get(datasource), + timeline, + skipIntervals.getOrDefault(datasource, Collections.emptyList()), + segmentPriority, + objectMapper + ) + ); + addNextItemForDatasourceToQueue(datasource); + }); + } + + @Override + public Map totalCompactedStatistics() + { + return CollectionUtils.mapValues( + datasourceIterators, + DataSourceCompactibleSegmentIterator::totalCompactedStatistics + ); + } + + @Override + public Map totalSkippedStatistics() + { + return CollectionUtils.mapValues( + datasourceIterators, + DataSourceCompactibleSegmentIterator::totalSkippedStatistics + ); + } + + @Override + public boolean hasNext() + { + return !queue.isEmpty(); + } + + @Override + public SegmentsToCompact next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + final SegmentsToCompact entry = queue.poll(); + if (entry == null) { + throw new NoSuchElementException(); + } + Preconditions.checkState(!entry.isEmpty(), "Queue entry must not be empty"); + + addNextItemForDatasourceToQueue(entry.getFirst().getDataSource()); + return entry; + } + + private void addNextItemForDatasourceToQueue(String dataSourceName) + { + final DataSourceCompactibleSegmentIterator iterator = datasourceIterators.get(dataSourceName); + if (iterator.hasNext()) { + final SegmentsToCompact segmentsToCompact = iterator.next(); + if (!segmentsToCompact.isEmpty()) { + queue.add(segmentsToCompact); + } + } + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java index 1bc53b7dbe71..27ce9beab81f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java @@ -107,9 +107,9 @@ public Interval getUmbrellaInterval() return umbrellaInterval; } - public long getNumIntervals() + public CompactionStatistics getStats() { - return numIntervals; + return CompactionStatistics.create(totalBytes, size(), numIntervals); } @Override 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 90fb684db6c7..01f3bc77e9ee 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 @@ -54,7 +54,6 @@ import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator; import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; -import org.apache.druid.server.coordinator.compact.CompactionStatistics; import org.apache.druid.server.coordinator.compact.SegmentsToCompact; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Dimension; @@ -87,6 +86,7 @@ public class CompactSegments implements CoordinatorCustomDuty private static final Logger LOG = new Logger(CompactSegments.class); + private static final String TASK_ID_PREFIX = "coordinator-issued"; private static final Predicate IS_COMPACTION_TASK = status -> null != status && COMPACTION_TASK_TYPE.equals(status.getType()); @@ -196,7 +196,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // Get iterator over segments to compact and submit compaction tasks Map dataSources = params.getUsedSegmentsTimelinesPerDataSource(); final CompactionSegmentIterator iterator = - policy.reset(compactionConfigs, dataSources, intervalsToSkipCompaction); + policy.createIterator(compactionConfigs, dataSources, intervalsToSkipCompaction); final int compactionTaskCapacity = getCompactionTaskCapacity(dynamicConfig); final int availableCompactionTaskSlots @@ -215,7 +215,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) stats.add(Stats.Compaction.MAX_SLOTS, compactionTaskCapacity); stats.add(Stats.Compaction.AVAILABLE_SLOTS, availableCompactionTaskSlots); stats.add(Stats.Compaction.SUBMITTED_TASKS, numSubmittedCompactionTasks); - addCompactionSnapshotStats(currentRunAutoCompactionSnapshotBuilders, iterator, stats); + updateCompactionSnapshotStats(currentRunAutoCompactionSnapshotBuilders, iterator, stats); return params; } @@ -392,28 +392,19 @@ private int submitCompactionTasks( while (iterator.hasNext() && totalTaskSlotsAssigned < numAvailableCompactionTaskSlots) { final SegmentsToCompact entry = iterator.next(); - final List segmentsToCompact = entry.getSegments(); - if (segmentsToCompact.isEmpty()) { + if (entry.isEmpty()) { throw new ISE("segmentsToCompact is empty?"); } - final String dataSourceName = segmentsToCompact.get(0).getDataSource(); + final String dataSourceName = entry.getFirst().getDataSource(); // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics - AutoCompactionSnapshot.Builder snapshotBuilder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent( - dataSourceName, - AutoCompactionSnapshot::builder - ); - snapshotBuilder - .incrementBytesCompacted( - segmentsToCompact.stream().mapToLong(DataSegment::getSize).sum() - ) - .incrementIntervalCountCompacted( - segmentsToCompact.stream().map(DataSegment::getInterval).distinct().count() - ) - .incrementSegmentCountCompacted(segmentsToCompact.size()); + currentRunAutoCompactionSnapshotBuilders + .computeIfAbsent(dataSourceName, AutoCompactionSnapshot::builder) + .incrementCompactedStats(entry.getStats()); final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName); + final List segmentsToCompact = entry.getSegments(); // Create granularitySpec to send to compaction task ClientCompactionTaskGranularitySpec granularitySpec; @@ -514,7 +505,6 @@ private int submitCompactionTasks( } final String taskId = compactSegments( - "coordinator-issued", segmentsToCompact, config.getTaskPriority(), ClientCompactionTaskQueryTuningConfig.from( @@ -536,7 +526,6 @@ private int submitCompactionTasks( taskId, segmentsToCompact.size(), dataSourceName, entry.getUmbrellaInterval() ); LOG.debugSegments(segmentsToCompact, "Compacting segments"); - // Count the compaction task itself + its sub tasks numSubmittedTasks++; totalTaskSlotsAssigned += slotsRequiredForCurrentTask; } @@ -554,7 +543,7 @@ private Map newAutoCompactionContext(@Nullable Map currentRunAutoCompactionSnapshotBuilders, CompactionSegmentIterator iterator, CoordinatorRunStats stats @@ -563,77 +552,45 @@ private void addCompactionSnapshotStats( // Mark all the segments remaining in the iterator as "awaiting compaction" while (iterator.hasNext()) { final SegmentsToCompact entry = iterator.next(); - final List segmentsToCompact = entry.getSegments(); - if (!segmentsToCompact.isEmpty()) { - final String dataSourceName = segmentsToCompact.get(0).getDataSource(); - AutoCompactionSnapshot.Builder snapshotBuilder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent( - dataSourceName, - AutoCompactionSnapshot::builder - ); - snapshotBuilder - .incrementBytesAwaitingCompaction( - segmentsToCompact.stream().mapToLong(DataSegment::getSize).sum() - ) - .incrementIntervalCountAwaitingCompaction( - segmentsToCompact.stream().map(DataSegment::getInterval).distinct().count() - ) - .incrementSegmentCountAwaitingCompaction(segmentsToCompact.size()); + if (!entry.isEmpty()) { + final String dataSourceName = entry.getFirst().getDataSource(); + currentRunAutoCompactionSnapshotBuilders + .computeIfAbsent(dataSourceName, AutoCompactionSnapshot::builder) + .incrementWaitingStats(entry.getStats()); } } // Statistics of all segments considered compacted after this run - Map allCompactedStatistics = iterator.totalCompactedStatistics(); - for (Map.Entry compactionStatisticsEntry : allCompactedStatistics.entrySet()) { - final String dataSource = compactionStatisticsEntry.getKey(); - final CompactionStatistics dataSourceCompactedStatistics = compactionStatisticsEntry.getValue(); - AutoCompactionSnapshot.Builder builder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent( - dataSource, - AutoCompactionSnapshot::builder - ); - builder.incrementBytesCompacted(dataSourceCompactedStatistics.getTotalBytes()); - builder.incrementSegmentCountCompacted(dataSourceCompactedStatistics.getNumSegments()); - builder.incrementIntervalCountCompacted(dataSourceCompactedStatistics.getNumIntervals()); - } + iterator.totalCompactedStatistics().forEach((dataSource, compactedStats) -> { + currentRunAutoCompactionSnapshotBuilders + .computeIfAbsent(dataSource, AutoCompactionSnapshot::builder) + .incrementCompactedStats(compactedStats); + }); // Statistics of all segments considered skipped after this run - Map allSkippedStatistics = iterator.totalSkippedStatistics(); - for (Map.Entry compactionStatisticsEntry : allSkippedStatistics.entrySet()) { - final String dataSource = compactionStatisticsEntry.getKey(); - final CompactionStatistics dataSourceSkippedStatistics = compactionStatisticsEntry.getValue(); - AutoCompactionSnapshot.Builder builder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent( - dataSource, - AutoCompactionSnapshot::builder - ); - builder.incrementBytesSkipped(dataSourceSkippedStatistics.getTotalBytes()) - .incrementSegmentCountSkipped(dataSourceSkippedStatistics.getNumSegments()) - .incrementIntervalCountSkipped(dataSourceSkippedStatistics.getNumIntervals()); - } + iterator.totalSkippedStatistics().forEach((dataSource, dataSourceSkippedStatistics) -> { + currentRunAutoCompactionSnapshotBuilders + .computeIfAbsent(dataSource, AutoCompactionSnapshot::builder) + .incrementSkippedStats(dataSourceSkippedStatistics); + }); final Map currentAutoCompactionSnapshotPerDataSource = new HashMap<>(); - for (Map.Entry autoCompactionSnapshotBuilderEntry - : currentRunAutoCompactionSnapshotBuilders.entrySet()) { - final String dataSource = autoCompactionSnapshotBuilderEntry.getKey(); - final AutoCompactionSnapshot.Builder builder = autoCompactionSnapshotBuilderEntry.getValue(); - - // Build the complete snapshot for the datasource - AutoCompactionSnapshot autoCompactionSnapshot = builder.build(); + currentRunAutoCompactionSnapshotBuilders.forEach((dataSource, builder) -> { + final AutoCompactionSnapshot autoCompactionSnapshot = builder.build(); currentAutoCompactionSnapshotPerDataSource.put(dataSource, autoCompactionSnapshot); - - // Use the complete snapshot to emit metrics - addStatsForDatasource(dataSource, autoCompactionSnapshot, stats); - } + collectSnapshotStats(autoCompactionSnapshot, stats); + }); // Atomic update of autoCompactionSnapshotPerDataSource with the latest from this coordinator run autoCompactionSnapshotPerDataSource.set(currentAutoCompactionSnapshotPerDataSource); } - private void addStatsForDatasource( - String dataSource, + private void collectSnapshotStats( AutoCompactionSnapshot autoCompactionSnapshot, CoordinatorRunStats stats ) { - final RowKey rowKey = RowKey.of(Dimension.DATASOURCE, dataSource); + final RowKey rowKey = RowKey.of(Dimension.DATASOURCE, autoCompactionSnapshot.getDataSource()); stats.add(Stats.Compaction.PENDING_BYTES, rowKey, autoCompactionSnapshot.getBytesAwaitingCompaction()); stats.add(Stats.Compaction.PENDING_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountAwaitingCompaction()); @@ -668,7 +625,6 @@ public Map getAutoCompactionSnapshot() } private String compactSegments( - String idPrefix, List segments, int compactionTaskPriority, @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig, @@ -692,7 +648,7 @@ private String compactSegments( context = context == null ? new HashMap<>() : context; context.put("priority", compactionTaskPriority); - final String taskId = IdUtils.newTaskId(idPrefix, ClientCompactionTaskQuery.TYPE, dataSource, null); + final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null); final Granularity segmentGranularity = granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); final ClientTaskQuery taskPayload = new ClientCompactionTaskQuery( taskId, 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 aead7b86c719..8ee3d5ba0814 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.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 687ca2ef5485..feb61965daa3 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.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 7dcd3b2c237a..c4c0cb8ce4b0 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 @@ 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 13502e18493c..000000000000 --- 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 29c1808ad515..000000000000 --- 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 66a022992e74..000000000000 --- 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 278a170910ad..ab2cdb3811f0 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 void configure(Binder binder) 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/curator/discovery/ServiceAnnouncerTest.java b/server/src/test/java/org/apache/druid/curator/discovery/ServiceAnnouncerTest.java index 57f7517239d6..f80f35bdca83 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 void testServiceAnnouncement() throws Exception 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 8ecc93dece2b..000000000000 --- 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/SQLMetadataConnectorSchemaPersistenceTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorSchemaPersistenceTest.java index e103b428fedb..b59af9ef690b 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorSchemaPersistenceTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorSchemaPersistenceTest.java @@ -56,7 +56,6 @@ public void testCreateTables() tables.add(tablesConfig.getSegmentsTable()); tables.add(tablesConfig.getRulesTable()); tables.add(tablesConfig.getLockTable(entryType)); - tables.add(tablesConfig.getLogTable(entryType)); tables.add(tablesConfig.getEntryTable(entryType)); tables.add(tablesConfig.getAuditTable()); tables.add(tablesConfig.getSupervisorTable()); @@ -67,7 +66,6 @@ public void testCreateTables() dropSequence.add(tablesConfig.getSegmentSchemasTable()); dropSequence.add(tablesConfig.getRulesTable()); dropSequence.add(tablesConfig.getLockTable(entryType)); - dropSequence.add(tablesConfig.getLogTable(entryType)); dropSequence.add(tablesConfig.getEntryTable(entryType)); dropSequence.add(tablesConfig.getAuditTable()); dropSequence.add(tablesConfig.getSupervisorTable()); diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java index 484299b5636b..a40f95c31b92 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -75,7 +75,6 @@ public void testCreateTables() tables.add(tablesConfig.getSegmentsTable()); tables.add(tablesConfig.getRulesTable()); tables.add(tablesConfig.getLockTable(entryType)); - tables.add(tablesConfig.getLogTable(entryType)); tables.add(tablesConfig.getEntryTable(entryType)); tables.add(tablesConfig.getAuditTable()); tables.add(tablesConfig.getSupervisorTable()); diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 6cddcd5d6468..8117d6dcb735 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -34,7 +34,6 @@ 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.jackson.JacksonUtils; import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup; import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup; import org.joda.time.DateTime; @@ -73,12 +72,10 @@ public void setUp() TestDerbyConnector connector = derbyConnectorRule.getConnector(); final String entryType = "entry"; - final String logTable = "logs"; final String lockTable = "locks"; connector.prepareTaskEntryTable(entryTable); connector.createLockTable(lockTable, entryType); - connector.createLogTable(logTable, entryType); handler = new DerbyMetadataStorageActionHandler<>( connector, @@ -101,12 +98,6 @@ public TypeReference> getStatusType() }; } - @Override - public TypeReference> getLogType() - { - return JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING; - } - @Override public TypeReference> getLockType() { @@ -117,7 +108,7 @@ public TypeReference> getLockType() }, entryType, entryTable, - logTable, + null, lockTable ); } @@ -247,37 +238,31 @@ public void testDuplicateInsertThrowsEntryExistsException() } @Test - public void testLogs() + public void testAddLogThrowsUnsupportedException() { - final String entryId = "abcd"; - Map entry = ImmutableMap.of("a", 1); - Map status = ImmutableMap.of("count", 42); - - handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, status, "type", "group"); - - Assert.assertEquals( - ImmutableList.of(), - handler.getLogs("non_exist_entry") + Exception exception = Assert.assertThrows( + DruidException.class, + () -> handler.addLog("abcd", ImmutableMap.of("logentry", "created")) ); - Assert.assertEquals( - ImmutableMap.of(), - handler.getLocks(entryId) + "Task actions are not logged anymore.", + exception.getMessage() ); + } - final ImmutableMap log1 = ImmutableMap.of("logentry", "created"); - final ImmutableMap log2 = ImmutableMap.of("logentry", "updated"); - - Assert.assertTrue(handler.addLog(entryId, log1)); - Assert.assertTrue(handler.addLog(entryId, log2)); - + @Test + public void testGetLogsThrowsUnsupportedException() + { + Exception exception = Assert.assertThrows( + DruidException.class, + () -> handler.getLogs("abcd") + ); Assert.assertEquals( - ImmutableList.of(log1, log2), - handler.getLogs(entryId) + "Task actions are not logged anymore.", + exception.getMessage() ); } - @Test public void testLocks() { @@ -388,19 +373,16 @@ public void testRemoveTasksOlderThan() Map entry1 = ImmutableMap.of("numericId", 1234); Map status1 = ImmutableMap.of("count", 42, "temp", 1); handler.insert(entryId1, DateTimes.of("2014-01-01T00:00:00.123"), "testDataSource", entry1, false, status1, "type", "group"); - Assert.assertTrue(handler.addLog(entryId1, ImmutableMap.of("logentry", "created"))); final String entryId2 = "ABC123"; Map entry2 = ImmutableMap.of("a", 1); Map status2 = ImmutableMap.of("count", 42); handler.insert(entryId2, DateTimes.of("2014-01-01T00:00:00.123"), "test", entry2, true, status2, "type", "group"); - Assert.assertTrue(handler.addLog(entryId2, ImmutableMap.of("logentry", "created"))); final String entryId3 = "DEF5678"; Map entry3 = ImmutableMap.of("numericId", 5678); Map status3 = ImmutableMap.of("count", 21, "temp", 2); handler.insert(entryId3, DateTimes.of("2014-01-02T12:00:00.123"), "testDataSource", entry3, false, status3, "type", "group"); - Assert.assertTrue(handler.addLog(entryId3, ImmutableMap.of("logentry", "created"))); Assert.assertEquals(Optional.of(entry1), handler.getEntry(entryId1)); Assert.assertEquals(Optional.of(entry2), handler.getEntry(entryId2)); @@ -438,10 +420,6 @@ public void testRemoveTasksOlderThan() .collect(Collectors.toList()) ); - // tasklogs - Assert.assertEquals(0, handler.getLogs(entryId1).size()); - Assert.assertEquals(1, handler.getLogs(entryId2).size()); - Assert.assertEquals(1, handler.getLogs(entryId3).size()); } @Test 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 ccd71cfaff5e..4053ec9ecf22 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 void testExecuteQuery() throws IOException File tmpFile = File.createTempFile("testQueryResults", ""); InputEntity.CleanableFile queryResult = SqlEntity.openCleanableFile( VALID_SQL, - testUtils.getDerbyFirehoseConnector(), + testUtils.getDerbyInputSourceConnector(), mapper, true, tmpFile @@ -95,7 +95,7 @@ public void testFileDeleteOnInvalidQuery() throws IOException 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 7a5ea7b21490..c48dfd839463 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.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 @@ 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 void setUp() @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 void testSerde() throws IOException @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 void testSingleSplit() throws Exception 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 void testMultipleSplits() throws Exception 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 void testNumSplits() 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 void testSample() throws Exception 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 void testSample() throws Exception } } + @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 void testEquals() 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 boolean equals(Object o) 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 2e99bfbb301b..f999e6dd41c4 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.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 Set getAllowedProperties() 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 void dropTable(final String tableName) ); } - public TestDerbyFirehoseConnector getDerbyFirehoseConnector() + public TestDerbyInputSourceConnector getDerbyInputSourceConnector() { - return derbyFirehoseConnector; + return derbyInputSourceConnector; } /** 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 870636fb416a..21d99c622255 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 71a3fe308f4d..05fb11e4b620 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 66b2281d33cc..000000000000 --- 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 269aeaca7c4c..ed63bca31956 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.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 55b5f2355067..8b9117705f52 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.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 void testSimpleIngestion() throws Exception @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 void testPushFailure() throws Exception @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 void testPeriodGranularityNonUTCIngestion() throws Exception public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( 100, 1024, null, @@ -316,7 +316,7 @@ public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throw public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( 100, 1024, null, @@ -346,7 +346,7 @@ public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadChec @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 void testMaxBytesInMemory() throws Exception @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 void testTaskFailAsPersistCannotFreeAnyMoreMemory() throws Exception public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( 100, 10, null, @@ -468,8 +468,8 @@ public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadChec @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 void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exc @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 void testMaxBytesInMemoryInMultipleSinks() throws Exception @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 void testIgnoreMaxBytesInMemory() throws Exception @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 void testMaxRowsInMemory() throws Exception @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 void testAllHydrantsAreRecovered() throws Exception @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 void testTotalRowsPerSegment() throws Exception @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 void testRestoreFromDisk() throws Exception @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 void testCleanupFromDiskAfterClose() throws Exception @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 void testTotalRowCount() throws Exception 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 void testVerifyRowIngestionMetrics() throws Exception 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 void testPushContract() throws Exception 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 void testCloseContract() throws Exception 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 @@ private static SegmentIdWithShardSpec createNonUTCSegmentId(String interval, Str 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 cf2d7f798986..22034aa33aa9 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.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 ClosedSegmensSinksBatchAppenderatorTester( this(maxRowsInMemory, -1, null, enablePushFailure); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final boolean enablePushFailure @@ -96,7 +96,7 @@ public ClosedSegmensSinksBatchAppenderatorTester( this(maxRowsInMemory, maxSizeInBytes, null, enablePushFailure); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final File basePersistDirectory, @@ -113,7 +113,7 @@ public ClosedSegmensSinksBatchAppenderatorTester( ); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, @Nullable final File basePersistDirectory, @@ -126,7 +126,7 @@ public ClosedSegmensSinksBatchAppenderatorTester( ); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, @Nullable final File basePersistDirectory, @@ -230,7 +230,7 @@ public Map makeLoadSpec(URI uri) 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 33a0ed2f8a4d..000000000000 --- 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 0c6fb552a4dc..000000000000 --- 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 2f5e5cde7ed0..000000000000 --- 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 2a39718667c0..cc0dc1fad1bc 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 @@ TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( 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 @@ TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( 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 9bf629931b54..23ac93db0096 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 void setup() 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 419b29ace7e2..000000000000 --- 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 38b16c79cab4..000000000000 --- 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 c3863708a66d..038fbce7d455 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 void testGroupByOnScanMultiValue() 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 void testTopNScanMultiValue() 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 void testScanOnScanWithStringExpression() .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 void testScanOnScanWithStringExpression() ) ) .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 3df1d95b33a2..f6bfebcf344d 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 static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerat .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/AutoCompactionSnapshotTest.java b/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java index e034459fc741..5517bf9e6a4e 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java @@ -19,6 +19,7 @@ package org.apache.druid.server.coordinator; +import org.apache.druid.server.coordinator.compact.CompactionStatistics; import org.junit.Assert; import org.junit.Test; @@ -30,17 +31,11 @@ public void testAutoCompactionSnapshotBuilder() final String expectedDataSource = "data"; final AutoCompactionSnapshot.Builder builder = AutoCompactionSnapshot.builder(expectedDataSource); - // Increment every stats twice + // Increment every stat twice for (int i = 0; i < 2; i++) { - builder.incrementIntervalCountSkipped(13) - .incrementBytesSkipped(13) - .incrementSegmentCountSkipped(13) - .incrementIntervalCountCompacted(13) - .incrementBytesCompacted(13) - .incrementSegmentCountCompacted(13) - .incrementIntervalCountAwaitingCompaction(13) - .incrementBytesAwaitingCompaction(13) - .incrementSegmentCountAwaitingCompaction(13); + builder.incrementSkippedStats(CompactionStatistics.create(13, 13, 13)); + builder.incrementWaitingStats(CompactionStatistics.create(13, 13, 13)); + builder.incrementCompactedStats(CompactionStatistics.create(13, 13, 13)); } final AutoCompactionSnapshot actual = builder.build(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTest.java new file mode 100644 index 000000000000..0e13f8cd0e1e --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTest.java @@ -0,0 +1,178 @@ +/* + * 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.compact; + +import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; +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.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; + +public class CompactionStatusTest +{ + private static final String DS_WIKI = "wiki"; + + @Test + public void testFindPartitionsSpecWhenGivenIsNull() + { + final ClientCompactionTaskQueryTuningConfig tuningConfig + = ClientCompactionTaskQueryTuningConfig.from(null); + Assert.assertEquals( + new DynamicPartitionsSpec(null, Long.MAX_VALUE), + CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) + ); + } + + @Test + public void testFindPartitionsSpecWhenGivenIsDynamicWithNullMaxTotalRows() + { + final PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); + final ClientCompactionTaskQueryTuningConfig tuningConfig + = ClientCompactionTaskQueryTuningConfig.from(createCompactionConfig(partitionsSpec)); + Assert.assertEquals( + new DynamicPartitionsSpec(null, Long.MAX_VALUE), + CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) + ); + } + + @Test + public void testFindPartitionsSpecWhenGivenIsDynamicWithMaxTotalRows() + { + final PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, 1000L); + final ClientCompactionTaskQueryTuningConfig tuningConfig + = ClientCompactionTaskQueryTuningConfig.from(createCompactionConfig(partitionsSpec)); + Assert.assertEquals( + partitionsSpec, + CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) + ); + } + + @Test + public void testFindPartitionsSpecWhenGivenIsDynamicWithMaxRowsPerSegment() + { + final PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(100, 1000L); + final ClientCompactionTaskQueryTuningConfig tuningConfig + = ClientCompactionTaskQueryTuningConfig.from(createCompactionConfig(partitionsSpec)); + Assert.assertEquals( + partitionsSpec, + CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) + ); + } + + @Test + public void testFindPartitionsSpecFromConfigWithDeprecatedMaxRowsPerSegmentAndMaxTotalRowsReturnGivenValues() + { + final DataSourceCompactionConfig config = new DataSourceCompactionConfig( + "datasource", + null, + null, + 100, + null, + new UserCompactionTaskQueryTuningConfig( + null, + null, + null, + 1000L, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ), + null, + null, + null, + null, + null, + null, + null + ); + Assert.assertEquals( + new DynamicPartitionsSpec(100, 1000L), + CompactionStatus.findPartitionsSpecFromConfig( + ClientCompactionTaskQueryTuningConfig.from(config) + ) + ); + } + + @Test + public void testFindPartitionsSpecWhenGivenIsHashed() + { + final PartitionsSpec partitionsSpec = + new HashedPartitionsSpec(null, 100, Collections.singletonList("dim")); + final ClientCompactionTaskQueryTuningConfig tuningConfig + = ClientCompactionTaskQueryTuningConfig.from(createCompactionConfig(partitionsSpec)); + Assert.assertEquals( + partitionsSpec, + CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) + ); + } + + @Test + public void testFindPartitionsSpecWhenGivenIsRange() + { + final PartitionsSpec partitionsSpec = + new DimensionRangePartitionsSpec(null, 10000, Collections.singletonList("dim"), false); + final ClientCompactionTaskQueryTuningConfig tuningConfig + = ClientCompactionTaskQueryTuningConfig.from(createCompactionConfig(partitionsSpec)); + Assert.assertEquals( + partitionsSpec, + CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) + ); + } + + private static DataSourceCompactionConfig createCompactionConfig( + PartitionsSpec partitionsSpec + ) + { + return new DataSourceCompactionConfig( + DS_WIKI, + null, null, null, null, createTuningConfig(partitionsSpec), + null, null, null, null, null, null, null + ); + } + + private static UserCompactionTaskQueryTuningConfig createTuningConfig( + PartitionsSpec partitionsSpec + ) + { + return new UserCompactionTaskQueryTuningConfig( + null, + null, null, null, null, partitionsSpec, null, null, null, + null, null, null, null, null, null, null, null, null, null + ); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIteratorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIteratorTest.java new file mode 100644 index 000000000000..a2765ccfc09c --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIteratorTest.java @@ -0,0 +1,77 @@ +/* + * 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.compact; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class DataSourceCompactibleSegmentIteratorTest +{ + @Test + public void testFilterSkipIntervals() + { + final Interval totalInterval = Intervals.of("2018-01-01/2019-01-01"); + final List expectedSkipIntervals = ImmutableList.of( + Intervals.of("2018-01-15/2018-03-02"), + Intervals.of("2018-07-23/2018-10-01"), + Intervals.of("2018-10-02/2018-12-25"), + Intervals.of("2018-12-31/2019-01-01") + ); + final List skipIntervals = DataSourceCompactibleSegmentIterator.filterSkipIntervals( + totalInterval, + Lists.newArrayList( + Intervals.of("2017-12-01/2018-01-15"), + Intervals.of("2018-03-02/2018-07-23"), + Intervals.of("2018-10-01/2018-10-02"), + Intervals.of("2018-12-25/2018-12-31") + ) + ); + + Assert.assertEquals(expectedSkipIntervals, skipIntervals); + } + + @Test + public void testAddSkipIntervalFromLatestAndSort() + { + final List expectedIntervals = ImmutableList.of( + Intervals.of("2018-12-24/2018-12-25"), + Intervals.of("2018-12-29/2019-01-01") + ); + final List fullSkipIntervals = DataSourceCompactibleSegmentIterator.sortAndAddSkipIntervalFromLatest( + DateTimes.of("2019-01-01"), + new Period(72, 0, 0, 0), + null, + ImmutableList.of( + Intervals.of("2018-12-30/2018-12-31"), + Intervals.of("2018-12-24/2018-12-25") + ) + ); + + Assert.assertEquals(expectedIntervals, fullSkipIntervals); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstIteratorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstIteratorTest.java deleted file mode 100644 index 9c96e6fcdd8b..000000000000 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstIteratorTest.java +++ /dev/null @@ -1,477 +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.coordinator.compact; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; -import org.joda.time.Interval; -import org.joda.time.Period; -import org.junit.Assert; -import org.junit.Test; - -import java.util.List; - -public class NewestSegmentFirstIteratorTest -{ - @Test - public void testFilterSkipIntervals() - { - final Interval totalInterval = Intervals.of("2018-01-01/2019-01-01"); - final List expectedSkipIntervals = ImmutableList.of( - Intervals.of("2018-01-15/2018-03-02"), - Intervals.of("2018-07-23/2018-10-01"), - Intervals.of("2018-10-02/2018-12-25"), - Intervals.of("2018-12-31/2019-01-01") - ); - final List skipIntervals = NewestSegmentFirstIterator.filterSkipIntervals( - totalInterval, - Lists.newArrayList( - Intervals.of("2017-12-01/2018-01-15"), - Intervals.of("2018-03-02/2018-07-23"), - Intervals.of("2018-10-01/2018-10-02"), - Intervals.of("2018-12-25/2018-12-31") - ) - ); - - Assert.assertEquals(expectedSkipIntervals, skipIntervals); - } - - @Test - public void testAddSkipIntervalFromLatestAndSort() - { - final List expectedIntervals = ImmutableList.of( - Intervals.of("2018-12-24/2018-12-25"), - Intervals.of("2018-12-29/2019-01-01") - ); - final List fullSkipIntervals = NewestSegmentFirstIterator.sortAndAddSkipIntervalFromLatest( - DateTimes.of("2019-01-01"), - new Period(72, 0, 0, 0), - null, - ImmutableList.of( - Intervals.of("2018-12-30/2018-12-31"), - Intervals.of("2018-12-24/2018-12-25") - ) - ); - - Assert.assertEquals(expectedIntervals, fullSkipIntervals); - } - - @Test - public void testFindPartitionsSpecFromConfigWithNullTuningConfigReturnDynamicPartitinosSpecWithMaxTotalRowsOfLongMax() - { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "datasource", - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); - Assert.assertEquals( - new DynamicPartitionsSpec(null, Long.MAX_VALUE), - CompactionStatus.findPartitionsSpecFromConfig( - ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null) - ) - ); - } - - @Test - public void testFindPartitionsSpecFromConfigWithNullMaxTotalRowsReturnLongMaxValue() - { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "datasource", - null, - null, - null, - null, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DynamicPartitionsSpec(null, null), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - null, - null - ); - Assert.assertEquals( - new DynamicPartitionsSpec(null, Long.MAX_VALUE), - CompactionStatus.findPartitionsSpecFromConfig( - ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null) - ) - ); - } - - @Test - public void testFindPartitionsSpecFromConfigWithNonNullMaxTotalRowsReturnGivenValue() - { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "datasource", - null, - null, - null, - null, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DynamicPartitionsSpec(null, 1000L), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - null, - null - ); - Assert.assertEquals( - new DynamicPartitionsSpec(null, 1000L), - CompactionStatus.findPartitionsSpecFromConfig( - ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null) - ) - ); - } - - @Test - public void testFindPartitionsSpecFromConfigWithNonNullMaxRowsPerSegmentReturnGivenValue() - { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "datasource", - null, - null, - null, - null, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DynamicPartitionsSpec(100, 1000L), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - null, - null - ); - Assert.assertEquals( - new DynamicPartitionsSpec(100, 1000L), - CompactionStatus.findPartitionsSpecFromConfig( - ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null) - ) - ); - } - - @Test - public void testFindPartitionsSpecFromConfigWithDeprecatedMaxRowsPerSegmentAndMaxTotalRowsReturnGivenValues() - { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "datasource", - null, - null, - 100, - null, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - 1000L, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - null, - null - ); - Assert.assertEquals( - new DynamicPartitionsSpec(100, 1000L), - CompactionStatus.findPartitionsSpecFromConfig( - ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null) - ) - ); - } - - @Test - public void testFindPartitionsSpecFromConfigWithDeprecatedMaxRowsPerSegmentAndPartitionsSpecIgnoreDeprecatedOne() - { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "datasource", - null, - null, - 100, - null, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DynamicPartitionsSpec(null, null), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - null, - null - ); - Assert.assertEquals( - new DynamicPartitionsSpec(null, Long.MAX_VALUE), - CompactionStatus.findPartitionsSpecFromConfig( - ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null) - ) - ); - } - - @Test - public void testFindPartitionsSpecFromConfigWithDeprecatedMaxTotalRowsAndPartitionsSpecIgnoreDeprecatedOne() - { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "datasource", - null, - null, - null, - null, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - 1000L, - null, - new DynamicPartitionsSpec(null, null), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - null, - null - ); - Assert.assertEquals( - new DynamicPartitionsSpec(null, Long.MAX_VALUE), - CompactionStatus.findPartitionsSpecFromConfig( - ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null) - ) - ); - } - - @Test - public void testFindPartitionsSpecFromConfigWithHashPartitionsSpec() - { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "datasource", - null, - null, - null, - null, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new HashedPartitionsSpec(null, 10, ImmutableList.of("dim")), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - null, - null - ); - Assert.assertEquals( - new HashedPartitionsSpec(null, 10, ImmutableList.of("dim")), - CompactionStatus.findPartitionsSpecFromConfig( - ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null) - ) - ); - } - - @Test - public void testFindPartitionsSpecFromConfigWithRangePartitionsSpec() - { - final DataSourceCompactionConfig config = new DataSourceCompactionConfig( - "datasource", - null, - null, - null, - null, - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new SingleDimensionPartitionsSpec(10000, null, "dim", false), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), - null, - null, - null, - null, - null, - null, - null - ); - Assert.assertEquals( - new SingleDimensionPartitionsSpec(10000, null, "dim", false), - CompactionStatus.findPartitionsSpecFromConfig( - ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null) - ) - ); - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java index 31e269f50f41..8f24a4ebb7e4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java @@ -88,7 +88,7 @@ public class NewestSegmentFirstPolicyTest public void testLargeOffsetAndSmallSegmentInterval() { final Period segmentPeriod = new Period("PT1H"); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P2D"), null)), ImmutableMap.of( DATA_SOURCE, @@ -113,7 +113,7 @@ public void testLargeOffsetAndSmallSegmentInterval() public void testSmallOffsetAndLargeSegmentInterval() { final Period segmentPeriod = new Period("PT1H"); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("PT1M"), null)), ImmutableMap.of( DATA_SOURCE, @@ -146,7 +146,7 @@ public void testSmallOffsetAndLargeSegmentInterval() public void testLargeGapInData() { final Period segmentPeriod = new Period("PT1H"); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("PT1H1M"), null)), ImmutableMap.of( DATA_SOURCE, @@ -179,7 +179,7 @@ public void testLargeGapInData() @Test public void testHugeShard() { - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P1D"), null)), ImmutableMap.of( DATA_SOURCE, @@ -229,7 +229,7 @@ public void testHugeShard() @Test public void testManySegmentsPerShard() { - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(800000, new Period("P1D"), null)), ImmutableMap.of( DATA_SOURCE, @@ -287,7 +287,7 @@ public void testSkipUnknownDataSource() { final String unknownDataSource = "unknown"; final Period segmentPeriod = new Period("PT1H"); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of( unknownDataSource, createCompactionConfig(10000, new Period("P2D"), null), @@ -337,7 +337,7 @@ public void testClearSegmentsToCompactWhenSkippingSegments() 2 ) ); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(inputSegmentSizeBytes, new Period("P0D"), null)), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -374,7 +374,7 @@ public void testIfFirstSegmentIsInSkipOffset() ) ); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), null)), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -395,7 +395,7 @@ public void testIfFirstSegmentOverlapsSkipOffset() ) ); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), null)), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -412,7 +412,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityEqual() new SegmentGenerateSpec(Intervals.of("2017-10-14T00:00:00/2017-10-15T00:00:00"), new Period("P1D")) ); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -445,7 +445,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityLarger() new SegmentGenerateSpec(Intervals.of("2017-10-14T00:00:00/2017-10-15T00:00:00"), new Period("PT5H")) ); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -471,7 +471,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularitySmaller() new SegmentGenerateSpec(Intervals.of("2017-10-14T00:00:00/2017-10-15T00:00:00"), new Period("PT5H")) ); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.MINUTE, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -496,7 +496,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularitySmaller() public void testWithSkipIntervals() { final Period segmentPeriod = new Period("PT1H"); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P1D"), null)), ImmutableMap.of( DATA_SOURCE, @@ -536,7 +536,7 @@ public void testWithSkipIntervals() public void testHoleInSearchInterval() { final Period segmentPeriod = new Period("PT1H"); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("PT1H"), null)), ImmutableMap.of( DATA_SOURCE, @@ -586,7 +586,7 @@ public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity() new SegmentGenerateSpec(Intervals.of("2017-10-01T00:00:00/2017-12-31T00:00:00"), new Period("P1D")) ); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -635,7 +635,7 @@ public void testIteratorReturnsSegmentsInMultipleIntervalIfConfiguredSegmentGran new SegmentGenerateSpec(Intervals.of("2020-02-08/2020-02-15"), new Period("P7D")) ); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -670,7 +670,7 @@ public void testIteratorDoesNotReturnCompactedInterval() new SegmentGenerateSpec(Intervals.of("2017-12-01T00:00:00/2017-12-02T00:00:00"), new Period("P1D")) ); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MINUTE, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -696,7 +696,7 @@ public void testIteratorReturnsAllMixedVersionSegmentsInConfiguredSegmentGranula new SegmentGenerateSpec(Intervals.of("2017-10-01T01:00:00/2017-10-01T02:00:00"), new Period("PT1H"), "1994-04-30T00:00:00.000Z", null) ); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -721,7 +721,7 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr // Same indexSpec as what is set in the auto compaction config Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( @@ -740,7 +740,7 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr ); // Auto compaction config sets segmentGranularity=DAY - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -754,7 +754,7 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr // Same indexSpec as what is set in the auto compaction config Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( @@ -773,7 +773,7 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr ); // Auto compaction config sets segmentGranularity=DAY - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -787,7 +787,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg // Same indexSpec as what is set in the auto compaction config Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( @@ -806,7 +806,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg ); // Auto compaction config sets segmentGranularity=YEAR - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -830,7 +830,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg // Same indexSpec as what is set in the auto compaction config Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( @@ -849,7 +849,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg ); // Auto compaction config sets segmentGranularity=YEAR - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -873,7 +873,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTim // Same indexSpec as what is set in the auto compaction config Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( @@ -887,7 +887,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTim // Duration of new segmentGranularity is the same as before (P1D), // but we changed the timezone from UTC to Bangkok in the auto compaction spec - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig( 130000, @@ -925,7 +925,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOri // Same indexSpec as what is set in the auto compaction config Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( @@ -938,7 +938,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOri ); // Duration of new segmentGranularity is the same as before (P1D), but we changed the origin in the autocompaction spec - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig( 130000, @@ -976,7 +976,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol // Same indexSpec as what is set in the auto compaction config Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have // rollup=false for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, @@ -1004,7 +1004,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol ); // Auto compaction config sets rollup=true - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(null, null, true))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -1036,7 +1036,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue // Same indexSpec as what is set in the auto compaction config Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have // queryGranularity=DAY for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, @@ -1064,7 +1064,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue ); // Auto compaction config sets queryGranularity=MINUTE - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(null, Granularities.MINUTE, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -1096,7 +1096,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim // Same indexSpec as what is set in the auto compaction config Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have // Dimensions=["foo", "bar"] for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, @@ -1131,7 +1131,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim ); // Auto compaction config sets Dimensions=["foo"] - CompactionSegmentIterator iterator = policy.reset( + CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig( 130000, new Period("P0D"), @@ -1172,7 +1172,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim Assert.assertFalse(iterator.hasNext()); // Auto compaction config sets Dimensions=null - iterator = policy.reset( + iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig( 130000, new Period("P0D"), @@ -1195,7 +1195,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil // Same indexSpec as what is set in the auto compaction config Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have // filter=SelectorDimFilter("dim1", "foo", null) for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, @@ -1251,7 +1251,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil ); // Auto compaction config sets filter=SelectorDimFilter("dim1", "bar", null) - CompactionSegmentIterator iterator = policy.reset( + CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig( 130000, new Period("P0D"), @@ -1292,7 +1292,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil Assert.assertFalse(iterator.hasNext()); // Auto compaction config sets filter=null - iterator = policy.reset( + iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig( 130000, new Period("P0D"), @@ -1319,7 +1319,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet // Same indexSpec as what is set in the auto compaction config Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have // metricsSpec={CountAggregatorFactory("cnt")} for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, @@ -1375,7 +1375,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet ); // Auto compaction config sets metricsSpec={CountAggregatorFactory("cnt"), LongSumAggregatorFactory("val", "val")} - CompactionSegmentIterator iterator = policy.reset( + CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig( 130000, new Period("P0D"), @@ -1416,7 +1416,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet Assert.assertFalse(iterator.hasNext()); // Auto compaction config sets metricsSpec=null - iterator = policy.reset( + iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig( 130000, new Period("P0D"), @@ -1440,7 +1440,7 @@ public void testIteratorReturnsSegmentsSmallerSegmentGranularityCoveringMultiple new SegmentGenerateSpec(Intervals.of("2017-10-01T01:00:00/2017-10-01T02:00:00"), new Period("PT1H"), "1994-04-30T00:00:00.000Z", null) ); - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() @@ -1468,7 +1468,7 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat // Different indexSpec as what is set in the auto compaction config IndexSpec newIndexSpec = IndexSpec.builder().withBitmapSerdeFactory(new ConciseBitmapSerdeFactory()).build(); Map newIndexSpecMap = mapper.convertValue(newIndexSpec, new TypeReference>() {}); - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( @@ -1481,7 +1481,7 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat ); // Duration of new segmentGranularity is the same as before (P1D) - final CompactionSegmentIterator iterator = policy.reset( + final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig( 130000, @@ -1517,7 +1517,7 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() { NullHandling.initializeForTests(); - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); final SegmentTimeline timeline = createTimeline( new SegmentGenerateSpec( Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), @@ -1534,7 +1534,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() ) ); - CompactionSegmentIterator iterator = policy.reset( + CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig( 130000, new Period("P0D"), @@ -1569,7 +1569,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() ); Assert.assertFalse(iterator.hasNext()); - iterator = policy.reset( + iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig( 130000, new Period("P0D"), @@ -1608,7 +1608,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() @Test public void testSkipAllGranularityToDefault() { - CompactionSegmentIterator iterator = policy.reset( + CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P0D"), @@ -1640,7 +1640,7 @@ public void testSkipAllGranularityToDefault() @Test public void testSkipFirstHalfEternityToDefault() { - CompactionSegmentIterator iterator = policy.reset( + CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P0D"), @@ -1672,7 +1672,7 @@ public void testSkipFirstHalfEternityToDefault() @Test public void testSkipSecondHalfOfEternityToDefault() { - CompactionSegmentIterator iterator = policy.reset( + CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P0D"), @@ -1704,7 +1704,7 @@ public void testSkipSecondHalfOfEternityToDefault() @Test public void testSkipAllToAllGranularity() { - CompactionSegmentIterator iterator = policy.reset( + CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P0D"), @@ -1736,7 +1736,7 @@ public void testSkipAllToAllGranularity() @Test public void testSkipAllToFinerGranularity() { - CompactionSegmentIterator iterator = policy.reset( + CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P0D"), @@ -1799,7 +1799,7 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() 0, 1); - CompactionSegmentIterator iterator = policy.reset( + CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P0D"), @@ -1850,7 +1850,7 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() TombstoneShardSpec.INSTANCE, 0, 1); - iterator = policy.reset( + iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P0D"), 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 312b6f6b05a3..5decef93018c 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.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 DataNodeService getDataNodeService(DruidServerConfig serverConfig) } }, 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 37b6501bf87f..2e542bc69745 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 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.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 WorkerNodeService getWorkerNodeService(WorkerConfig workerConfig) } }, 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 24e98427ce95..636d775f030f 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.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; @@ -58,7 +57,6 @@ import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionToolbox; -import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; @@ -68,6 +66,7 @@ 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; @@ -75,9 +74,9 @@ import org.apache.druid.indexing.overlord.RemoteTaskRunnerFactory; 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.TaskRunnerFactory; import org.apache.druid.indexing.overlord.TaskStorage; -import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter; import org.apache.druid.indexing.overlord.autoscaling.PendingTaskBasedWorkerProvisioningConfig; import org.apache.druid.indexing.overlord.autoscaling.PendingTaskBasedWorkerProvisioningStrategy; import org.apache.druid.indexing.overlord.autoscaling.ProvisioningSchedulerConfig; @@ -108,10 +107,10 @@ 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; @@ -209,9 +208,9 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.indexer.tasklock", TaskLockConfig.class); JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); JsonConfigProvider.bind(binder, "druid.indexer.task.default", DefaultTaskConfig.class); - JsonConfigProvider.bind(binder, "druid.indexer.auditlog", TaskAuditLogConfig.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); @@ -231,7 +230,7 @@ public void configure(Binder binder) binder.bind(TaskActionClientFactory.class).to(LocalTaskActionClientFactory.class).in(LazySingleton.class); binder.bind(TaskActionToolbox.class).in(LazySingleton.class); binder.bind(TaskLockbox.class).in(LazySingleton.class); - binder.bind(TaskStorageQueryAdapter.class).in(LazySingleton.class); + binder.bind(TaskQueryTool.class).in(LazySingleton.class); binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class); binder.bind(SupervisorManager.class).in(LazySingleton.class); @@ -384,11 +383,11 @@ public TaskStorageDirTracker getTaskStorageDirTracker(WorkerConfig workerConfig, @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; }; @@ -428,7 +427,6 @@ private void configureOverlordHelpers(Binder binder) .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 f78a763cec4d..eb572850cda2 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.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; @@ -78,7 +77,6 @@ import org.apache.druid.indexing.common.actions.RemoteTaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionToolbox; -import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; @@ -117,11 +115,11 @@ 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; @@ -344,7 +342,6 @@ public LookupLoadingSpec getLookupsToLoad(final Task task) } }, new QueryablePeonModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTuningConfigModule(), new InputSourceModule(), @@ -483,7 +480,6 @@ static void bindTaskConfigAndClients(Binder binder) binder.bind(TaskToolboxFactory.class).in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); - JsonConfigProvider.bind(binder, "druid.indexer.auditlog", TaskAuditLogConfig.class); JsonConfigProvider.bind(binder, "druid.peon.taskActionClient.retry", RetryPolicyConfig.class); configureTaskActionClient(binder); 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 60415ea4f323..0f77da1b4c4e 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 0518c7259e63..50684cee0e87 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.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 void run() 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 000000000000..d33c05838a2e --- /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 b843465147eb..b617b7e6b877 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 void testTaskValidator() throws Exception 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 eceb4ebbf800..7d66eebcad1b 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 @@ 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 @@ */ 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 Aggregation toDruidAggregation( ); } + @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/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAuditLogConfig.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/NativelySupportsDistinct.java similarity index 57% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAuditLogConfig.java rename to sql/src/main/java/org/apache/druid/sql/calcite/aggregation/NativelySupportsDistinct.java index c78aec4f39b5..19bbaf8a0f26 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAuditLogConfig.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/NativelySupportsDistinct.java @@ -17,30 +17,20 @@ * under the License. */ -package org.apache.druid.indexing.common.actions; +package org.apache.druid.sql.calcite.aggregation; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; /** - * The configuration for task audit logging. - * This class will be removed in future releases. See https://github.com/apache/druid/issues/5859. + * This annotation is to distinguish {@link org.apache.calcite.sql.SqlAggFunction} + * which supports the distinct aggregation natively */ -@Deprecated -public class TaskAuditLogConfig +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE}) +public @interface NativelySupportsDistinct { - @JsonProperty - private final boolean enabled; - @JsonCreator - public TaskAuditLogConfig(@JsonProperty("enabled") boolean enabled) - { - this.enabled = enabled; - } - - @JsonProperty("enabled") - public boolean isEnabled() - { - return enabled; - } } 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 a5e62f5e2a9b..d20999d3afc4 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.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 Aggregation toDruidAggregation( } } + @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 efb84dca6251..1045a79870bb 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.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 RelDataType inferReturnType(SqlOperatorBinding sqlOperatorBinding) } } + @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 699c7a8d1c6b..c756aa64cc31 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.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 Aggregation toDruidAggregation( 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 Aggregation toDruidAggregation( } 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 @@ private static class BuiltinApproxCountDistinctSqlAggFunction extends SqlAggFunc 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 @@ private static class BuiltinApproxCountDistinctSqlAggFunction extends SqlAggFunc ); } } + + 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 a78b3a7a4797..49469decf996 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.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 RelDataType inferReturnType(SqlOperatorBinding sqlOperatorBinding) } } + @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/planner/DruidSqlValidator.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java index 75778daf5593..e00a2915a2e8 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.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; @@ -36,6 +37,7 @@ import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlOperatorTable; 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; @@ -64,6 +66,7 @@ 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; @@ -110,6 +113,10 @@ protected DruidSqlValidator( @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 +762,10 @@ public void validateCall(SqlCall call, SqlValidatorScope scope) 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,6 +780,19 @@ public void validateCall(SqlCall call, SqlValidatorScope scope) 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); } @@ -857,4 +879,11 @@ private SqlNode getSqlNodeFor(SqlInsert insert, int idx) } return src; } + + private boolean isSqlCallDistinct(@Nullable SqlCall call) + { + return call != null + && call.getFunctionQuantifier() != null + && call.getFunctionQuantifier().getValue() == SqlSelectKeyword.DISTINCT; + } } 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 a7d534db7002..bec5335f7c95 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 @@ private ScanQuery toScanQuery(final boolean considerSorting) 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 afd775ef4eef..8e1fc3ee2750 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 static Windowing fromCalciteStuff( 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 WindowFrame getWindowFrame() 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 fecabd00ec39..f0632006d106 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 @@ 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 static Aggregation translateAggregateCall( 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 d02d302437b8..2477ac38dec1 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 boolean featureAvailable(EngineFeature feature) 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 boolean featureAvailable(EngineFeature feature) 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 a2e45c4af8c2..0ad2ba48c58c 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 static ExpressionPostAggregator expressionPostAgg(final String name, fina 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 @@ protected void skipVectorize() 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 10abd67c2b39..2fe1919f6a16 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 void testArraySlice() .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 void testArraySliceArrayColumns() .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 void testArrayAggAsArrayFromJoin() .columns("dim4", "j0.a0", "v0") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), @@ -3840,7 +3837,6 @@ public void testArrayAggArrayContainsSubquery() .columns("dim1", "dim2") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), @@ -3950,7 +3946,6 @@ public void testUnnestInline() ) .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 void testUnnest() )) .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 void testUnnestArrayColumnsString() )) .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 void testUnnestArrayColumnsStringNulls() )) .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 void testUnnestArrayColumnsLong() )) .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 void testUnnestArrayColumnsLongNulls() )) .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 void testUnnestArrayColumnsDouble() )) .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 void testUnnestArrayColumnsDoubleNulls() )) .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 void testUnnestTwice() ) ) .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 void testUnnestTwiceArrayColumns() ) .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 void testUnnestTwiceWithFiltersAndExpressions() ) ) .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 void testUnnestThriceWithFiltersOnDimAndUnnestCol() ) .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 void testUnnestThriceWithFiltersOnDimAndAllUnnestColumns() "'Baz'", ColumnType.STRING )) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "dimZipf", "v0")) .build() @@ -4826,7 +4809,6 @@ public void testUnnestThriceWithFiltersOnDimAndAllUnnestColumnsArrayColumns() 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 void testUnnestThriceWithFiltersOnDimAndUnnestColumnsORCombinations() ) .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 void testUnnestThriceWithFiltersOnDimAndAllUnnestColumnsArrayColumnsOrFil .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 void testUnnestWithLimit() )) .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 void testUnnestFirstQueryOnSelect() )) .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 void testUnnestVirtualWithColumns1() )) .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 void testUnnestVirtualWithColumns2() )) .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 void testUnnestWithFilters() )) .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 void testUnnestWithFiltersWithExpressionInInnerQuery() 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 void testUnnestWithInFiltersWithExpressionInInnerQuery() "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 void testUnnestWithFiltersInnerLimit() ) .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 void testUnnestWithFiltersInnerLimit() )) .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 void testUnnestWithFiltersInsideAndOutside() )) .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 void testUnnestWithFiltersInsideAndOutside1() )) .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 void testUnnestWithFiltersOutside() )) .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 void testUnnestWithInFilters() )) .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 void testUnnestVirtualWithColumns() )) .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 void testUnnestWithJoinOnTheLeft() ) .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 void testUnnestWithJoinOnTheLeft() )) .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 void testUnnestWithConstant() ) .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 void testUnnestWithSQLFunctionOnUnnestedColumn() .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 void testUnnestWithINFiltersWithLeftRewrite() )) .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 void testUnnestWithINFiltersWithNoLeftRewrite() )) .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 void testUnnestWithInvalidINFiltersOnUnnestedColumn() )) .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 void testUnnestWithNotFiltersOnUnnestedColumn() )) .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 void testUnnestWithSelectorFiltersOnSelectedColumn() )) .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 void testUnnestWithSelectorFiltersOnVirtualColumn() )) .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 void testUnnestWithSelectorFiltersOnVirtualStringColumn() )) .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 void testUnnestWithMultipleAndFiltersOnSelectedColumns() )) .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 void testUnnestWithMultipleOrFiltersOnSelectedColumns() )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .filters( or( @@ -6185,7 +6139,6 @@ public void testUnnestWithMultipleAndFiltersOnSelectedUnnestedColumns() )) .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 void testUnnestWithMultipleOrFiltersOnUnnestedColumns() )) .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 void testUnnestWithMultipleOrFiltersOnVariationsOfUnnestedColumns() )) .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 void testUnnestWithMultipleOrFiltersOnSelectedNonUnnestedColumns() )) .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 void testUnnestWithMultipleOrFiltersOnSelectedVirtualColumns() )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .filters( or( @@ -6361,7 +6310,6 @@ public void testUnnestWithMultipleOrFiltersOnUnnestedColumnsAndOnOriginalColumn( equality("dim3", "d", ColumnType.STRING) ) ) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) .build() @@ -6395,7 +6343,6 @@ public void testUnnestWithMultipleOrFiltersOnUnnestedColumnsAndOnOriginalColumnD equality("j0.unnest", "a", ColumnType.STRING) ) ) - .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("dim3", "j0.unnest")) .build() @@ -6686,8 +6633,8 @@ public void testUnnestVirtualWithColumnsAndNullIf() ? 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 void testUnnestWithTimeFilterOnly() )) .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 void testUnnestWithTimeFilterOnlyArrayColumn() )) .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 void testUnnestWithTimeFilterAndAnotherFilter() )) .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 void testUnnestWithTimeFilterOrAnotherFilter() )) .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 void testUnnestWithTimeFilterOnlyNested() )) .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 void testUnnestWithTimeFilterOnlyNestedAndNestedAgain() )) .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 void testUnnestWithTimeFilterInsideSubquery() .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 void testUnnestWithTimeFilterInsideSubquery() : 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 void testUnnestWithTimeFilterInsideSubqueryArrayColumns() .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 void testUnnestWithTimeFilterInsideSubqueryArrayColumns() 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 void testUnnestWithFilterAndUnnestNestedBackToBack() ) .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 void testUnnestWithLookup() .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 void testUnnestWithGroupByOnExpression() ) .eternityInterval() .columns("d0", "j0.unnest") - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -7322,7 +7256,6 @@ public void testUnnestExtractionFn() .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 void testUnnestExtractionFnNull() .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 3d7c820902bd..4be552569926 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 @@ -131,12 +131,12 @@ public void testExplainExactCountDistinctOfSemiJoinResult() ? "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\":\"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\"}}], 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\":\"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\"}}], 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() ? "[" @@ -184,14 +184,13 @@ public void testExplainSelectStarWithOverrides() // 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 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\"],\"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\"}]," @@ -204,7 +203,6 @@ public void testExplainSelectStarWithOverrides() + "\"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\"}]," @@ -256,33 +254,33 @@ public void testExplainMultipleTopLevelUnionAllQueries() + "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" + + " 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\"],\"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" + + " 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\"],\"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\"],\"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" + + " 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\"],\"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"; + + " 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\"],\"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\"],\"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\"}}," + + "\"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\"}}," + "\"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\"}}," + + "\"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\"}}," + "\"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\"}}," + + "\"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\"}}," + "\"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\"}},\"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\"}},\"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\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]}]"; final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"SELECT\"}"; testQuery( @@ -331,7 +329,7 @@ public void testExplainSelectMvfilterExpressions() + "]," + "\"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\"}}," @@ -358,7 +356,7 @@ public void testExplainSelectMvfilterExpressions() + "]," + "\"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\"}}," @@ -398,7 +396,7 @@ public void testExplainSelectTimestampExpression() + "]," + "\"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\"}}," 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 244d7226acbf..af77671cee41 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 void testReplaceIntoExtern() .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 void testInsertIntoExtern() .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 void testInsertIntoExternParameterized() .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 void testReplaceIntoExternParameterized() .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 void testSelectFromTableNamedExport() .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 b40a4c87c3ab..d369c23bc1ac 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 @@ -777,7 +777,7 @@ public void testExplainPlanInsertWithClusteredBy() throws JsonProcessingExceptio + "{\"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," + + "{\"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\"}}," + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"}," + "{\"name\":\"v1\",\"type\":\"DOUBLE\"}]," @@ -843,7 +843,7 @@ public void testExplainPlanInsertWithAsSubQueryClusteredBy() + "\"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\\\"}\"," + + "\"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\"}}]," + " signature=[{__time:LONG, namespace:STRING, country:STRING}])\n"; @@ -875,7 +875,7 @@ public void testExplainPlanInsertWithAsSubQueryClusteredBy() + "\"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\\\"}\"," + + "\"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\"}}," + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]," + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"namespace\",\"outputColumn\":\"namespace\"}," @@ -943,15 +943,15 @@ public void testExplainPlanInsertJoinQuery() 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\"," + + "{\"columnName\":\"Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"Capital\",\"isRobot\",\"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\"}}], 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," + + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"cityName\",\"countryIsoCode\",\"isRobot\",\"regionName\",\"timestamp\"]," + "\"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"; + + "\"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\"}}], signature=[{Capital:STRING, ISO2:STRING}])\n"; // Use testQuery for EXPLAIN (not testIngestionQuery). testQuery( PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, @@ -977,10 +977,10 @@ public void testExplainPlanInsertJoinQuery() + ",\"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\"," + + "\"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\"}}},\"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\"," + + "{\"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\"}},\"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\"}]}]"; @@ -1310,7 +1310,7 @@ public void testExplainInsertFromExternal() throws IOException + "\"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," + + "\"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\"}},\"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\"}]" @@ -1411,7 +1411,7 @@ public void testExplainPlanForInsertWithClusteredBy() throws JsonProcessingExcep + "{\"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," + + "{\"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\"}}," + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"}," + "{\"name\":\"v1\",\"type\":\"DOUBLE\"}]," 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 8f3fde509d5b..264ebf6890ef 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 void testLeftJoinRightTableCanBeEmpty() .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .filters(equality("m2", "1000", ColumnType.DOUBLE)) .columns("dim2") - .legacy(false) .build() ); @@ -3545,7 +3544,6 @@ public void testLeftJoinRightTableCanBeEmpty() false )) .columns("dim2") - .legacy(false) .build() ), rightTable, @@ -5633,7 +5631,6 @@ public void testPlanWithInFilterMoreThanInSubQueryThreshold() .columns("l1") .intervals(querySegmentSpec(Filtration.eternity())) .context(queryContext) - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() ), @@ -6173,7 +6170,6 @@ public void testJoinsWithUnnestOnLeft() .intervals(querySegmentSpec(Filtration.eternity())) .dataSource(CalciteTests.DATASOURCE3) .columns("dim2") - .legacy(false) .context(context) .build() ), @@ -6232,7 +6228,6 @@ public void testJoinsWithUnnestOverFilteredDSOnLeft() .intervals(querySegmentSpec(Filtration.eternity())) .dataSource(CalciteTests.DATASOURCE3) .columns("dim2") - .legacy(false) .context(context) .build() ), @@ -6287,7 +6282,6 @@ public void testJoinsWithUnnestOverJoin() .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 void testJoinsWithUnnestOverJoin() .intervals(querySegmentSpec(Filtration.eternity())) .dataSource(CalciteTests.DATASOURCE3) .columns("dim2") - .legacy(false) .context(context) .build() ), @@ -6379,7 +6372,6 @@ public void testSelfJoinsWithUnnestOnLeftAndRight() null )) .columns("dim2", "j0.unnest") - .legacy(false) .context(context) .build() ), @@ -6451,7 +6443,6 @@ public void testJoinsOverUnnestOverFilterDSOverJoin() ImmutableList.of("a", "ab", "abc", "b") ) ) - .legacy(false) .context(context) .build() ), @@ -6471,7 +6462,6 @@ public void testJoinsOverUnnestOverFilterDSOverJoin() .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 cb0a836fb24f..ba4e32b81017 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 void testLookupOnValueThatIsNull() 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 void testLookupOnValueThatIsNotDistinctFromNull() 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 @@ private List> buildFilterTestExpectedQueryAlwaysFalse() .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 275b4e9a164f..8b21d336fdc3 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 void testMultiValueStringWorksLikeStringScan() .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 void testMultiValueStringWorksLikeStringSelfConcatScan() .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 void testMultiValueStringWorksLikeStringScanWithFilter() .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 void testMultiValueStringSlice() .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 e2bc9d45eb27..9e6e5da2bb0d 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.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 void testJsonValueArrays() ) .columns("v0", "v1", "v2", "v3") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) ) @@ -1270,7 +1270,6 @@ public void testUnnestRootSingleTypeArrayLongNulls() .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 void testUnnestRootSingleTypeArrayStringNulls() .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 void testUnnestRootSingleTypeArrayDoubleNulls() .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 void testJsonQuery() ) .columns("v0", "v1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -4908,7 +4904,6 @@ public void testJsonQueryAndJsonObject() ) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -4956,7 +4951,6 @@ public void testCompositionTyping() ) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5007,7 +5001,6 @@ public void testToJsonAndParseJson() ) .columns("string", "v0", "v1", "v2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5103,7 +5096,6 @@ public void testJsonPathNegativeIndex() ) .columns("v0", "v1", "v2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5140,7 +5132,6 @@ public void testJsonPathsNonJsonInput() ) .columns("v0", "v1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5179,7 +5170,6 @@ public void testJsonKeysNonJsonInput() ) .columns("v0", "v1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5308,7 +5298,6 @@ public void testSelectPathSelectorFilterVariantNull() .filters(isNull("v0")) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -5512,7 +5501,6 @@ public void testScanStringNotNullCast() .filters(notNull("v0")) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), expectedResults, @@ -6018,7 +6006,6 @@ public void testScanAllTypesAuto() "variantWithArrays" ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), useDefault ? @@ -6612,7 +6599,6 @@ public void testFilterJsonIsNotNull() .columns("nest") .filters(notNull("nest")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), NullHandling.replaceWithDefault() @@ -6642,7 +6628,6 @@ public void testFilterJsonIsNull() .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 void testCoalesceOnNestedColumns() .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 void testCoalesceOnNestedColumnsLater() .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 void testCoalesceOnNestedColumnsLater() .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 void testJsonQueryDynamicArg() ) .columns("v0", "v1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -6888,7 +6869,6 @@ public void testJsonQueryArrays() ) .columns("v0") .context(QUERY_CONTEXT_DEFAULT) - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() ) @@ -6943,7 +6923,6 @@ public void testJsonQueryArrayNullArray() .columns("v0") .limit(1) .context(QUERY_CONTEXT_DEFAULT) - .legacy(false) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build() ) @@ -6983,7 +6962,6 @@ public void testUnnestJsonQueryArrays() .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 void testJsonValueNestedEmptyArray() ) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -7321,6 +7298,33 @@ public void testNvlJsonValueDoubleSometimesMissingRangeFilter() ); } + @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 6bd273696e2a..bd9c5649df17 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 void testSelectConstantParamGetsConstant() .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -685,7 +684,6 @@ public void testWrongTypeParameter() .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 2ae095d41c74..5ceba91eb37c 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 @@ -1351,7 +1351,6 @@ public void testStringLatestGroupByWithAlwaysFalseCondition() .columns("EXPR$0", "dim2") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of() @@ -1376,7 +1375,6 @@ public void testStringLatestByGroupByWithAlwaysFalseCondition() .columns("EXPR$0", "dim2") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of() @@ -2528,7 +2526,6 @@ public void testDECODE_BASE64_UTF8() .limit(1) .columns(ImmutableList.of("v0")) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -3470,7 +3467,6 @@ public void testNullLongFilter() .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) : ImmutableList.of( @@ -3508,7 +3504,6 @@ public void testNullDoubleFilter() .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) : ImmutableList.of( @@ -3546,7 +3541,6 @@ public void testNullFloatFilter() .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) : ImmutableList.of( @@ -3965,7 +3959,6 @@ public void testNullStringEquality() .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of(new Object[]{0L}) @@ -4243,7 +4236,6 @@ public void testGroupByNothingWithLiterallyFalseFilter() .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0", "EXPR$1") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -4316,7 +4308,6 @@ public void testGroupByOneColumnWithLiterallyFalseFilter() .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0", "EXPR$1") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of() @@ -6389,7 +6380,6 @@ public void testCountStarWithNotOfDegenerateFilter() .intervals(querySegmentSpec(Filtration.eternity())) .columns("EXPR$0") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of( @@ -6777,7 +6767,6 @@ public void testTimeInIntervalBooleanNullable() .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 +12698,6 @@ public void testConcatDecomposeIsNull() .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1", "dim2", "dimX") .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), @@ -13851,7 +13839,6 @@ public void testRoundFunc() 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 +14919,6 @@ public void testPlanWithInFilterLessThanInSubQueryThreshold() .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 +15164,6 @@ public void testComplexDecode() ) ) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .limit(10) .build() ), @@ -15516,6 +15501,20 @@ public void testWindowingErrorWithoutFeatureFlag() 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 +15576,22 @@ public void testNtileNotSupportedWithFrame() 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 testInGroupByLimitOutGroupByOrderBy() { @@ -15818,7 +15833,6 @@ public void testWindowingWithScanAndSort() .columns("a0", "w0") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .limit(10) .build() ) @@ -15886,7 +15900,7 @@ public void testWindowingWithOrderBy() OperatorFactoryBuilders.naivePartitionOperator(), OperatorFactoryBuilders.windowOperators( OperatorFactoryBuilders.framedAggregateProcessor( - WindowFrame.forOrderBy(ColumnWithDirection.ascending("d0")), + WindowFrame.forOrderBy("d0"), new LongSumAggregatorFactory("w0", "a0") ) ) @@ -15938,7 +15952,7 @@ public void testScanAndSortOnJoin() .run(); } - @NotYetSupported(Modes.WINDOW_OPERATOR_QUERY_ON_UNSUPPORTED_DATASOURCE) + @NotYetSupported(Modes.CANNOT_RETRIEVE_ROWS) @Test public void testWindowingOverJoin() { @@ -15993,7 +16007,6 @@ public void testCastCharToVarcharInFlattenConcat() ) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .columns(ImmutableList.of( "EXPR$0" )) @@ -16078,7 +16091,6 @@ public void testIpv4ParseWithNullableType() .columns("EXPR$0") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) : ImmutableList.of( @@ -16094,7 +16106,6 @@ public void testIpv4ParseWithNullableType() .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 +16216,6 @@ public void testStringOperationsNullableInference() .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 444db2f42040..49db00c2519e 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 @@ -705,7 +705,7 @@ public void testExplainReplaceFromExternal() throws IOException + "\"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," + + "\"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\"]," @@ -790,7 +790,7 @@ public void testExplainReplaceTimeChunksWithPartitioningAndClustering() throws I + "], 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\"}},\"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\"}"; @@ -883,7 +883,7 @@ public void testExplainReplaceWithLimitAndClusteredByOrdinals() throws IOExcepti + "\"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\"," + + "\"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\"]," 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 d2699e77f79b..efe04f99a630 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 @@ -92,7 +92,6 @@ public void testSelectConstantExpression() ) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -126,7 +125,6 @@ public void testExpressionContainingNull() ) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -176,7 +174,6 @@ public void testValuesContainingNull() .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 +201,6 @@ public void testMultipleValuesContainingNull() .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 +228,6 @@ public void testMultipleValuesContainingNullAndIntegerValues() .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 +265,6 @@ public void testSelectNonNumericNumberLiterals() .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 +491,6 @@ public void testBitwiseExpressions() ) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), expected @@ -533,7 +526,6 @@ public void testSafeDivideWithoutTable() expressionVirtualColumn("v4", "5", ColumnType.LONG) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(context) .build() ), @@ -591,7 +583,6 @@ public void testSafeDivideExpressions() ) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), expected @@ -610,13 +601,12 @@ public void testExplainSelectConstantExpression() + "\"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\"}," + 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\"],\"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 resources = "[]"; final String attributes = "{\"statementType\":\"SELECT\"}"; @@ -896,7 +886,6 @@ public void testSelectNonAggregatingWithLimitLiterallyZero() .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -922,7 +911,6 @@ public void testSelectNonAggregatingWithLimitReducedToZero() .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of() @@ -947,7 +935,6 @@ public void testSelectAggregatingWithLimitReducedToZero() .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ), ImmutableList.of() @@ -1012,7 +999,6 @@ public void testSelectCurrentTimeAndDateLosAngeles() .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,14 +1330,13 @@ public void testExplainSelectStar() 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 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\"],\"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\"}]," 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 6269e2a5c8cc..c1236162fa02 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.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 void testEmptyGroupWithOffsetDoesntInfiniteLoop(String testName, Map query .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .columns("__time") - .legacy(false) .context(queryContext) .build()), "j0.", @@ -1307,7 +1307,6 @@ public void testSingleValueStringAgg(String testName, Map queryC .limit(1L) .order(ScanQuery.Order.DESCENDING) .columns("__time", "channel") - .legacy(false) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -1388,6 +1387,220 @@ public void testSingleValueEmptyInnerAgg(String testName, Map qu ); } + @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 8b6170cbfaac..02b8787df233 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 void testUnion() .virtualColumns( expressionVirtualColumn("v0", "null", null) ) - .legacy(false) .build(), Druids.newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) @@ -57,7 +56,6 @@ public void testUnion() .columns("dim1", "dim4") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) ) @@ -94,7 +92,6 @@ public void testAppend2() .columns("d1", "dim1", "dim4", "f1") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) .expectedResults( @@ -137,7 +134,6 @@ public void testAppendSameTableMultipleTimes() .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 void testAppendtSingleTableIsValid() .columns("dim1") .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) .build() ) .expectedResults( @@ -196,7 +191,6 @@ public void testAppendCompatibleColumns() .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 cbf0e7851a90..042812933d2b 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 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,18 @@ 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 + ); + + private static final Map DEFAULT_QUERY_CONTEXT_WITH_SUBQUERY_BYTES = + ImmutableMap.builder() + .putAll(DEFAULT_QUERY_CONTEXT) + .put(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000") + .put(QueryContexts.MAX_SUBQUERY_ROWS_KEY, "0") + .build(); + public static Object[] parametersForWindowQueryTest() throws Exception { final URL windowFolderUrl = ClassLoader.getSystemResource("calcite/tests/window"); @@ -184,6 +198,11 @@ private void maybeDumpActualResults(List results) throws Exception log.info("Actual results:\n%s", sb.toString()); } } + + public Map getQueryContext() + { + return input.queryContext == null ? Collections.emptyMap() : input.queryContext; + } } @MethodSource("parametersForWindowQueryTest") @@ -199,10 +218,12 @@ public void windowQueryTest(String filename) throws Exception testBuilder() .skipVectorize(true) .sql(testCase.getSql()) - .queryContext(ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true - )) + .queryContext( + ImmutableMap.builder() + .putAll(DEFAULT_QUERY_CONTEXT) + .putAll(testCase.getQueryContext()) + .build() + ) .addCustomVerification(QueryVerification.ofResults(testCase)) .run(); } @@ -211,7 +232,7 @@ public void windowQueryTest(String filename) throws Exception @MethodSource("parametersForWindowQueryTest") @ParameterizedTest(name = "{0}") @SuppressWarnings("unchecked") - public void windowQueryTestWithCustomContextMaxSubqueryBytes(String filename) throws Exception + public void windowQueryTestsWithSubqueryBytes(String filename) throws Exception { TestCase testCase = new TestCase(filename); @@ -221,11 +242,11 @@ public void windowQueryTestWithCustomContextMaxSubqueryBytes(String filename) th 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", - QueryContexts.MAX_SUBQUERY_ROWS_KEY, "0" - ) + .queryContext( + ImmutableMap.builder() + .putAll(DEFAULT_QUERY_CONTEXT_WITH_SUBQUERY_BYTES) + .putAll(testCase.getQueryContext()) + .build() ) .addCustomVerification(QueryVerification.ofResults(testCase)) .run(); @@ -233,91 +254,29 @@ public void windowQueryTestWithCustomContextMaxSubqueryBytes(String filename) th } @Test - public void testEmptyWindowInSubquery() + public void testWithArrayConcat() { 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" + .sql("select countryName, cityName, channel, " + + "array_concat_agg(ARRAY['abc', channel], 10000) over (partition by cityName order by countryName) as c\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} - )) + + "where countryName in ('Austria', 'Republic of Korea') " + + "and (cityName in ('Vienna', 'Seoul') or cityName is null)\n" + + "group by countryName, cityName, channel") + .queryContext(DEFAULT_QUERY_CONTEXT) + .expectedResults( + ResultMatchMode.RELAX_NULLS, + ImmutableList.of( + new Object[]{"Austria", null, "#de.wikipedia", "[\"abc\",\"#de.wikipedia\"]"}, + new Object[]{"Republic of Korea", null, "#en.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#en.wikipedia\",\"abc\",\"#ja.wikipedia\",\"abc\",\"#ko.wikipedia\"]"}, + new Object[]{"Republic of Korea", null, "#ja.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#en.wikipedia\",\"abc\",\"#ja.wikipedia\",\"abc\",\"#ko.wikipedia\"]"}, + new Object[]{"Republic of Korea", null, "#ko.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#en.wikipedia\",\"abc\",\"#ja.wikipedia\",\"abc\",\"#ko.wikipedia\"]"}, + new Object[]{"Republic of Korea", "Seoul", "#ko.wikipedia", "[\"abc\",\"#ko.wikipedia\"]"}, + new Object[]{"Austria", "Vienna", "#de.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#es.wikipedia\",\"abc\",\"#tr.wikipedia\"]"}, + new Object[]{"Austria", "Vienna", "#es.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#es.wikipedia\",\"abc\",\"#tr.wikipedia\"]"}, + new Object[]{"Austria", "Vienna", "#tr.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#es.wikipedia\",\"abc\",\"#tr.wikipedia\"]"} + ) + ) .run(); } @@ -337,9 +296,13 @@ enum TestType 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 cb7bed7e0416..4a2f09450871 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 void test_last_val_lastValFn_39() 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 void test_nestedAggs_emtyOvrCls_13() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) + @NotYetSupported(Modes.DISTINCT_AGGREGATE_NOT_SUPPORTED) @DrillTest("nestedAggs/emtyOvrCls_8") @Test public void test_nestedAggs_emtyOvrCls_8() @@ -7533,4 +7533,242 @@ public void test_nestedAggs_multiWin_8() { windowQueryTest(); } + + /* + Druid query tests + */ + + @DrillTest("druid_queries/same_window_across_columns/wikipedia_query_1") + @Test + public void test_same_window_wikipedia_query_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/same_window_across_columns/wikipedia_query_1_named_window") + @Test + public void test_same_window_wikipedia_query_1_named_window() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/multiple_windows/wikipedia_query_1") + @Test + public void test_multiple_windows_wikipedia_query_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/multiple_windows/wikipedia_query_1_named_windows") + @Test + public void test_multiple_windows_wikipedia_query_1_named_windows() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/shuffle_columns/wikipedia_query_1") + @Test + public void test_shuffle_columns_wikipedia_query_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1") + @Test + public void test_shuffle_columns_wikipedia_query_1_shuffle_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/shuffle_columns/wikipedia_query_2") + @Test + public void test_shuffle_columns_wikipedia_query_2() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1") + @Test + public void test_shuffle_columns_wikipedia_query_2_shuffle_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/partition_by_multiple_columns/wikipedia_query_1") + @Test + public void test_partition_by_multiple_columns_wikipedia_query_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/partition_by_multiple_columns/wikipedia_query_2") + @Test + public void test_partition_by_multiple_columns_wikipedia_query_2() + { + 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 97556b16fb20..badbbdc0e66f 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 @@ -319,7 +319,7 @@ public void testExplainHttpFn() "\"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," + + "\"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\"}," + 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 5d53593b7ce0..e5442a2bda24 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 @@ 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 @@ enum Modes 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/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq index e1ae27eef0d2..11f1a7eb80ee 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,7 +101,6 @@ 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" 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 92ce8c261726..f14d26cd58a9 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,7 +39,6 @@ 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" 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 7cbc21bd5513..993680ae4c21 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,7 +60,6 @@ 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" @@ -81,7 +80,6 @@ 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" 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 f9c384b6ad2f..aab3d8b50e5a 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,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 99b6b8653fc4..3bce28a44e98 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,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 191120de0f7e..74ef149e685c 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,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 8b2a57ef9c5e..4a5a77ef1bc0 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,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 c327a62751a9..e9e396fcf9d4 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,7 +79,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -105,7 +104,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 404071a7b2a4..d6f26f85c927 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,7 +80,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -107,7 +106,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 614328ce40aa..6cfc2e1f12e0 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,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 f100ba33fd51..fc38514a401e 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,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 03d22e1c0d5a..5ff1078fc87b 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,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 ad90e641a99b..d6d1bd8bd601 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,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 bfb82dce17d6..d3400229b731 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,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 a2866beb2ed7..e95642a5a235 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,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 da25c7a65241..d907fb0f3eac 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,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 20124df76e2c..6e02805ed7ca 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,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "__time", "v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" @@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" 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 4581d237a812..19e256b68f88 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 588444b911a6..cc8a2f42832f 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 98f34424e83a..16f70e47b8ee 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 5d6e4aa29414..d479bc1cc150 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 85dffdbac3d7..d595c7f9a5bf 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,7 +57,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -78,7 +77,6 @@ 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" 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 28cc7fc8f792..54e573201090 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,7 +57,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -78,7 +77,6 @@ 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" 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 b787c9f6b2ca..e5ead14de40e 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 3ca909dbbe7a..ff43508cb0f5 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 23363f31d947..8e8e12dda6e6 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 20f06ae17de9..83ad89882c7f 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 7ec2378fe2a2..bd90718c84f2 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 fca3275f788c..d817f0eaad29 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 a17fe3fbce47..8e7c0c15b167 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 94cf34202276..ea3d3cdc18f9 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,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "m1", "v0" ], - "legacy" : false, "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" @@ -81,7 +80,6 @@ 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" 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 e5bd0124fcf3..94510babc6d9 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,7 +63,6 @@ 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" @@ -84,7 +83,6 @@ 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" 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 659a20e45337..03a371936a8c 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,7 +63,6 @@ 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" @@ -84,7 +83,6 @@ 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" 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 44f2c01da3cc..a0eee78264f2 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,7 +60,6 @@ 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" @@ -81,7 +80,6 @@ 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" 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 cb70cfd6786c..fa647b06367f 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,7 +63,6 @@ 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" @@ -84,7 +83,6 @@ 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" 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 c6a24964fbf3..ca5fbac7643d 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,7 +63,6 @@ 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" @@ -84,7 +83,6 @@ 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" 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 238d9fcddb9b..fc69d073ebfa 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,7 +63,6 @@ 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" @@ -84,7 +83,6 @@ 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" 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 f5a8444cd952..6b14408362ab 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,7 +63,6 @@ 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" @@ -84,7 +83,6 @@ 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" 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 1c97bb7ee961..30dd4d9e21cb 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 d3cf5105a913..58c79d95df54 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 bd8b49ead4e9..df3eec1a32a9 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 3fdf39476ee4..8c7a76b721af 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 2f80224ec77c..af233e77cc25 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,7 +72,6 @@ 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" @@ -98,7 +97,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -121,7 +119,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 5dafdf1cb581..2ebaee2e4994 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,7 +73,6 @@ 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" @@ -100,7 +99,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -123,7 +121,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 94526e1ddb29..ac4309e9a58f 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 dd795fb1c510..258c9dd8427e 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 9db67c9d7331..dd9bedccdf3d 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 e60075a65746..4e4a97596c8f 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 b07e13731bba..f32757809fc6 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 cb5807a26930..2d27fe88b449 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 6ebe59515c8a..526f45cea0a5 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 935f19e21a60..fa3b2f99623c 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 a644d80a90d6..ee44745b7cfc 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 d77f13502a37..ddc0d96e7e62 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 f905ad0706e6..7ae5addaf83f 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 64d4eddf4885..6f76974ff437 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 b7c64a788479..3b03d8444587 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,7 +72,6 @@ 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" @@ -98,7 +97,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -121,7 +119,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 11994dcdafb8..4f7c30602d5a 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,7 +73,6 @@ 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" @@ -100,7 +99,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -123,7 +121,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 0410dbbc2dd3..6d5e53608862 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 81b0a62d042d..169a3c25edb8 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 5335021cba1e..3ac3494003e0 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 8be64f51fc34..a1b6329e34c2 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 7c7b15456377..c249c4927def 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 2750b09b2ad1..0d647e31f09f 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 8bf56ebc4761..1a4a44d61c4b 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 8190f45efafd..bbdacff71d32 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 fbff0879b5cd..d0e9639d66f8 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,7 +104,6 @@ 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" 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 3409cd00e568..9704809f8f17 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,7 +104,6 @@ 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" 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 0c4f8abd6f92..08b61c0a5a42 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,7 +101,6 @@ 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" 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 3d61195af638..46eeb29104c5 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,7 +104,6 @@ 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" 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 30b667f33cb4..997434994c3f 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,7 +104,6 @@ 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" 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 279e3a590c0a..6311f5644408 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,7 +104,6 @@ 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" 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 52c6bc9980ae..9eeea292be79 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,7 +104,6 @@ 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" 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 c8d92aff808a..d650be35d63b 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,7 +70,6 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -87,7 +86,6 @@ 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" 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 95cdbe258584..6d28747396b3 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,7 +70,6 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -87,7 +86,6 @@ 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" 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 3fb227e9c34e..282084738197 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,7 +67,6 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -84,7 +83,6 @@ 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" 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 94433129db0f..d0bd1b3f545b 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,7 +70,6 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -87,7 +86,6 @@ 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" 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 87be9556a359..bfc74f5a1888 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,7 +70,6 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -87,7 +86,6 @@ 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" 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 207003ad6f36..69fbe42c908d 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,7 +70,6 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -87,7 +86,6 @@ 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" 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 c2fc24a63075..00329a655e0a 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,7 +70,6 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "k", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -87,7 +86,6 @@ 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" 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 c395fdf3ba99..8c8353e582bd 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,7 +70,6 @@ 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" 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 4c8cbccbacfd..f2fa3adc9933 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,7 +70,6 @@ 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" 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 5478c6b9381a..10c2b232d065 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,7 +70,6 @@ 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" 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 5e815f2c3cbf..4287ae123461 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,7 +70,6 @@ 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" 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 7167a4ce8db2..ec22a1c73df7 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,7 +70,6 @@ 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" 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 073386da08c3..381f1ff4c5c4 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,7 +70,6 @@ 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" 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 babde45b3cea..ef493a09cf04 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,7 +59,6 @@ DruidAggregate(group=[{}], EXPR$0=[COUNT() FILTER $0], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "LONG" ], "granularity" : { "type" : "all" 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 7b233392a0e1..85fd47af3285 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,7 +67,6 @@ DruidAggregate(group=[{}], EXPR$0=[COUNT() FILTER $0], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "LONG" ], "granularity" : { "type" : "all" 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 d4dfbd0ad1d2..2fd819422471 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 bcefbd4c4ebd..38c72482fb6f 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 daa8e4e768f8..c8d4c066bb75 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 9645a60b0073..a482f98c76d5 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 33c10207f8f1..94c655fec20d 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,7 +72,6 @@ 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" @@ -98,7 +97,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -121,7 +119,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 d9d19e39d772..ccbcfff78f3b 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,7 +73,6 @@ 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" @@ -100,7 +99,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -123,7 +121,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 5b7547275fa2..d46a06eba337 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 a15a61f877c6..8242b6ae8510 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 12362cdaaaab..1bd8d6341b01 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 91b4312973ed..f774ff7fec6e 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 e7dc799ba595..aa6268bf9d32 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 b1d869fc0083..b1cc848462ed 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 9d917f9bc973..bc4a7a0a9255 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 8df872ddb8a0..e61c0c0fd254 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 18d5b2c281ec..7860b81b5fff 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,7 +75,6 @@ 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" @@ -107,7 +106,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -130,7 +128,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 223cf55e124a..e7d334a00c1c 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,7 +76,6 @@ 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" @@ -109,7 +108,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -132,7 +130,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 38c060edc25b..990e3516e882 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,7 +75,6 @@ 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" @@ -107,7 +106,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -130,7 +128,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 399581bceb52..ef3fdd2fb7ec 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,7 +76,6 @@ 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" @@ -109,7 +108,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -132,7 +130,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 e151ac97f52c..290528701d04 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,7 +72,6 @@ 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" @@ -104,7 +103,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -127,7 +125,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 6fb8344b350a..ba4342c92d28 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,7 +73,6 @@ 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" @@ -106,7 +105,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -129,7 +127,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 0005b6f21996..1e68ddc963c5 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,7 +75,6 @@ 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" @@ -107,7 +106,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -130,7 +128,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 b58ccfb74421..a3326834777e 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,7 +76,6 @@ 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" @@ -109,7 +108,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -132,7 +130,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 1b66e080ef2f..221c16c161f1 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,7 +75,6 @@ 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" @@ -107,7 +106,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -130,7 +128,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 466504c9ee49..3ec24f4656f7 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,7 +76,6 @@ 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" @@ -109,7 +108,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -132,7 +130,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 dfc9e307dcaf..625caf2321b6 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,7 +75,6 @@ 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" @@ -107,7 +106,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -130,7 +128,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 a9678e70060f..1b8c22da4eda 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,7 +76,6 @@ 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" @@ -109,7 +108,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -132,7 +130,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 583c592d3960..54a4b2e286ee 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,7 +75,6 @@ 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" @@ -107,7 +106,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -130,7 +128,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 689bf3b8562f..56c560bffc46 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,7 +76,6 @@ 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" @@ -109,7 +108,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -132,7 +130,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 bbde46874f6e..2a0418aa9ded 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 27682661b4c6..052684a77619 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 28a80ff003c3..50bdfbff93b2 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 c27a4b7a1200..28786182e82a 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 697888623332..0b7305d3c782 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,7 +72,6 @@ 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" @@ -98,7 +97,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -121,7 +119,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 a4342bed6590..f452d2f32919 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,7 +73,6 @@ 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" @@ -100,7 +99,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -123,7 +121,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 a18be661b59c..74950e46f4f5 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 6f2c0cfa4764..daa23575474a 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 bad3fea5e407..e5e92502c1f0 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 c4ee8dcac084..8a74da0eaa51 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 a69dfe749f68..951b7c878473 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 69f6925ce7e1..64682aa187b4 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 c6de594d4ba9..1779bb4f12df 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,7 +75,6 @@ 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" @@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "value" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 63131da8ae3e..cb0c4b5aa323 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,7 +76,6 @@ 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" @@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "matchValue" : "10.1" }, "columns" : [ "dim1" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "__time", "_v0" ], - "legacy" : false, "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" 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 f69e5c22b5ea..391d1b758ded 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,7 +91,6 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 0fcfdbe725d7..c092b8b1699f 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,7 +91,6 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 b2f470dcb748..2f95ad4db95a 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,7 +88,6 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 e0b664541d4c..e1746ea9f7da 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,7 +91,6 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 f9e5916b7991..ee1e57ad0e1e 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,7 +91,6 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 07aac4aeacad..3b6855b65783 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,7 +91,6 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 2b975eab86a6..e72fd71ea960 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,7 +91,6 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) }, "resultFormat" : "compactedList", "columns" : [ "dim1", "j0.d0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 bc0fcae56d4d..4e684853916b 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 915925987a1d..df068fe78fa0 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 612567cb0a38..d180546744e5 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 b08fd8766fd5..064b58eaf540 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 4f3ddcb829f2..b022186b3aba 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,7 +62,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 e158564b8015..1cc0aa9a19d1 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,7 +62,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 021473e3f44c..870ff8404926 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 ac02ef44b472..d3db0f7d7a3e 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 5d2e94ea90f6..18b98ac0b4e5 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 44563315f01d..11cd61bc8588 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 c771a5e6a9f0..64189b47997e 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 4e31bf0d9857..a26f7d991315 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 e8201c2800a4..603c31aec412 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 62bfdc1a0c53..8a87da90dd7a 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,7 +65,6 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim2", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 389fb0eeaeff..5fc2e7fb7963 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,7 +67,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -100,7 +99,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -117,7 +115,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 01e268234fc3..bdbb0d2e62fe 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,7 +67,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -100,7 +99,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -117,7 +115,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 ae58b2deb851..dcceabcd2b19 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,7 +64,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -97,7 +96,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -114,7 +112,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 ec901fab83b2..b210ffd96ac5 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,7 +67,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -100,7 +99,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -117,7 +115,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 be025d6afb7d..66357bec5f25 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,7 +67,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -100,7 +99,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -117,7 +115,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 95250d0012d2..ef5ad5c6a333 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,7 +67,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -100,7 +99,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -117,7 +115,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 6028287a7be9..3f279d8f893c 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,7 +67,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -100,7 +99,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -117,7 +115,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "dim3", "j0.dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 c026e25e85ce..dd97fb01134f 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,7 +75,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -114,7 +113,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -147,7 +145,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -176,7 +173,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -193,7 +189,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 2e13a83a2a85..3a4d7482e4c1 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,7 +75,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -114,7 +113,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -147,7 +145,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -176,7 +173,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -193,7 +189,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 fe86f2126f5a..e42214277c5d 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,7 +72,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -111,7 +110,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -144,7 +142,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -173,7 +170,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -190,7 +186,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 40462afd1d02..f1fd8213bafa 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,7 +75,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -114,7 +113,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -147,7 +145,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -176,7 +173,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -193,7 +189,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 8a33035d0402..438e9166fa01 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,7 +75,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -114,7 +113,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -147,7 +145,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -176,7 +173,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -193,7 +189,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 d07362349463..5b19876eb77f 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,7 +75,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -114,7 +113,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -147,7 +145,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -176,7 +173,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -193,7 +189,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" 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 a844b048a2ec..5ac8bae8bfb0 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,7 +75,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -114,7 +113,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "dim3", "v0" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -147,7 +145,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "v0" ], - "legacy" : false, "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" @@ -176,7 +173,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) } ], "resultFormat" : "compactedList", "columns" : [ "_v0", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" @@ -193,7 +189,6 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) }, "resultFormat" : "compactedList", "columns" : [ "_j0.dim3", "dim3" ], - "legacy" : false, "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByWithLiteralInSubqueryGrouping.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByWithLiteralInSubqueryGrouping.iq index b5d2c682643b..85c1b392c49a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByWithLiteralInSubqueryGrouping.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByWithLiteralInSubqueryGrouping.iq @@ -30,8 +30,8 @@ SELECT +-------+----+ | t1 | t2 | +-------+----+ -| dummy | | | dummy | b | +| dummy | | +-------+----+ (2 rows) 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 e7322a876fc9..2ed1880fef86 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,7 +222,6 @@ 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" 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 098415f6b1b1..f0b4a554df67 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,7 +233,6 @@ 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" 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 5368d23981f9..22304fccf7b7 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,7 +139,6 @@ 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" 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 86e22e97353c..824ea018611d 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,7 +150,6 @@ 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" diff --git a/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest b/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest index 0c9d88b5041f..a2f82ff2905d 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 16dbe924fdb3..e65e27e8794c 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 000000000000..af6b0451761c --- /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 d5a324c9e2d8..aa0a4a2a0198 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 000000000000..fd79234a5624 --- /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 7c9dae4aad39..2b6f7f7fddbb 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 7a579c3fc331..4d78b197e312 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/simpleSum.sqlTest b/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest index 9ca9f88e850b..84bd5ca71af0 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 0a86a691e26a..9b5aa6e1365e 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 cc59868482a2..b4ef8006ea9d 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 9368f00e9b49..ebcc060eaa52 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 c25f1ff03528..87873d44c485 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 1e75e69b97bc..3843519aa790 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 d310f6a8f1ce..4939057621e5 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 000000000000..5672a5a17f35 --- /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 c96b979c0dac..7c7fd03c3c81 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 000000000000..b1913bc200c2 --- /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 000000000000..b490523a0dca --- /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 000000000000..dc9f77f71e08 --- /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 000000000000..6dc676e3385a --- /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 000000000000..8cd625edad22 --- /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 000000000000..041a4ef39634 --- /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 000000000000..ad0dead8add6 --- /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 000000000000..b62aaabc7a21 --- /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 000000000000..722700444fc8 --- /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 000000000000..47452f7c97f3 --- /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 000000000000..970b68a4d5cb --- /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 000000000000..ac530688e53c --- /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/multiple_windows/wikipedia_query_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1.e new file mode 100644 index 000000000000..3625be892e2d --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1.e @@ -0,0 +1,13 @@ +null Austria 1 1 +null Republic of Korea 1 2 +null Republic of Korea 2 3 +null Republic of Korea 3 4 +Horsching Austria 2 1 +Jeonju Republic of Korea 4 1 +Seongnam-si Republic of Korea 5 1 +Seoul Republic of Korea 6 1 +Suwon-si Republic of Korea 7 1 +Vienna Austria 3 1 +Vienna Austria 4 2 +Vienna Austria 5 3 +Yongsan-dong Republic of Korea 8 1 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1.q new file mode 100644 index 000000000000..d61a33e401f4 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1.q @@ -0,0 +1,6 @@ +select cityName, countryName, +row_number() over (partition by countryName order by countryName, cityName, channel) as c1, +count(channel) over (partition by cityName order by countryName, cityName, 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/multiple_windows/wikipedia_query_1_named_windows.e b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1_named_windows.e new file mode 100644 index 000000000000..3625be892e2d --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1_named_windows.e @@ -0,0 +1,13 @@ +null Austria 1 1 +null Republic of Korea 1 2 +null Republic of Korea 2 3 +null Republic of Korea 3 4 +Horsching Austria 2 1 +Jeonju Republic of Korea 4 1 +Seongnam-si Republic of Korea 5 1 +Seoul Republic of Korea 6 1 +Suwon-si Republic of Korea 7 1 +Vienna Austria 3 1 +Vienna Austria 4 2 +Vienna Austria 5 3 +Yongsan-dong Republic of Korea 8 1 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1_named_windows.q b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1_named_windows.q new file mode 100644 index 000000000000..12739d58ceb3 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/multiple_windows/wikipedia_query_1_named_windows.q @@ -0,0 +1,9 @@ +select cityName, countryName, +row_number() over w1 as c1, +count(channel) over w2 as c2 +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel +WINDOW + w1 AS (partition by countryName order by countryName, cityName, channel), + w2 AS (partition by cityName order 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 000000000000..6a827d2285aa --- /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 000000000000..16819458691a --- /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 000000000000..fd2bc5ed2aa9 --- /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 000000000000..251806120540 --- /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 000000000000..e4005d233de0 --- /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 000000000000..39d97a83de2d --- /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 000000000000..be82d150428e --- /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 000000000000..c325e6c6c7ff --- /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 000000000000..e0e8a666879d --- /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 000000000000..e7450711256c --- /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 000000000000..790533db1477 --- /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 000000000000..3287be32cf54 --- /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 000000000000..1d8ef5cdc535 --- /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 000000000000..8824c3e1760f --- /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 000000000000..151090f3be24 --- /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 000000000000..886bc0ac11f7 --- /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 000000000000..52148287c423 --- /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 000000000000..f0a07b6666a8 --- /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 000000000000..6710f17c2a99 --- /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 000000000000..e2608974f8d3 --- /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 000000000000..bfe949f40f55 --- /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 000000000000..723e0ea29636 --- /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 000000000000..3b1046264ef3 --- /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 000000000000..10b81e433b1f --- /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 000000000000..244392e74b18 --- /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 000000000000..deeb5113026d --- /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 000000000000..6ff49b69fb6b --- /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 000000000000..edce01210da1 --- /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 000000000000..26c251a35fb5 --- /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 000000000000..b10b52af389f --- /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 000000000000..a1b116035c18 --- /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 000000000000..99245d7f9530 --- /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 000000000000..ebd91f9f8933 --- /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 000000000000..9241f2ee94ef --- /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/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.e new file mode 100644 index 000000000000..36812a418aeb --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.e @@ -0,0 +1,15 @@ +Austria null 94 7 +Austria null 4685 7 +Austria null 14 7 +Austria null 0 7 +Austria null 272 7 +Austria null 0 7 +Austria null 6979 7 +Guatemala null 0 1 +Guatemala El Salvador 1 1 +Guatemala Guatemala City 173 1 +Austria Horsching 0 1 +Austria Vienna 93 4 +Austria Vienna 72 4 +Austria Vienna 0 4 +Austria Vienna 0 4 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.q new file mode 100644 index 000000000000..5d0dd0756784 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_1.q @@ -0,0 +1,7 @@ +SELECT +countryName, +cityName, +added, +count(added) OVER (PARTITION BY countryName, cityName) +FROM "wikipedia" +where countryName in ('Guatemala', 'Austria') diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.e new file mode 100644 index 000000000000..a1b94f5a865d --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.e @@ -0,0 +1,15 @@ +Austria null 0 7 12044 1 +Austria null 0 7 12044 2 +Austria null 14 7 12044 1 +Austria null 94 7 12044 1 +Austria null 272 7 12044 1 +Austria null 4685 7 12044 1 +Austria null 6979 7 12044 1 +Guatemala null 0 1 0 1 +Guatemala El Salvador 1 1 1 1 +Guatemala Guatemala City 173 1 173 1 +Austria Horsching 0 1 0 1 +Austria Vienna 0 4 165 1 +Austria Vienna 0 4 165 2 +Austria Vienna 72 4 165 1 +Austria Vienna 93 4 165 1 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.q new file mode 100644 index 000000000000..b1a594beedaf --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_multiple_columns/wikipedia_query_2.q @@ -0,0 +1,9 @@ +SELECT +countryName, +cityName, +added, +count(added) OVER (PARTITION BY countryName, cityName), +sum(added) OVER (PARTITION BY countryName, cityName), +ROW_NUMBER() OVER (PARTITION BY countryName, cityName, added) +FROM "wikipedia" +where countryName in ('Guatemala', 'Austria') diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.e new file mode 100644 index 000000000000..0dfb6a832b8a --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.e @@ -0,0 +1,15 @@ +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Guatemala 167 7 174 +Guatemala 167 7 174 +Guatemala 167 7 174 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.q new file mode 100644 index 000000000000..dcb83c09c231 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1.q @@ -0,0 +1,6 @@ +SELECT countryName, +sum("deleted") OVER (PARTITION BY countryName) as count_c3, +sum(delta) OVER (PARTITION BY countryName) as count_c1, +sum(added) OVER (PARTITION BY countryName) as count_c2 +FROM "wikipedia" +where countryName in ('Guatemala', 'Austria') diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.e b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.e new file mode 100644 index 000000000000..0dfb6a832b8a --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.e @@ -0,0 +1,15 @@ +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Austria 162 12047 12209 +Guatemala 167 7 174 +Guatemala 167 7 174 +Guatemala 167 7 174 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.q b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.q new file mode 100644 index 000000000000..adb9287d3788 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/same_window_across_columns/wikipedia_query_1_named_window.q @@ -0,0 +1,7 @@ +SELECT countryName, +sum("deleted") OVER w as count_c3, +sum(delta) OVER w as count_c1, +sum(added) OVER w as count_c2 +FROM "wikipedia" +where countryName in ('Guatemala', 'Austria') +WINDOW w AS (PARTITION BY countryName) diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.e new file mode 100644 index 000000000000..e934bc8fc276 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.e @@ -0,0 +1,15 @@ +Austria 1017.4166666666666 +Austria 1017.4166666666666 +Austria 1017.4166666666666 +Austria 1017.4166666666666 +Austria 1017.4166666666666 +Austria 1017.4166666666666 +Austria 1017.4166666666666 +Austria 1017.4166666666666 +Austria 1017.4166666666666 +Austria 1017.4166666666666 +Austria 1017.4166666666666 +Austria 1017.4166666666666 +Guatemala 58 +Guatemala 58 +Guatemala 58 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.q new file mode 100644 index 000000000000..f1a7bcb09b17 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1.q @@ -0,0 +1,5 @@ +SELECT +countryName, +AVG(added) OVER(PARTITION BY countryName) +FROM wikipedia +where countryName in ('Guatemala', 'Austria') diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.e new file mode 100644 index 000000000000..e74706be0098 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.e @@ -0,0 +1,15 @@ +1017.4166666666666 Austria +1017.4166666666666 Austria +1017.4166666666666 Austria +1017.4166666666666 Austria +1017.4166666666666 Austria +1017.4166666666666 Austria +1017.4166666666666 Austria +1017.4166666666666 Austria +1017.4166666666666 Austria +1017.4166666666666 Austria +1017.4166666666666 Austria +1017.4166666666666 Austria +58 Guatemala +58 Guatemala +58 Guatemala diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.q new file mode 100644 index 000000000000..c2dc11546a94 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_1_shuffle_1.q @@ -0,0 +1,5 @@ +SELECT +AVG(added) OVER(PARTITION BY countryName), +countryName +FROM wikipedia +where countryName in ('Guatemala', 'Austria') diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.e new file mode 100644 index 000000000000..daf6eff61ba1 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.e @@ -0,0 +1,16 @@ +Austria null 1 #de.wikipedia 1 +Guatemala null 1 #es.wikipedia 2 +Republic of Korea null 1 #en.wikipedia 3 +Republic of Korea null 2 #ja.wikipedia 4 +Republic of Korea null 3 #ko.wikipedia 5 +Guatemala El Salvador 2 #es.wikipedia 1 +Guatemala Guatemala City 3 #es.wikipedia 1 +Austria Horsching 2 #de.wikipedia 1 +Republic of Korea Jeonju 4 #ko.wikipedia 1 +Republic of Korea Seongnam-si 5 #ko.wikipedia 1 +Republic of Korea Seoul 6 #ko.wikipedia 1 +Republic of Korea Suwon-si 7 #ko.wikipedia 1 +Austria Vienna 3 #de.wikipedia 1 +Austria Vienna 4 #es.wikipedia 2 +Austria Vienna 5 #tr.wikipedia 3 +Republic of Korea Yongsan-dong 8 #ko.wikipedia 1 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.q new file mode 100644 index 000000000000..d3ea2dfc729a --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2.q @@ -0,0 +1,9 @@ +SELECT +countryName, +cityName, +ROW_NUMBER() OVER(PARTITION BY countryName), +channel, +COUNT(channel) over (PARTITION BY cityName order by countryName, cityName, channel) +FROM wikipedia +where countryName in ('Guatemala', 'Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.e new file mode 100644 index 000000000000..813ccdbf6aaf --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.e @@ -0,0 +1,16 @@ +1 Austria null 1 #de.wikipedia +1 Guatemala null 2 #es.wikipedia +1 Republic of Korea null 3 #en.wikipedia +2 Republic of Korea null 4 #ja.wikipedia +3 Republic of Korea null 5 #ko.wikipedia +2 Guatemala El Salvador 1 #es.wikipedia +3 Guatemala Guatemala City 1 #es.wikipedia +2 Austria Horsching 1 #de.wikipedia +4 Republic of Korea Jeonju 1 #ko.wikipedia +5 Republic of Korea Seongnam-si 1 #ko.wikipedia +6 Republic of Korea Seoul 1 #ko.wikipedia +7 Republic of Korea Suwon-si 1 #ko.wikipedia +3 Austria Vienna 1 #de.wikipedia +4 Austria Vienna 2 #es.wikipedia +5 Austria Vienna 3 #tr.wikipedia +8 Republic of Korea Yongsan-dong 1 #ko.wikipedia diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.q new file mode 100644 index 000000000000..779aaf3a86f3 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/shuffle_columns/wikipedia_query_2_shuffle_1.q @@ -0,0 +1,9 @@ +SELECT +ROW_NUMBER() OVER(PARTITION BY countryName), +countryName, +cityName, +COUNT(channel) over (PARTITION BY cityName order by countryName, cityName, channel), +channel +FROM wikipedia +where countryName in ('Guatemala', '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 50a5762c15fc..f3f7c1a434d5 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 4462641cb9a1..102ef2d75bfa 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 2945955d9d66..412f728d56d6 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==", - "dependencies": { - "@blueprintjs/colors": "^4.2.1", - "@blueprintjs/icons": "^4.16.0", - "@juggle/resize-observer": "^3.4.0", - "@types/dom4": "^2.0.2", + "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": "^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==", - "dependencies": { - "@blueprintjs/core": "^4.20.1", - "@blueprintjs/datetime": "^4.4.36", - "@blueprintjs/popover2": "^1.14.9", - "@blueprintjs/select": "^4.9.22", + "version": "2.3.7", + "resolved": "https://registry.npmjs.org/@blueprintjs/datetime2/-/datetime2-2.3.7.tgz", + "integrity": "sha512-Q6IBo5hcvoDtH9FsFaysZhadXNS6iyhuyaRY+rJD3y3qLaUt6wiYDhw5LN6L4wuasbPO1g0lVDwY6bkGuIG4jg==", + "dependencies": { + "@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/icons": { - "version": "4.16.0", - "resolved": "https://registry.npmjs.org/@blueprintjs/icons/-/icons-4.16.0.tgz", - "integrity": "sha512-cyfgjUZcZCtQrXWUV8FwqYTFEzduV4a0N7yhOU38jY+cBRCLu/sDrD0Osvfk4DGRvNe4YjY7pohVLFSxpg68Uw==", - "dependencies": { - "change-case": "^4.1.2", - "classnames": "^2.3.1", - "tslib": "~2.5.0" + "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/popover2": { - "version": "1.14.11", - "resolved": "https://registry.npmjs.org/@blueprintjs/popover2/-/popover2-1.14.11.tgz", - "integrity": "sha512-5XAjeb2mlWjYXC0pqrNDLzHSsX85Zaiv8jixxUN9abarMUUFKGATgGF8MRsWTLAW94Gli6CB1lzVkrYkRHHf6Q==", + "node_modules/@blueprintjs/icons": { + "version": "5.10.0", + "resolved": "https://registry.npmjs.org/@blueprintjs/icons/-/icons-5.10.0.tgz", + "integrity": "sha512-Qp9H2U2/m6+p0Lh9ACuF2O6o0uVINMFimo+EgasgGaLx8SkJsl7ApB3hYQ+iRSYlQWotnoqd4Vtzj4c7sKukHQ==", "dependencies": { - "@blueprintjs/core": "^4.20.2", - "@juggle/resize-observer": "^3.4.0", - "@popperjs/core": "^2.11.7", + "change-case": "^4.1.2", "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==", - "requires": { - "@blueprintjs/colors": "^4.2.1", - "@blueprintjs/icons": "^4.16.0", - "@juggle/resize-observer": "^3.4.0", - "@types/dom4": "^2.0.2", + "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": "^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==", - "requires": { - "@blueprintjs/core": "^4.20.1", - "@blueprintjs/datetime": "^4.4.36", - "@blueprintjs/popover2": "^1.14.9", - "@blueprintjs/select": "^4.9.22", + "version": "2.3.7", + "resolved": "https://registry.npmjs.org/@blueprintjs/datetime2/-/datetime2-2.3.7.tgz", + "integrity": "sha512-Q6IBo5hcvoDtH9FsFaysZhadXNS6iyhuyaRY+rJD3y3qLaUt6wiYDhw5LN6L4wuasbPO1g0lVDwY6bkGuIG4jg==", + "requires": { + "@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 6e38ba24684a..0c9370f88083 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 1dc820b94c1b..90f1420282ae 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 c345d815b80d..1f5c774dfced 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 39df4d75f2ab..8382d04a1986 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 267ac4a27260..0d4c3928a27d 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 ca10be55628d..a32862302175 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 1c3695693d0e..d3fa168cbc3d 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 a5bdd46adf17..4af64af7f53f 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`] = ` >