diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java index 5bc67bc9f729..8d79a611d0e6 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -39,9 +39,11 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; @@ -192,11 +194,11 @@ public void timeFloorUsingCursor(Blackhole blackhole) try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final CursorGranularizer granularizer = CursorGranularizer.create( - adapter, cursor, + QueryableIndexTimeBoundaryInspector.create(index), + Cursors.getTimeOrdering(index.getOrdering()), Granularities.HOUR, - adapter.getInterval(), - false + adapter.getInterval() ); final Sequence results = Sequences.simple(granularizer.getBucketIterable()) diff --git a/docs/ingestion/ingestion-spec.md b/docs/ingestion/ingestion-spec.md index a1a0ff1ee5b7..37f0bf0a1fd5 100644 --- a/docs/ingestion/ingestion-spec.md +++ b/docs/ingestion/ingestion-spec.md @@ -232,7 +232,7 @@ A `dimensionsSpec` can have the following components: | `spatialDimensions` | An array of [spatial dimensions](../querying/geo.md). | `[]` | | `includeAllDimensions` | Note that this field only applies to string-based schema discovery where Druid ingests dimensions it discovers as strings. This is different from schema auto-discovery where Druid infers the type for data. You can set `includeAllDimensions` to true to ingest both explicit dimensions in the `dimensions` field and other dimensions that the ingestion task discovers from input data. In this case, the explicit dimensions will appear first in the order that you specify them, and the dimensions dynamically discovered will come after. This flag can be useful especially with auto schema discovery using [`flattenSpec`](./data-formats.md#flattenspec). If this is not set and the `dimensions` field is not empty, Druid will ingest only explicit dimensions. If this is not set and the `dimensions` field is empty, all discovered dimensions will be ingested. | false | | `useSchemaDiscovery` | Configure Druid to use schema auto-discovery to discover some or all of the dimensions and types for your data. For any dimensions that aren't a uniform type, Druid ingests them as JSON. You can use this for native batch or streaming ingestion. | false | - +| `forceSegmentSortByTime` | When set to true (the default), segments created by the ingestion job are sorted by `{__time, dimensions[0], dimensions[1], ...}`. When set to false, segments created by the ingestion job are sorted by `{dimensions[0], dimensions[1], ...}`. To include `__time` in the sort order when this parameter is set to `false`, you must include a dimension named `__time` with type `long` explicitly in the `dimensions` list.

Setting this to `false` is an experimental feature; see [Sorting](partitioning.md#sorting) for details. | `true` | #### Dimension objects diff --git a/docs/ingestion/partitioning.md b/docs/ingestion/partitioning.md index 6cf5b0a74d28..d1501f5fe0ff 100644 --- a/docs/ingestion/partitioning.md +++ b/docs/ingestion/partitioning.md @@ -44,33 +44,60 @@ Partitioning by time is important for two reasons: The most common choices to balance these considerations are `hour` and `day`. For streaming ingestion, `hour` is especially common, because it allows compaction to follow ingestion with less of a time delay. +The following table describes how to configure time chunk partitioning. + +|Method|Configuration| +|------|------------| +|[SQL](../multi-stage-query/index.md)|[`PARTITIONED BY`](../multi-stage-query/concepts.md#partitioning)| +|[Kafka](../ingestion/kafka-ingestion.md) or [Kinesis](../ingestion/kinesis-ingestion.md)|`segmentGranularity` inside the [`granularitySpec`](ingestion-spec.md#granularityspec)| +|[Native batch](native-batch.md) or [Hadoop](hadoop.md)|`segmentGranularity` inside the [`granularitySpec`](ingestion-spec.md#granularityspec)| + ## Secondary partitioning -Druid can partition segments within a particular time chunk further depending upon options that vary based on the ingestion type you have chosen. In general, secondary partitioning on a particular dimension improves locality. This means that rows with the same value for that dimension are stored together, decreasing access time. +Druid further partitions each time chunk into immutable segments. Secondary partitioning on a particular dimension improves locality. This means that rows with the same value for that dimension are stored together, decreasing access time. -To achieve the best performance and smallest overall footprint, partition your data on a "natural" -dimension that you often use as a filter when possible. Such partitioning often improves compression and query performance. For example, some cases have yielded threefold storage size decreases. +To achieve the best performance and smallest overall footprint, partition your data on a "natural" dimension that +you often use as a filter, or that achieves some alignment within your data. Such partitioning can improve compression +and query performance by significant multiples. -## Partitioning and sorting +The following table describes how to configure secondary partitioning. -Partitioning and sorting work well together. If you do have a "natural" partitioning dimension, consider placing it first in the `dimensions` list of your `dimensionsSpec`. This way Druid sorts rows within each segment by that column. This sorting configuration frequently improves compression more than using partitioning alone. +|Method|Configuration| +|------|------------| +|[SQL](../multi-stage-query/index.md)|[`CLUSTERED BY`](../multi-stage-query/concepts.md#clustering)| +|[Kafka](../ingestion/kafka-ingestion.md) or [Kinesis](../ingestion/kinesis-ingestion.md)|Upstream partitioning defines how Druid partitions the datasource. You can also alter clustering using [`REPLACE`](../multi-stage-query/concepts.md#replace) (with `CLUSTERED BY`) or [compaction](../data-management/compaction.md) after initial ingestion.| +|[Native batch](native-batch.md) or [Hadoop](hadoop.md)|[`partitionsSpec`](native-batch.md#partitionsspec) inside the `tuningConfig`| -Note that Druid always sorts rows within a segment by timestamp first, even before the first dimension listed in your `dimensionsSpec`. This sorting can preclude the efficacy of dimension sorting. To work around this limitation if necessary, set your `queryGranularity` equal to `segmentGranularity` in your [`granularitySpec`](./ingestion-spec.md#granularityspec). Druid will set all timestamps within the segment to the same value, letting you identify a [secondary timestamp](schema-design.md#secondary-timestamps) as the "real" timestamp. +## Sorting -## How to configure partitioning +Each segment is internally sorted to promote compression and locality. -Not all ingestion methods support an explicit partitioning configuration, and not all have equivalent levels of flexibility. If you are doing initial ingestion through a less-flexible method like -Kafka), you can use [reindexing](../data-management/update.md#reindex) or [compaction](../data-management/compaction.md) to repartition your data after initial ingestion. This is a powerful technique you can use to optimally partition any data older than a certain time threshold while you continuously add new data from a stream. +Partitioning and sorting work well together. If you do have a "natural" partitioning dimension, consider placing it +first in your sort order as well. This way, Druid sorts rows within each segment by that column. This sorting configuration +frequently improves compression and performance more than using partitioning alone. -The following table shows how each ingestion method handles partitioning: +The following table describes how to configure sorting. -|Method|How it works| +|Method|Configuration| |------|------------| -|[Native batch](native-batch.md)|Configured using [`partitionsSpec`](native-batch.md#partitionsspec) inside the `tuningConfig`.| -|[SQL](../multi-stage-query/index.md)|Configured using [`PARTITIONED BY`](../multi-stage-query/concepts.md#partitioning) and [`CLUSTERED BY`](../multi-stage-query/concepts.md#clustering).| -|[Hadoop](hadoop.md)|Configured using [`partitionsSpec`](hadoop.md#partitionsspec) inside the `tuningConfig`.| -|[Kafka indexing service](../ingestion/kafka-ingestion.md)|Kafka topic partitioning defines how Druid partitions the datasource. You can also [reindex](../data-management/update.md#reindex) or [compact](../data-management/compaction.md) to repartition after initial ingestion.| -|[Kinesis indexing service](../ingestion/kinesis-ingestion.md)|Kinesis stream sharding defines how Druid partitions the datasource. You can also [reindex](../data-management/update.md#reindex) or [compact](../data-management/compaction.md) to repartition after initial ingestion.| +|[SQL](../multi-stage-query/index.md)|Uses order of fields in [`CLUSTERED BY`](../multi-stage-query/concepts.md#clustering) or [`segmentSortOrder`](../multi-stage-query/reference.md#context) in the query context| +|[Kafka](../ingestion/kafka-ingestion.md) or [Kinesis](../ingestion/kinesis-ingestion.md)|Uses order of fields in [`dimensionsSpec`](ingestion-spec.md#granularityspec)| +|[Native batch](native-batch.md) or [Hadoop](hadoop.md)|Uses order of fields in [`dimensionsSpec`](ingestion-spec.md#granularityspec)| + +:::info +Druid implicitly sorts rows within a segment by `__time` first before any `dimensions` or `CLUSTERED BY` fields, unless +you set `forceSegmentSortByTime` to `false` in your +[query context](../multi-stage-query/reference.md#context-parameters) (for SQL) or in your +[`dimensionsSpec`](ingestion-spec.md#dimensionsspec) (for other ingestion forms). + +Setting `forceSegmentSortByTime` to `false` is an experimental feature. Segments created with sort orders that +do not start with `__time` can only be read by Druid 31 or later. Additionally, at this time, certain queries are not +supported on such segments, including: + +- Native queries with `granularity` other than `all`. +- Native `scan` query with ascending or descending time order. +- SQL queries that plan into an unsupported native query. +::: ## Learn more diff --git a/docs/multi-stage-query/concepts.md b/docs/multi-stage-query/concepts.md index cae88a0f3750..27cde86e9a50 100644 --- a/docs/multi-stage-query/concepts.md +++ b/docs/multi-stage-query/concepts.md @@ -172,13 +172,13 @@ Clustering is important for two reasons: 1. Lower storage footprint due to improved locality, and therefore improved compressibility. 2. Better query performance due to dimension-based segment pruning, which removes segments from consideration when they - cannot possibly contain data matching a query's filter. This speeds up filters like `x = 'foo'` and `x IN ('foo', - 'bar')`. + cannot possibly contain data matching a query's filter. This speeds up filters like `x = 'foo'` and + `x IN ('foo', 'bar')`. To activate dimension-based pruning, these requirements must be met: - Segments were generated by a `REPLACE` statement, not an `INSERT` statement. -- All `CLUSTERED BY` columns are single-valued string columns. +- `CLUSTERED BY` begins with single-valued string columns. These single-valued string columns are used for pruning. If these requirements are _not_ met, Druid still clusters data during ingestion but will not be able to perform dimension-based segment pruning at query time. You can tell if dimension-based segment pruning is possible by using the @@ -188,6 +188,10 @@ available in the **Segments** view under the **Partitioning** column. For more information about syntax, see [`CLUSTERED BY`](./reference.md#clustered-by). +For more information about the mechanics of clustering, refer to +[Secondary partitioning](../ingestion/partitioning#secondary-partitioning) and +[Sorting](../ingestion/partitioning#sorting). + ### Rollup [Rollup](../ingestion/rollup.md) is a technique that pre-aggregates data during ingestion to reduce the amount of data diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 0f8e710a59f5..c4345fedf81b 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -362,8 +362,13 @@ For more information about partitioning, see [Partitioning](concepts.md#partitio ### `CLUSTERED BY` The `CLUSTERED BY ` clause is optional for [INSERT](#insert) and [REPLACE](#replace). It accepts a list of -column names or expressions. Druid's segment generation only supports ascending order, so an `INSERT` or `REPLACE` query with -`CLUSTERED BY` columns in `DESC` ordering is not allowed. +column names or expressions. + +This column list is used for [secondary partitioning](../ingestion/partitioning.md#secondary-partitioning) of segments +within a time chunk, and [sorting](../ingestion/partitioning.md#sorting) of rows within a segment. For sorting purposes, +Druid implicitly prepends `__time` to the `CLUSTERED BY` column list, unless +[`forceSegmentSortByTime`](#context) is set to `false` +(an experimental feature; see [Sorting](../ingestion/partitioning.md#sorting) for details). For more information about clustering, see [Clustering](concepts.md#clustering). @@ -397,7 +402,8 @@ The following table lists the context parameters for the MSQ task engine: | `arrayIngestMode` | INSERT, REPLACE

Controls how ARRAY type values are stored in Druid segments. When set to `array` (recommended for SQL compliance), Druid will store all ARRAY typed values in [ARRAY typed columns](../querying/arrays.md), and supports storing both VARCHAR and numeric typed arrays. When set to `mvd` (the default, for backwards compatibility), Druid only supports VARCHAR typed arrays, and will store them as [multi-value string columns](../querying/multi-value-dimensions.md). See [`arrayIngestMode`] in the [Arrays](../querying/arrays.md) page for more details. | `mvd` (for backwards compatibility, recommended to use `array` for SQL compliance)| | `sqlJoinAlgorithm` | SELECT, INSERT, REPLACE

Algorithm to use for JOIN. Use `broadcast` (the default) for broadcast hash join or `sortMerge` for sort-merge join. Affects all JOIN operations in the query. This is a hint to the MSQ engine and the actual joins in the query may proceed in a different way than specified. See [Joins](#joins) for more details. | `broadcast` | | `rowsInMemory` | INSERT or REPLACE

Maximum number of rows to store in memory at once before flushing to disk during the segment generation process. Ignored for non-INSERT queries. In most cases, use the default value. You may need to override the default if you run into one of the [known issues](./known-issues.md) around memory usage. | 100,000 | -| `segmentSortOrder` | INSERT or REPLACE

Normally, Druid sorts rows in individual segments using `__time` first, followed by the [CLUSTERED BY](#clustered-by) clause. When you set `segmentSortOrder`, Druid sorts rows in segments using this column list first, followed by the CLUSTERED BY order.

You provide the column list as comma-separated values or as a JSON array in string form. If your query includes `__time`, then this list must begin with `__time`. For example, consider an INSERT query that uses `CLUSTERED BY country` and has `segmentSortOrder` set to `__time,city`. Within each time chunk, Druid assigns rows to segments based on `country`, and then within each of those segments, Druid sorts those rows by `__time` first, then `city`, then `country`. | empty list | +| `segmentSortOrder` | INSERT or REPLACE

Normally, Druid sorts rows in individual segments using `__time` first, followed by the [CLUSTERED BY](#clustered-by) clause. When you set `segmentSortOrder`, Druid uses the order from this context parameter instead. Provide the column list as comma-separated values or as a JSON array in string form.
< br/>For example, consider an INSERT query that uses `CLUSTERED BY country` and has `segmentSortOrder` set to `__time,city,country`. Within each time chunk, Druid assigns rows to segments based on `country`, and then within each of those segments, Druid sorts those rows by `__time` first, then `city`, then `country`. | empty list | +| `forceSegmentSortByTime` | INSERT or REPLACE

When set to `true` (the default), Druid prepends `__time` to [CLUSTERED BY](#clustered-by) when determining the sort order for individual segments. Druid also requires that `segmentSortOrder`, if provided, starts with `__time`.

When set to `false`, Druid uses the [CLUSTERED BY](#clustered-by) alone to determine the sort order for individual segments, and does not require that `segmentSortOrder` begin with `__time`. Setting this parameter to `false` is an experimental feature; see [Sorting](../ingestion/partitioning#sorting) for details. | `true` | | `maxParseExceptions`| SELECT, INSERT, REPLACE

Maximum number of parse exceptions that are ignored while executing the query before it stops with `TooManyWarningsFault`. To ignore all the parse exceptions, set the value to -1. | 0 | | `rowsPerSegment` | INSERT or REPLACE

The number of rows per segment to target. The actual number of rows per segment may be somewhat higher or lower than this number. In most cases, use the default. For general information about sizing rows per segment, see [Segment Size Optimization](../operations/segment-optimization.md). | 3,000,000 | | `indexSpec` | INSERT or REPLACE

An [`indexSpec`](../ingestion/ingestion-spec.md#indexspec) to use when generating segments. May be a JSON string or object. See [Front coding](../ingestion/ingestion-spec.md#front-coding) for details on configuring an `indexSpec` with front coding. | See [`indexSpec`](../ingestion/ingestion-spec.md#indexspec). | diff --git a/docs/querying/sql-query-context.md b/docs/querying/sql-query-context.md index dc73c9e1ab3f..e003ceec69d9 100644 --- a/docs/querying/sql-query-context.md +++ b/docs/querying/sql-query-context.md @@ -50,11 +50,11 @@ Configure Druid SQL query planning using the parameters in the table below. |`sqlReverseLookup`|Whether to consider the [reverse-lookup rewrite](lookups.md#reverse-lookup) of the `LOOKUP` function during SQL planning.

Calls to `LOOKUP` are only reversed when the number of matching keys is lower than both `inSubQueryThreshold` and `sqlReverseLookupThreshold`.|true| |`sqlReverseLookupThreshold`|Maximum size of `IN` filter to create when applying a [reverse-lookup rewrite](lookups.md#reverse-lookup). If a `LOOKUP` call matches more keys than this threshold, it is left as-is.

If `inSubQueryThreshold` is lower than `sqlReverseLookupThreshold`, the `inSubQueryThreshold` is used as the threshold instead.|10000| |`sqlPullUpLookup`|Whether to consider the [pull-up rewrite](lookups.md#pull-up) of the `LOOKUP` function during SQL planning.|true| -|`enableJoinLeftTableScanDirect`|`false`|This flag applies to queries which have joins. For joins, where left child is a simple scan with a filter, by default, druid will run the scan as a query and the join the results to the right child on broker. Setting this flag to true overrides that behavior and druid will attempt to push the join to data servers instead. Please note that the flag could be applicable to queries even if there is no explicit join. since queries can internally translated into a join by the SQL planner.| -|`maxNumericInFilters`|`-1`|Max limit for the amount of numeric values that can be compared for a string type dimension when the entire SQL WHERE clause of a query translates only to an [OR](../querying/filters.md#or) of [Bound filter](../querying/filters.md#bound-filter). By default, Druid does not restrict the amount of of numeric Bound Filters on String columns, although this situation may block other queries from running. Set this parameter to a smaller value to prevent Druid from running queries that have prohibitively long segment processing times. The optimal limit requires some trial and error; we recommend starting with 100. Users who submit a query that exceeds the limit of `maxNumericInFilters` should instead rewrite their queries to use strings in the `WHERE` clause instead of numbers. For example, `WHERE someString IN (‘123’, ‘456’)`. This value cannot exceed the set system configuration `druid.sql.planner.maxNumericInFilters`. This value is ignored if `druid.sql.planner.maxNumericInFilters` is not set explicitly.| -|`inFunctionThreshold`|`100`| At or beyond this threshold number of values, SQL `IN` is converted to [`SCALAR_IN_ARRAY`](sql-functions.md#scalar_in_array). A threshold of 0 forces this conversion in all cases. A threshold of [Integer.MAX_VALUE] disables this conversion. The converted function is eligible for fewer planning-time optimizations, which speeds up planning, but may prevent certain planning-time optimizations.| -|`inFunctionExprThreshold`|`2`| At or beyond this threshold number of values, SQL `IN` is eligible for execution using the native function `scalar_in_array` rather than an || of `==`, even if the number of values is below `inFunctionThreshold`. This property only affects translation of SQL `IN` to a [native expression](math-expr.md). It does not affect translation of SQL `IN` to a [native filter](filters.md). This property is provided for backwards compatibility purposes, and may be removed in a future release.| -|`inSubQueryThreshold`|`2147483647`| At or beyond this threshold number of values, SQL `IN` is converted to `JOIN` on an inline table. `inFunctionThreshold` takes priority over this setting. A threshold of 0 forces usage of an inline table in all cases where the size of a SQL `IN` is larger than `inFunctionThreshold`. A threshold of `2147483647` disables the rewrite of SQL `IN` to `JOIN`. | +|`enableJoinLeftTableScanDirect`|This flag applies to queries which have joins. For joins, where left child is a simple scan with a filter, by default, druid will run the scan as a query and the join the results to the right child on broker. Setting this flag to true overrides that behavior and druid will attempt to push the join to data servers instead. Please note that the flag could be applicable to queries even if there is no explicit join. since queries can internally translated into a join by the SQL planner.|`false`| +|`maxNumericInFilters`|Max limit for the amount of numeric values that can be compared for a string type dimension when the entire SQL WHERE clause of a query translates only to an [OR](../querying/filters.md#or) of [Bound filter](../querying/filters.md#bound-filter). By default, Druid does not restrict the amount of of numeric Bound Filters on String columns, although this situation may block other queries from running. Set this parameter to a smaller value to prevent Druid from running queries that have prohibitively long segment processing times. The optimal limit requires some trial and error; we recommend starting with 100. Users who submit a query that exceeds the limit of `maxNumericInFilters` should instead rewrite their queries to use strings in the `WHERE` clause instead of numbers. For example, `WHERE someString IN (‘123’, ‘456’)`. This value cannot exceed the set system configuration `druid.sql.planner.maxNumericInFilters`. This value is ignored if `druid.sql.planner.maxNumericInFilters` is not set explicitly.|`-1`| +|`inFunctionThreshold`| At or beyond this threshold number of values, SQL `IN` is converted to [`SCALAR_IN_ARRAY`](sql-functions.md#scalar_in_array). A threshold of 0 forces this conversion in all cases. A threshold of [Integer.MAX_VALUE] disables this conversion. The converted function is eligible for fewer planning-time optimizations, which speeds up planning, but may prevent certain planning-time optimizations.| `100`| +|`inFunctionExprThreshold`|At or beyond this threshold number of values, SQL `IN` is eligible for execution using the native function `scalar_in_array` rather than an || of `==`, even if the number of values is below `inFunctionThreshold`. This property only affects translation of SQL `IN` to a [native expression](math-expr.md). It does not affect translation of SQL `IN` to a [native filter](filters.md). This property is provided for backwards compatibility purposes, and may be removed in a future release.|`2`| +|`inSubQueryThreshold`|At or beyond this threshold number of values, SQL `IN` is converted to `JOIN` on an inline table. `inFunctionThreshold` takes priority over this setting. A threshold of 0 forces usage of an inline table in all cases where the size of a SQL `IN` is larger than `inFunctionThreshold`. A threshold of `2147483647` disables the rewrite of SQL `IN` to `JOIN`. |`2147483647`| ## Setting the query context The query context parameters can be specified as a "context" object in the [JSON API](../api-reference/sql-api.md) or as a [JDBC connection properties object](../api-reference/sql-jdbc.md). diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java index c5faefc65ca6..c06b56496815 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java @@ -32,6 +32,7 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryEngine; import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -101,7 +102,12 @@ public void testTimeseriesWithDistinctCountAgg() throws Exception .build(); final Iterable> results = - engine.process(query, new IncrementalIndexStorageAdapter(index), new DefaultTimeseriesQueryMetrics()).toList(); + engine.process( + query, + new IncrementalIndexStorageAdapter(index), + new IncrementalIndexTimeBoundaryInspector(index), + new DefaultTimeseriesQueryMetrics() + ).toList(); List> expectedResults = Collections.singletonList( new Result<>( diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java index c61a793ff837..ff5a0c313634 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java @@ -33,6 +33,7 @@ import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.query.topn.TopNQueryEngine; import org.apache.druid.query.topn.TopNResultValue; +import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -130,7 +131,12 @@ public void testTopNWithDistinctCountAgg() throws Exception .build(); final Iterable> results = - engine.query(query, new IncrementalIndexStorageAdapter(index), null).toList(); + engine.query( + query, + new IncrementalIndexStorageAdapter(index), + new IncrementalIndexTimeBoundaryInspector(index), + null + ).toList(); List> expectedResults = Collections.singletonList( new Result<>( diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index 5be792966a97..ca4642bde2fe 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -97,7 +97,7 @@ public String toString() for (GroupByQueryConfig config : configs) { final GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(config, BUFFER_POOLS); - for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunnersToMerge(factory)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunnersToMerge(factory, false)) { final String testName = StringUtils.format( "config=%s, runner=%s", config.toString(), diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index 410e6bd3c5d8..ae6c634ad0bc 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -61,7 +61,8 @@ public static void teardown() throws IOException @Parameterized.Parameters(name = "{0}") public static Iterable constructorFeeder() { - return QueryRunnerTestHelper.transformToConstructionFeeder(TopNQueryRunnerTest.queryRunners()); + // Don't test with non-time-ordered segments, because results vary too much due to order-dependence. + return QueryRunnerTestHelper.transformToConstructionFeeder(TopNQueryRunnerTest.queryRunners(false)); } private final QueryRunner runner; diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java index 49f91ada0068..983ffc006303 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java @@ -97,7 +97,7 @@ public String toString() for (GroupByQueryConfig config : configs) { final GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(config, BUFFER_POOLS); - for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunnersToMerge(factory)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunnersToMerge(factory, true)) { final String testName = StringUtils.format( "config=%s, runner=%s", config.toString(), diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java index 7ea3dd31e599..cead791167d3 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java @@ -61,7 +61,7 @@ public static void teardown() throws IOException @Parameterized.Parameters(name = "{0}") public static Iterable constructorFeeder() { - return QueryRunnerTestHelper.transformToConstructionFeeder(TopNQueryRunnerTest.queryRunners()); + return QueryRunnerTestHelper.transformToConstructionFeeder(TopNQueryRunnerTest.queryRunners(true)); } private final QueryRunner runner; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 4310619f81f1..875212605b80 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -40,6 +40,7 @@ import org.apache.druid.data.input.StringTuple; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.discovery.BrokerClient; import org.apache.druid.error.DruidException; @@ -1095,24 +1096,20 @@ private List generateSegmentIdsWithShardSpecsForReplace( final RowKeyReader keyReader = clusterBy.keyReader(signature); final SegmentIdWithShardSpec[] retVal = new SegmentIdWithShardSpec[partitionBoundaries.size()]; final Granularity segmentGranularity = destination.getSegmentGranularity(); - final List shardColumns; - final Pair, String> shardReasonPair; - - shardReasonPair = computeShardColumns( + final Pair, String> shardReasonPair = computeShardColumns( signature, clusterBy, querySpec.getColumnMappings(), mayHaveMultiValuedClusterByFields ); + final List shardColumns = shardReasonPair.lhs; + final String commentary = shardReasonPair.rhs; - shardColumns = shardReasonPair.lhs; - String reason = shardReasonPair.rhs; - log.info("ShardSpec chosen: %s", reason); - + log.info("ShardSpec chosen: %s", commentary); if (shardColumns.isEmpty()) { - segmentReport = new MSQSegmentReport(NumberedShardSpec.class.getSimpleName(), reason); + segmentReport = new MSQSegmentReport(NumberedShardSpec.class.getSimpleName(), commentary); } else { - segmentReport = new MSQSegmentReport(DimensionRangeShardSpec.class.getSimpleName(), reason); + segmentReport = new MSQSegmentReport(DimensionRangeShardSpec.class.getSimpleName(), commentary); } // Group partition ranges by bucket (time chunk), so we can generate shardSpecs for each bucket independently. @@ -1156,9 +1153,13 @@ private List generateSegmentIdsWithShardSpecsForReplace( } else { final ClusterByPartition range = ranges.get(segmentNumber).rhs; final StringTuple start = - segmentNumber == 0 ? null : makeStringTuple(clusterBy, keyReader, range.getStart()); + segmentNumber == 0 + ? null + : makeStringTuple(clusterBy, keyReader, range.getStart(), shardColumns.size()); final StringTuple end = - segmentNumber == ranges.size() - 1 ? null : makeStringTuple(clusterBy, keyReader, range.getEnd()); + segmentNumber == ranges.size() - 1 + ? null + : makeStringTuple(clusterBy, keyReader, range.getEnd(), shardColumns.size()); shardSpec = new DimensionRangeShardSpec(shardColumns, start, end, segmentNumber, ranges.size()); } @@ -1942,12 +1943,15 @@ private static DataSchema makeDataSchemaForIngestion( { final DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); final boolean isRollupQuery = isRollupQuery(querySpec.getQuery()); + final boolean forceSegmentSortByTime = + MultiStageQueryContext.isForceSegmentSortByTime(querySpec.getQuery().context()); - final Pair, List> dimensionsAndAggregators = + final Pair> dimensionsAndAggregators = makeDimensionsAndAggregatorsForIngestion( querySignature, queryClusterBy, destination.getSegmentSortOrder(), + forceSegmentSortByTime, columnMappings, isRollupQuery, querySpec.getQuery(), @@ -1957,7 +1961,7 @@ private static DataSchema makeDataSchemaForIngestion( return new DataSchema( destination.getDataSource(), new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null), - new DimensionsSpec(dimensionsAndAggregators.lhs), + dimensionsAndAggregators.lhs, dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]), makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper), new TransformSpec(null, Collections.emptyList()) @@ -2038,6 +2042,8 @@ private static boolean isRollupQuery(Query query) /** * Compute shard columns for {@link DimensionRangeShardSpec}. Returns an empty list if range-based sharding * is not applicable. + * + * @return pair of shard columns and commentary */ private static Pair, String> computeShardColumns( final RowSignature signature, @@ -2048,7 +2054,10 @@ private static Pair, String> computeShardColumns( { if (mayHaveMultiValuedClusterByFields) { // DimensionRangeShardSpec cannot handle multivalued fields. - return Pair.of(Collections.emptyList(), "Cannot use RangeShardSpec, the fields in the CLUSTERED BY clause contains multivalued fields. Using NumberedShardSpec instead."); + return Pair.of( + Collections.emptyList(), + "Cannot use 'range' shard specs since CLUSTERED BY contains multi-valued fields." + ); } final List clusterByColumns = clusterBy.getColumns(); final List shardColumns = new ArrayList<>(); @@ -2056,7 +2065,7 @@ private static Pair, String> computeShardColumns( final int numShardColumns = clusterByColumns.size() - clusterBy.getBucketByCount() - (boosted ? 1 : 0); if (numShardColumns == 0) { - return Pair.of(Collections.emptyList(), "Using NumberedShardSpec as no columns are supplied in the 'CLUSTERED BY' clause."); + return Pair.of(Collections.emptyList(), "CLUSTERED BY clause is empty."); } for (int i = clusterBy.getBucketByCount(); i < clusterBy.getBucketByCount() + numShardColumns; i++) { @@ -2065,25 +2074,47 @@ private static Pair, String> computeShardColumns( // DimensionRangeShardSpec only handles ascending order. if (column.order() != KeyOrder.ASCENDING) { - return Pair.of(Collections.emptyList(), "Cannot use RangeShardSpec, RangedShardSpec only supports ascending CLUSTER BY keys. Using NumberedShardSpec instead."); + return Pair.of( + shardColumns, + StringUtils.format( + "Using[%d] CLUSTERED BY columns for 'range' shard specs, since the next column has order[%s].", + shardColumns.size(), + column.order() + ) + ); } ColumnType columnType = signature.getColumnType(column.columnName()).orElse(null); // DimensionRangeShardSpec only handles strings. if (!(ColumnType.STRING.equals(columnType))) { - return Pair.of(Collections.emptyList(), "Cannot use RangeShardSpec, RangedShardSpec only supports string CLUSTER BY keys. Using NumberedShardSpec instead."); + return Pair.of( + shardColumns, + StringUtils.format( + "Using[%d] CLUSTERED BY columns for 'range' shard specs, since the next column is of type[%s]. " + + "Only string columns are included in 'range' shard specs.", + shardColumns.size(), + columnType + ) + ); } // DimensionRangeShardSpec only handles columns that appear as-is in the output. if (outputColumns.isEmpty()) { - return Pair.of(Collections.emptyList(), StringUtils.format("Cannot use RangeShardSpec, Could not find output column name for column [%s]. Using NumberedShardSpec instead.", column.columnName())); + return Pair.of( + shardColumns, + StringUtils.format( + "Using only[%d] CLUSTERED BY columns for 'range' shard specs, since the next column was not mapped to " + + "an output column.", + shardColumns.size() + ) + ); } shardColumns.add(columnMappings.getOutputColumnName(outputColumns.getInt(0))); } - return Pair.of(shardColumns, "Using RangeShardSpec to generate segments."); + return Pair.of(shardColumns, "Using 'range' shard specs with all CLUSTERED BY fields."); } /** @@ -2102,22 +2133,15 @@ private static boolean isClusterByBoosted(final ClusterBy clusterBy) private static StringTuple makeStringTuple( final ClusterBy clusterBy, final RowKeyReader keyReader, - final RowKey key + final RowKey key, + final int shardFieldCount ) { final String[] array = new String[clusterBy.getColumns().size() - clusterBy.getBucketByCount()]; - final boolean boosted = isClusterByBoosted(clusterBy); - for (int i = 0; i < array.length; i++) { + for (int i = 0; i < shardFieldCount; i++) { final Object val = keyReader.read(key, clusterBy.getBucketByCount() + i); - - if (i == array.length - 1 && boosted) { - // Boost column - //noinspection RedundantCast: false alarm; the cast is necessary - array[i] = StringUtils.format("%016d", (long) val); - } else { - array[i] = (String) val; - } + array[i] = (String) val; } return new StringTuple(array); @@ -2133,7 +2157,8 @@ private static DimensionSchema getDimensionSchema( if (dimensionSchemas != null && dimensionSchemas.containsKey(outputColumnName)) { return dimensionSchemas.get(outputColumnName); } - // In case of ingestion, or when metrics are converted to dimensions when compaction is performed without rollup, + // In case of ingestion, or when metrics are converted to dimensions when compaction is performed without rollu + // we won't have an entry in the map. For those cases, use the default config. return DimensionSchemaUtils.createDimensionSchema( outputColumnName, @@ -2143,10 +2168,11 @@ private static DimensionSchema getDimensionSchema( ); } - private static Pair, List> makeDimensionsAndAggregatorsForIngestion( + private static Pair> makeDimensionsAndAggregatorsForIngestion( final RowSignature querySignature, final ClusterBy queryClusterBy, - final List segmentSortOrder, + final List contextSegmentSortOrder, + final boolean forceSegmentSortByTime, final ColumnMappings columnMappings, final boolean isRollupQuery, final Query query, @@ -2176,7 +2202,12 @@ private static Pair, List> makeDimensio // that order. // Start with segmentSortOrder. - final Set outputColumnsInOrder = new LinkedHashSet<>(segmentSortOrder); + final Set outputColumnsInOrder = new LinkedHashSet<>(contextSegmentSortOrder); + + // Then __time, if it's an output column and forceSegmentSortByTime is set. + if (columnMappings.hasOutputColumn(ColumnHolder.TIME_COLUMN_NAME) && forceSegmentSortByTime) { + outputColumnsInOrder.add(ColumnHolder.TIME_COLUMN_NAME); + } // Then the query-level CLUSTERED BY. // Note: this doesn't work when CLUSTERED BY specifies an expression that is not being selected. @@ -2211,7 +2242,7 @@ private static Pair, List> makeDimensio } } - // Each column can be of either time, dimension, aggregator. For this method. we can ignore the time column. + // Each column can be either a dimension or an aggregator. // For non-complex columns, If the aggregator factory of the column is not available, we treat the column as // a dimension. For complex columns, certains hacks are in place. for (final String outputColumnName : outputColumnsInOrder) { @@ -2226,10 +2257,26 @@ private static Pair, List> makeDimensio querySignature.getColumnType(queryColumn) .orElseThrow(() -> new ISE("No type for column [%s]", outputColumnName)); - if (!outputColumnName.equals(ColumnHolder.TIME_COLUMN_NAME)) { - - if (!type.is(ValueType.COMPLEX)) { - // non complex columns + if (!type.is(ValueType.COMPLEX)) { + // non complex columns + populateDimensionsAndAggregators( + dimensions, + aggregators, + outputColumnAggregatorFactories, + outputColumnName, + type, + query.context(), + dimensionSchemas + ); + } else { + // complex columns only + if (DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.containsKey(type.getComplexTypeName())) { + dimensions.add( + getDimensionSchema(outputColumnName, type, query.context(), dimensionSchemas) + ); + } else if (!isRollupQuery) { + aggregators.add(new PassthroughAggregatorFactory(outputColumnName, type.getComplexTypeName())); + } else { populateDimensionsAndAggregators( dimensions, aggregators, @@ -2239,32 +2286,23 @@ private static Pair, List> makeDimensio query.context(), dimensionSchemas ); - } else { - // complex columns only - if (DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.containsKey(type.getComplexTypeName())) { - dimensions.add( - getDimensionSchema(outputColumnName, type, query.context(), dimensionSchemas) - ); - } else if (!isRollupQuery) { - aggregators.add(new PassthroughAggregatorFactory(outputColumnName, type.getComplexTypeName())); - } else { - populateDimensionsAndAggregators( - dimensions, - aggregators, - outputColumnAggregatorFactories, - outputColumnName, - type, - query.context(), - dimensionSchemas - ); - } } } } - return Pair.of(dimensions, aggregators); - } + final DimensionsSpec.Builder dimensionsSpecBuilder = DimensionsSpec.builder(); + + if (!dimensions.isEmpty() && dimensions.get(0).getName().equals(ColumnHolder.TIME_COLUMN_NAME)) { + // Skip __time if it's in the first position, for compatibility with legacy dimensionSpecs. + dimensions.remove(0); + dimensionsSpecBuilder.setForceSegmentSortByTime(null); + } else { + // Store explicit forceSegmentSortByTime only if false, for compatibility with legacy dimensionSpecs. + dimensionsSpecBuilder.setForceSegmentSortByTime(forceSegmentSortByTime ? null : false); + } + return Pair.of(dimensionsSpecBuilder.setDimensions(dimensions).build(), aggregators); + } /** * If the output column is present in the outputColumnAggregatorFactories that means we already have the aggregator information for this column. @@ -2286,7 +2324,12 @@ private static void populateDimensionsAndAggregators( Map dimensionSchemas ) { - if (outputColumnAggregatorFactories.containsKey(outputColumn)) { + if (ColumnHolder.TIME_COLUMN_NAME.equals(outputColumn)) { + if (!type.is(ValueType.LONG)) { + throw DruidException.defensive("Incorrect type[%s] for column[%s]", type, outputColumn); + } + dimensions.add(new LongDimensionSchema(outputColumn)); + } else if (outputColumnAggregatorFactories.containsKey(outputColumn)) { aggregators.add(outputColumnAggregatorFactories.get(outputColumn)); } else { dimensions.add( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index ac568322400e..a859ea8cd534 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -55,6 +55,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.timeline.SegmentId; @@ -147,11 +148,13 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment { if (resultYielder == null) { final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); + final SegmentReference mappedSegment = mapSegment(segmentHolder.get()); final Sequence rowSequence = groupingEngine.process( query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())), - mapSegment(segmentHolder.get()).asStorageAdapter(), + mappedSegment.asStorageAdapter(), + mappedSegment.as(TimeBoundaryInspector.class), null ); @@ -179,11 +182,13 @@ protected ReturnOrAwait runWithInputChannel( if (inputChannel.canRead()) { final Frame frame = inputChannel.read(); final FrameSegment frameSegment = new FrameSegment(frame, inputFrameReader, SegmentId.dummy("x")); + final SegmentReference mappedSegment = mapSegment(frameSegment); final Sequence rowSequence = groupingEngine.process( query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)), - mapSegment(frameSegment).asStorageAdapter(), + mappedSegment.asStorageAdapter(), + mappedSegment.as(TimeBoundaryInspector.class), null ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java index 7af34a1eb55b..60fa197c4a58 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java @@ -237,21 +237,20 @@ public QueryResponse runQuery(final DruidQuery druidQuery) final List segmentSortOrder = MultiStageQueryContext.getSortOrder(sqlQueryContext); - MSQTaskQueryMakerUtils.validateSegmentSortOrder( + MSQTaskQueryMakerUtils.validateContextSortOrderColumnsExist( segmentSortOrder, - fieldMapping.stream().map(f -> f.getValue()).collect(Collectors.toList()) + fieldMapping.stream().map(Entry::getValue).collect(Collectors.toSet()) ); - final DataSourceMSQDestination dataSourceMSQDestination = new DataSourceMSQDestination( + final DataSourceMSQDestination dataSourceDestination = new DataSourceMSQDestination( targetDataSource.getDestinationName(), segmentGranularityObject, segmentSortOrder, replaceTimeChunks, null ); - MultiStageQueryContext.validateAndGetTaskLockType(sqlQueryContext, - dataSourceMSQDestination.isReplaceTimeChunks()); - destination = dataSourceMSQDestination; + MultiStageQueryContext.validateAndGetTaskLockType(sqlQueryContext, dataSourceDestination.isReplaceTimeChunks()); + destination = dataSourceDestination; } else { final MSQSelectDestination msqSelectDestination = MultiStageQueryContext.getSelectDestination(sqlQueryContext); if (msqSelectDestination.equals(MSQSelectDestination.TASKREPORT)) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java index e3baa058b41a..d20abffb0f9f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java @@ -23,6 +23,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; +import org.apache.calcite.rel.RelCollation; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.RelRoot; import org.apache.calcite.rel.core.Project; @@ -33,6 +35,7 @@ import org.apache.calcite.schema.Table; import org.apache.calcite.sql.dialect.CalciteSqlDialect; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; import org.apache.druid.error.InvalidSqlInput; @@ -43,6 +46,7 @@ import org.apache.druid.msq.util.ArrayIngestMode; import org.apache.druid.msq.util.DimensionSchemaUtils; import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.QueryContext; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; @@ -61,7 +65,6 @@ import org.apache.druid.sql.destination.TableDestination; import javax.annotation.Nullable; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -176,8 +179,7 @@ public QueryMaker buildQueryMakerForInsert( ) { validateInsert( - relRoot.rel, - relRoot.fields, + relRoot, destination instanceof TableDestination ? plannerContext.getPlannerToolbox() .rootSchema() @@ -221,35 +223,38 @@ private static void validateSelect(final PlannerContext plannerContext) * Engine-specific validation that happens after the query is planned. */ private static void validateInsert( - final RelNode rootRel, - final List> fieldMappings, + final RelRoot relRoot, @Nullable Table targetTable, final PlannerContext plannerContext ) { - final int timeColumnIndex = getTimeColumnIndex(fieldMappings); + final Map outputFieldMap = validateNoDuplicateAliases(relRoot.fields); + final int timeColumnIndex = getTimeColumnIndex(outputFieldMap); final Granularity segmentGranularity = getSegmentGranularity(plannerContext); - - validateNoDuplicateAliases(fieldMappings); - validateTimeColumnType(rootRel, timeColumnIndex); + validateTimeColumnType(relRoot.rel, timeColumnIndex); validateTimeColumnExistsIfNeeded(timeColumnIndex, segmentGranularity); - validateLimitAndOffset(rootRel, Granularities.ALL.equals(segmentGranularity)); - validateTypeChanges(rootRel, fieldMappings, targetTable, plannerContext); + validateLimitAndOffset(relRoot.rel, Granularities.ALL.equals(segmentGranularity)); + validateTypeChanges(relRoot.rel, relRoot.fields, targetTable, plannerContext); + validateSortOrderBeginsWithTimeIfRequired(relRoot.fields, relRoot.collation, plannerContext); } /** * SQL allows multiple output columns with the same name. However, we don't allow this for INSERT or REPLACE * queries, because we use these output names to generate columns in segments. They must be unique. + * + * @return map of output alias (rhs of the Pair) to position in the {@link RelRoot#rel} (lhs of the Pair) */ - private static void validateNoDuplicateAliases(final List> fieldMappings) + private static Map validateNoDuplicateAliases(final List> fieldMappings) { - final Set aliasesSeen = new HashSet<>(); + final Map retVal = new Object2IntOpenHashMap<>(); for (final Entry field : fieldMappings) { - if (!aliasesSeen.add(field.getValue())) { + if (retVal.put(field.getValue(), field.getKey()) != null) { throw InvalidSqlInput.exception("Duplicate field in SELECT: [%s]", field.getValue()); } } + + return retVal; } /** @@ -347,7 +352,7 @@ private static void validateLimitAndOffset(final RelNode rootRel, final boolean * MVDs and arrays are mixed. So, we detect them as invalid. * * @param rootRel root rel - * @param fieldMappings field mappings from {@link #validateInsert(RelNode, List, Table, PlannerContext)} + * @param fieldMappings field mappings from {@link #validateInsert(RelRoot, Table, PlannerContext)} * @param targetTable table we are inserting (or replacing) into, if any * @param plannerContext planner context */ @@ -422,20 +427,80 @@ private static void validateTypeChanges( } /** - * Returns the index of {@link ColumnHolder#TIME_COLUMN_NAME} within a list of field mappings from - * {@link #validateInsert(RelNode, List, Table, PlannerContext)}. + * Validate that the sort order given by CLUSTERED BY or {@link MultiStageQueryContext#getSortOrder(QueryContext)} + * begins with {@link ColumnHolder#TIME_COLUMN_NAME}, unless {@link MultiStageQueryContext#CTX_FORCE_TIME_SORT} + * is set. * - * Returns -1 if the list does not contain a time column. + * @param fieldMappings field mappings from {@link #validateInsert(RelRoot, Table, PlannerContext)} + * @param rootCollation collation of the root rel. Corresponds to the CLUSTERED BY + * @param plannerContext planner context */ - private static int getTimeColumnIndex(final List> fieldMappings) + private static void validateSortOrderBeginsWithTimeIfRequired( + final List> fieldMappings, + final RelCollation rootCollation, + final PlannerContext plannerContext + ) { - for (final Entry field : fieldMappings) { - if (field.getValue().equals(ColumnHolder.TIME_COLUMN_NAME)) { - return field.getKey(); + // Segment sort order is determined by the segmentSortOrder parameter if set. Otherwise it's determined by + // the rootCollation. + + final QueryContext context = plannerContext.queryContext(); + + if (!MultiStageQueryContext.isForceSegmentSortByTime(context)) { + // Any sort order is allowed. Skip check. + return; + } + + final List contextSortOrder = MultiStageQueryContext.getSortOrder(context); + + if (!contextSortOrder.isEmpty()) { + final boolean timeIsFirst = ColumnHolder.TIME_COLUMN_NAME.equals(contextSortOrder.get(0)); + + if (!timeIsFirst) { + throw InvalidSqlInput.exception( + "Context parameter[%s] must start with[%s] unless context parameter[%s] is set to[false]. %s", + MultiStageQueryContext.CTX_SORT_ORDER, + ColumnHolder.TIME_COLUMN_NAME, + MultiStageQueryContext.CTX_FORCE_TIME_SORT, + DimensionsSpec.WARNING_NON_TIME_SORT_ORDER + ); + } + } else if (!rootCollation.getFieldCollations().isEmpty()) { + int timePosition = -1; + for (int i = 0; i < rootCollation.getFieldCollations().size(); i++) { + final String fieldCollationName = + fieldMappings.get(rootCollation.getFieldCollations().get(i).getFieldIndex()).getValue(); + if (ColumnHolder.TIME_COLUMN_NAME.equals(fieldCollationName)) { + timePosition = i; + break; + } + } + + if (timePosition > 0) { + throw InvalidSqlInput.exception( + "Sort order (CLUSTERED BY) cannot include[%s] in position[%d] unless context parameter[%s] " + + "is set to[false]. %s", + ColumnHolder.TIME_COLUMN_NAME, + timePosition, + MultiStageQueryContext.CTX_FORCE_TIME_SORT, + DimensionsSpec.WARNING_NON_TIME_SORT_ORDER + ); } } + } - return -1; + /** + * Returns the index of {@link ColumnHolder#TIME_COLUMN_NAME} within a list of field mappings from + * {@link #validateInsert(RelRoot, Table, PlannerContext)}. + * + * @param outputFieldMapping mapping from {@link #validateNoDuplicateAliases(List)} + * + * @return field position, or -1 if the list does not contain a time column. + */ + private static int getTimeColumnIndex(final Map outputFieldMapping) + { + final Integer position = outputFieldMapping.get(ColumnHolder.TIME_COLUMN_NAME); + return position != null ? position : -1; } /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtils.java index ae1558babf5c..a30c9bb0aec0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtils.java @@ -21,15 +21,13 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.common.IAE; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.exec.SegmentSource; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; -import org.apache.druid.segment.column.ColumnHolder; -import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -76,24 +74,25 @@ public static String maskSensitiveJsonKeys(String sqlQuery) } /** - * Validates if each element of the sort order appears in the final output and if it is not empty then it starts with the - * __time column + * Validates that each element of {@link MultiStageQueryContext#CTX_SORT_ORDER}, if provided, appears in the + * final output. */ - public static void validateSegmentSortOrder(final List sortOrder, final Collection allOutputColumns) + public static void validateContextSortOrderColumnsExist( + final List contextSortOrder, + final Set allOutputColumns + ) { final Set allOutputColumnsSet = new HashSet<>(allOutputColumns); - for (final String column : sortOrder) { + for (final String column : contextSortOrder) { if (!allOutputColumnsSet.contains(column)) { - throw new IAE("Column [%s] in segment sort order does not appear in the query output", column); + throw InvalidSqlInput.exception( + "Column[%s] from context parameter[%s] does not appear in the query output", + column, + MultiStageQueryContext.CTX_SORT_ORDER + ); } } - - if (sortOrder.size() > 0 - && allOutputColumns.contains(ColumnHolder.TIME_COLUMN_NAME) - && !ColumnHolder.TIME_COLUMN_NAME.equals(sortOrder.get(0))) { - throw new IAE("Segment sort order must begin with column [%s]", ColumnHolder.TIME_COLUMN_NAME); - } } /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 6810b54457a5..9b715f8c8cf9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -25,6 +25,7 @@ import com.google.common.annotations.VisibleForTesting; import com.opencsv.RFC4180Parser; import com.opencsv.RFC4180ParserBuilder; +import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.error.DruidException; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.task.Tasks; @@ -167,6 +168,9 @@ public class MultiStageQueryContext public static final String CTX_ARRAY_INGEST_MODE = "arrayIngestMode"; public static final ArrayIngestMode DEFAULT_ARRAY_INGEST_MODE = ArrayIngestMode.ARRAY; + public static final String CTX_FORCE_TIME_SORT = DimensionsSpec.PARAMETER_FORCE_TIME_SORT; + private static final boolean DEFAULT_FORCE_TIME_SORT = DimensionsSpec.DEFAULT_FORCE_TIME_SORT; + public static final String MAX_ROWS_MATERIALIZED_IN_WINDOW = "maxRowsMaterializedInWindow"; public static final String CTX_SKIP_TYPE_VERIFICATION = "skipTypeVerification"; @@ -361,6 +365,11 @@ public static ArrayIngestMode getArrayIngestMode(final QueryContext queryContext return queryContext.getEnum(CTX_ARRAY_INGEST_MODE, ArrayIngestMode.class, DEFAULT_ARRAY_INGEST_MODE); } + public static boolean isForceSegmentSortByTime(final QueryContext queryContext) + { + return queryContext.getBoolean(CTX_FORCE_TIME_SORT, DEFAULT_FORCE_TIME_SORT); + } + public static Set getColumnsExcludedFromTypeVerification(final QueryContext queryContext) { return new HashSet<>(decodeList(CTX_SKIP_TYPE_VERIFICATION, queryContext.getString(CTX_SKIP_TYPE_VERIFICATION))); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index cbdddf4a46ef..96da93c34d07 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -76,6 +76,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -103,6 +104,7 @@ public static Collection data() }; return Arrays.asList(data); } + @MethodSource("data") @ParameterizedTest(name = "{index}:with context {0}") public void testReplaceOnFooWithAll(String contextName, Map context) @@ -129,10 +131,7 @@ public void testReplaceOnFooWithAll(String contextName, Map cont Mockito.doCallRealMethod() .doReturn(ImmutableSet.of(existingDataSegment0, existingDataSegment1)) .when(testTaskActionClient) - .submit(new RetrieveUsedSegmentsAction( - EasyMock.eq("foo"), - EasyMock.eq(ImmutableList.of(Intervals.ETERNITY)) - )); + .submit(new RetrieveUsedSegmentsAction("foo", ImmutableList.of(Intervals.ETERNITY))); testIngestQuery().setSql(" REPLACE INTO foo OVERWRITE ALL " + "SELECT __time, m1 " @@ -194,6 +193,378 @@ public void testReplaceOnFooWithAll(String contextName, Map cont .verifyResults(); } + @MethodSource("data") + @ParameterizedTest(name = "{index}:with context {0}") + public void testReplaceOnFooWithAllClusteredByDim(String contextName, Map context) + { + // Tests [CLUSTERED BY dim1] with the default forceSegmentSortByTime (true). In this case, + // partitioning uses [dim1] but segment sort uses [__time, dim1]. + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("m1", ColumnType.FLOAT) + .build(); + + DataSegment existingDataSegment0 = DataSegment.builder() + .interval(Intervals.of("2000-01-01T/2000-01-04T")) + .size(50) + .version(MSQTestTaskActionClient.VERSION) + .dataSource("foo") + .build(); + + DataSegment existingDataSegment1 = DataSegment.builder() + .interval(Intervals.of("2001-01-01T/2001-01-04T")) + .size(50) + .version(MSQTestTaskActionClient.VERSION) + .dataSource("foo") + .build(); + + Mockito.doCallRealMethod() + .doReturn(ImmutableSet.of(existingDataSegment0, existingDataSegment1)) + .when(testTaskActionClient) + .submit(new RetrieveUsedSegmentsAction("foo", ImmutableList.of(Intervals.ETERNITY))); + + testIngestQuery().setSql(" REPLACE INTO foo OVERWRITE ALL " + + "SELECT __time, dim1, m1 " + + "FROM foo " + + "PARTITIONED BY ALL " + + "CLUSTERED BY dim1") + .setExpectedDataSource("foo") + .setExpectedRowSignature(rowSignature) + .setQueryContext(context) + .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) + .setExpectedSegments( + ImmutableSet.of( + SegmentId.of("foo", Intervals.ETERNITY, "test", 0) + ) + ) + .setExpectedShardSpec(DimensionRangeShardSpec.class) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, NullHandling.sqlCompatible() ? "" : null, 1.0f}, + new Object[]{946771200000L, "10.1", 2.0f}, + new Object[]{946857600000L, "2", 3.0f}, + new Object[]{978307200000L, "1", 4.0f}, + new Object[]{978393600000L, "def", 5.0f}, + new Object[]{978480000000L, "abc", 6.0f} + ) + ) + .setExpectedSegmentGenerationProgressCountersForStageWorker( + CounterSnapshotMatcher + .with().segmentRowsProcessed(6), + 1, 0 + ) + .setExpectedLastCompactionState( + expectedCompactionState( + context, + Collections.singletonList("dim1"), + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dim1"), + new FloatDimensionSchema("m1") + ) + ) + .setDimensionExclusions(Collections.singletonList("__time")) + .build(), + GranularityType.ALL, + Intervals.ETERNITY + ) + ) + .verifyResults(); + } + + @MethodSource("data") + @ParameterizedTest(name = "{index}:with context {0}") + public void testReplaceOnFooWithAllClusteredByDimExplicitSort(String contextName, Map context) + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("m1", ColumnType.FLOAT) + .build(); + + DataSegment existingDataSegment0 = DataSegment.builder() + .interval(Intervals.of("2000-01-01T/2000-01-04T")) + .size(50) + .version(MSQTestTaskActionClient.VERSION) + .dataSource("foo") + .build(); + + DataSegment existingDataSegment1 = DataSegment.builder() + .interval(Intervals.of("2001-01-01T/2001-01-04T")) + .size(50) + .version(MSQTestTaskActionClient.VERSION) + .dataSource("foo") + .build(); + + Map queryContext = new HashMap<>(context); + queryContext.put(DimensionsSpec.PARAMETER_FORCE_TIME_SORT, false); + + Mockito.doCallRealMethod() + .doReturn(ImmutableSet.of(existingDataSegment0, existingDataSegment1)) + .when(testTaskActionClient) + .submit(new RetrieveUsedSegmentsAction("foo", ImmutableList.of(Intervals.ETERNITY))); + + testIngestQuery().setSql(" REPLACE INTO foo OVERWRITE ALL " + + "SELECT __time, dim1, m1 " + + "FROM foo " + + "PARTITIONED BY ALL " + + "CLUSTERED BY dim1") + .setExpectedDataSource("foo") + .setExpectedRowSignature(rowSignature) + .setQueryContext(queryContext) + .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) + .setExpectedSegments( + ImmutableSet.of( + SegmentId.of("foo", Intervals.ETERNITY, "test", 0) + ) + ) + .setExpectedShardSpec(DimensionRangeShardSpec.class) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, NullHandling.sqlCompatible() ? "" : null, 1.0f}, + new Object[]{978307200000L, "1", 4.0f}, + new Object[]{946771200000L, "10.1", 2.0f}, + new Object[]{946857600000L, "2", 3.0f}, + new Object[]{978480000000L, "abc", 6.0f}, + new Object[]{978393600000L, "def", 5.0f} + ) + ) + .setExpectedSegmentGenerationProgressCountersForStageWorker( + CounterSnapshotMatcher + .with().segmentRowsProcessed(6), + 1, 0 + ) + .setExpectedLastCompactionState( + expectedCompactionState( + context, + Collections.singletonList("dim1"), + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("__time"), + new FloatDimensionSchema("m1") + ) + ) + .setForceSegmentSortByTime(false) + .build(), + GranularityType.ALL, + Intervals.ETERNITY + ) + ) + .verifyResults(); + } + + @MethodSource("data") + @ParameterizedTest(name = "{index}:with context {0}") + public void testReplaceOnFooWithAllClusteredByDimThenTimeExplicitSort(String contextName, Map context) + { + // Tests that [CLUSTERED BY dim1, __time] and [CLUSTERED BY dim1] are same when + // forceSegmentSortByTime = false. (Same expectations as the prior test, + // testReplaceOnFooWithAllClusteredByDimExplicitSort.) + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("m1", ColumnType.FLOAT) + .build(); + + DataSegment existingDataSegment0 = DataSegment.builder() + .interval(Intervals.of("2000-01-01T/2000-01-04T")) + .size(50) + .version(MSQTestTaskActionClient.VERSION) + .dataSource("foo") + .build(); + + DataSegment existingDataSegment1 = DataSegment.builder() + .interval(Intervals.of("2001-01-01T/2001-01-04T")) + .size(50) + .version(MSQTestTaskActionClient.VERSION) + .dataSource("foo") + .build(); + + Map queryContext = new HashMap<>(context); + queryContext.put(DimensionsSpec.PARAMETER_FORCE_TIME_SORT, false); + + Mockito.doCallRealMethod() + .doReturn(ImmutableSet.of(existingDataSegment0, existingDataSegment1)) + .when(testTaskActionClient) + .submit(new RetrieveUsedSegmentsAction("foo", ImmutableList.of(Intervals.ETERNITY))); + + testIngestQuery().setSql(" REPLACE INTO foo OVERWRITE ALL " + + "SELECT __time, dim1, m1 " + + "FROM foo " + + "PARTITIONED BY ALL " + + "CLUSTERED BY dim1, __time") + .setExpectedDataSource("foo") + .setExpectedRowSignature(rowSignature) + .setQueryContext(queryContext) + .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) + .setExpectedSegments( + ImmutableSet.of( + SegmentId.of("foo", Intervals.ETERNITY, "test", 0) + ) + ) + .setExpectedShardSpec(DimensionRangeShardSpec.class) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, NullHandling.sqlCompatible() ? "" : null, 1.0f}, + new Object[]{978307200000L, "1", 4.0f}, + new Object[]{946771200000L, "10.1", 2.0f}, + new Object[]{946857600000L, "2", 3.0f}, + new Object[]{978480000000L, "abc", 6.0f}, + new Object[]{978393600000L, "def", 5.0f} + ) + ) + .setExpectedSegmentGenerationProgressCountersForStageWorker( + CounterSnapshotMatcher + .with().segmentRowsProcessed(6), + 1, 0 + ) + .setExpectedLastCompactionState( + expectedCompactionState( + context, + Collections.singletonList("dim1"), + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("__time"), + new FloatDimensionSchema("m1") + ) + ) + .setForceSegmentSortByTime(false) + .build(), + GranularityType.ALL, + Intervals.ETERNITY + ) + ) + .verifyResults(); + } + + @MethodSource("data") + @ParameterizedTest(name = "{index}:with context {0}") + public void testReplaceOnFooWithAllClusteredByDimThenTimeError(String contextName, Map context) + { + // Tests that [CLUSTERED BY dim1, __time] is an error when forceSegmentSortByTime = true (the default). + testIngestQuery().setSql(" REPLACE INTO foo OVERWRITE ALL " + + "SELECT __time, dim1, m1 " + + "FROM foo " + + "PARTITIONED BY ALL " + + "CLUSTERED BY dim1, __time") + .setExpectedDataSource("foo") + .setQueryContext(context) + .setExpectedValidationErrorMatcher(invalidSqlContains( + "Sort order (CLUSTERED BY) cannot include[__time] in position[1] unless context " + + "parameter[forceSegmentSortByTime] is set to[false]." + )) + .verifyPlanningErrors(); + } + + @MethodSource("data") + @ParameterizedTest(name = "{index}:with context {0}") + public void testReplaceOnFooWithAllClusteredByDimThenTimeError2(String contextName, Map context) + { + // Tests that setting segmentSortOrder = [dim1, __time] is an error when + // forceSegmentSortByTime = false (the default). + Map queryContext = new HashMap<>(context); + queryContext.put(MultiStageQueryContext.CTX_SORT_ORDER, "dim1, __time"); + + testIngestQuery().setSql(" REPLACE INTO foo OVERWRITE ALL " + + "SELECT __time, dim1, m1 " + + "FROM foo " + + "PARTITIONED BY ALL " + + "CLUSTERED BY dim1") + .setExpectedDataSource("foo") + .setQueryContext(queryContext) + .setExpectedValidationErrorMatcher(invalidSqlContains( + "Context parameter[segmentSortOrder] must start with[__time] unless context " + + "parameter[forceSegmentSortByTime] is set to[false]." + )) + .verifyPlanningErrors(); + } + + @MethodSource("data") + @ParameterizedTest(name = "{index}:with context {0}") + public void testReplaceOnFooWithAllClusteredByTimeThenDimExplicitSort(String contextName, Map context) + { + // Tests [CLUSTERED BY __time, dim1] with forceSegmentSortByTime = false. + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("m1", ColumnType.FLOAT) + .build(); + + DataSegment existingDataSegment0 = DataSegment.builder() + .interval(Intervals.of("2000-01-01T/2000-01-04T")) + .size(50) + .version(MSQTestTaskActionClient.VERSION) + .dataSource("foo") + .build(); + + DataSegment existingDataSegment1 = DataSegment.builder() + .interval(Intervals.of("2001-01-01T/2001-01-04T")) + .size(50) + .version(MSQTestTaskActionClient.VERSION) + .dataSource("foo") + .build(); + + Map queryContext = new HashMap<>(context); + queryContext.put(DimensionsSpec.PARAMETER_FORCE_TIME_SORT, false); + + Mockito.doCallRealMethod() + .doReturn(ImmutableSet.of(existingDataSegment0, existingDataSegment1)) + .when(testTaskActionClient) + .submit(new RetrieveUsedSegmentsAction("foo", ImmutableList.of(Intervals.ETERNITY))); + + testIngestQuery().setSql(" REPLACE INTO foo OVERWRITE ALL " + + "SELECT __time, dim1, m1 " + + "FROM foo " + + "PARTITIONED BY ALL " + + "CLUSTERED BY __time, dim1") + .setExpectedDataSource("foo") + .setExpectedRowSignature(rowSignature) + .setQueryContext(queryContext) + .setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY) + .setExpectedSegments( + ImmutableSet.of( + SegmentId.of("foo", Intervals.ETERNITY, "test", 0) + ) + ) + .setExpectedShardSpec(NumberedShardSpec.class) + .setExpectedResultRows( + ImmutableList.of( + new Object[]{946684800000L, NullHandling.sqlCompatible() ? "" : null, 1.0f}, + new Object[]{946771200000L, "10.1", 2.0f}, + new Object[]{946857600000L, "2", 3.0f}, + new Object[]{978307200000L, "1", 4.0f}, + new Object[]{978393600000L, "def", 5.0f}, + new Object[]{978480000000L, "abc", 6.0f} + ) + ) + .setExpectedSegmentGenerationProgressCountersForStageWorker( + CounterSnapshotMatcher + .with().segmentRowsProcessed(6), + 1, 0 + ) + .setExpectedLastCompactionState( + expectedCompactionState( + context, + Collections.emptyList(), + // For backwards-compatibility, compaction state is stored as if + // forceSegmentSortByTime = true. + ImmutableList.of( + new StringDimensionSchema("dim1"), + new FloatDimensionSchema("m1") + ), + GranularityType.ALL, + Intervals.ETERNITY + ) + ) + .verifyResults(); + } + @MethodSource("data") @ParameterizedTest(name = "{index}:with context {0}") public void testReplaceOnFooWithWhere(String contextName, Map context) @@ -281,7 +652,8 @@ public void testReplaceOnFoo1WithAllExtern(String contextName, Map c Mockito.doReturn(ImmutableSet.of(existingDataSegment0)) .when(testTaskActionClient) - .submit(new RetrieveUsedSegmentsAction( - EasyMock.eq("foo"), - EasyMock.eq(ImmutableList.of(Intervals.of("2000-01-01/2000-03-01"))) - )); + .submit(new RetrieveUsedSegmentsAction("foo", ImmutableList.of(Intervals.of("2000-01-01/2000-03-01")))); testIngestQuery().setSql(" REPLACE INTO foo " + "OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01' AND __time < TIMESTAMP '2000-03-01' " @@ -960,7 +1326,7 @@ public void testReplaceOnFoo1WithLimit(String contextName, Map c .setExpectedMSQSegmentReport( new MSQSegmentReport( DimensionRangeShardSpec.class.getSimpleName(), - "Using RangeShardSpec to generate segments." + "Using 'range' shard specs with all CLUSTERED BY fields." ) ) .setExpectedLastCompactionState(expectedCompactionState( @@ -1119,7 +1485,7 @@ public void testReplaceOnFoo1Range(String contextName, Map conte .setExpectedMSQSegmentReport( new MSQSegmentReport( DimensionRangeShardSpec.class.getSimpleName(), - "Using RangeShardSpec to generate segments." + "Using 'range' shard specs with all CLUSTERED BY fields." ) ) .setExpectedRowSignature(rowSignature) @@ -1338,7 +1704,7 @@ public void testReplaceUnnestSegmentEntireTable(String contextName, Map context, + List partitionDimensions, + DimensionsSpec dimensionsSpec, + GranularityType segmentGranularity, + Interval interval + ) { if (!context.containsKey(Tasks.STORE_COMPACTION_STATE_KEY) || !((Boolean) context.get(Tasks.STORE_COMPACTION_STATE_KEY))) { @@ -2187,11 +2572,6 @@ private CompactionState expectedCompactionState( false ); } - DimensionsSpec dimensionsSpec = new DimensionsSpec.Builder() - .setDimensions(dimensions) - .setDimensionExclusions(Collections.singletonList( - "__time")) - .build(); IndexSpec indexSpec = new IndexSpec(null, null, null, null, null, null, null); GranularitySpec granularitySpec = new UniformGranularitySpec( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java index 60d40c6d4f17..b5141e12dc85 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java @@ -42,6 +42,7 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.query.OrderBy; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.Metadata; @@ -334,6 +335,12 @@ public ColumnHolder getColumnHolder(String columnName) return null; } + @Override + public List getOrdering() + { + return Collections.emptyList(); + } + @Override public void close() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtilsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtilsTest.java index 00b76e7ef268..bb0bdcd34d11 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtilsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtilsTest.java @@ -20,6 +20,8 @@ package org.apache.druid.msq.util; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.error.DruidException; import org.junit.Assert; import org.junit.Test; @@ -28,25 +30,21 @@ public class MSQTaskQueryMakerUtilsTest { @Test - public void testValidateSegmentSortOrder() + public void testValidateContextSortOrderColumnsExist() { // These are all OK, so validateSegmentSortOrder does nothing. - MSQTaskQueryMakerUtils.validateSegmentSortOrder(Collections.emptyList(), ImmutableList.of("__time", "a", "b")); - MSQTaskQueryMakerUtils.validateSegmentSortOrder(ImmutableList.of("__time"), ImmutableList.of("__time", "a", "b")); - MSQTaskQueryMakerUtils.validateSegmentSortOrder(ImmutableList.of("__time", "b"), ImmutableList.of("__time", "a", "b")); - MSQTaskQueryMakerUtils.validateSegmentSortOrder(ImmutableList.of("b"), ImmutableList.of("a", "b")); + MSQTaskQueryMakerUtils.validateContextSortOrderColumnsExist(Collections.emptyList(), ImmutableSet.of("__time", "a", "b")); + MSQTaskQueryMakerUtils.validateContextSortOrderColumnsExist(ImmutableList.of("__time"), ImmutableSet.of("__time", "a", "b")); + MSQTaskQueryMakerUtils.validateContextSortOrderColumnsExist(ImmutableList.of("__time", "b"), ImmutableSet.of("__time", "a", "b")); + MSQTaskQueryMakerUtils.validateContextSortOrderColumnsExist(ImmutableList.of("b"), ImmutableSet.of("a", "b")); + MSQTaskQueryMakerUtils.validateContextSortOrderColumnsExist(ImmutableList.of("b", "__time"), ImmutableSet.of("__time", "a", "b")); // These are not OK. Assert.assertThrows( - IllegalArgumentException.class, - () -> MSQTaskQueryMakerUtils.validateSegmentSortOrder(ImmutableList.of("c"), ImmutableList.of("a", "b")) - ); - - Assert.assertThrows( - IllegalArgumentException.class, - () -> MSQTaskQueryMakerUtils.validateSegmentSortOrder( - ImmutableList.of("b", "__time"), - ImmutableList.of("__time", "a", "b") + DruidException.class, + () -> MSQTaskQueryMakerUtils.validateContextSortOrderColumnsExist( + ImmutableList.of("c"), + ImmutableSet.of("a", "b") ) ); } diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java index a91e6353c757..361c84542ada 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.query.QueryRunner; @@ -38,21 +39,24 @@ import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.GroupByQueryRunnerTestHelper; import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.groupby.TestGroupByBuffers; import org.apache.druid.query.groupby.having.GreaterThanHavingSpec; import org.apache.druid.query.groupby.having.OrHavingSpec; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.segment.TestHelper; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.joda.time.Period; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.stream.Collectors; /** * @@ -60,6 +64,9 @@ @RunWith(Parameterized.class) public class VarianceGroupByQueryTest extends InitializedNullHandlingTest { + @MonotonicNonNull + private static TestGroupByBuffers BUFFER_POOLS = null; + private final GroupByQueryConfig config; private final QueryRunner runner; private final GroupByQueryRunnerFactory factory; @@ -69,17 +76,26 @@ public class VarianceGroupByQueryTest extends InitializedNullHandlingTest @Parameterized.Parameters(name = "{0}") public static Collection constructorFeeder() { - return GroupByQueryRunnerTest.constructorFeeder().stream() - .map( - constructor -> - new Object[]{ - constructor[0], - constructor[1], - constructor[2], - constructor[3] - } - ) - .collect(Collectors.toList()); + setUpClass(); + + final List constructors = new ArrayList<>(); + for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { + final GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(config, BUFFER_POOLS); + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory, false)) { + final String testName = StringUtils.format("config=%s, runner=%s", config, runner); + constructors.add(new Object[]{testName, config, factory, runner}); + } + } + + return constructors; + } + + @BeforeClass + public static void setUpClass() + { + if (BUFFER_POOLS == null) { + BUFFER_POOLS = TestGroupByBuffers.createDefault(); + } } public VarianceGroupByQueryTest( diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java index 7e17e6b5848c..b78988701d14 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java @@ -53,7 +53,7 @@ public class VarianceTopNQueryTest extends InitializedNullHandlingTest @Parameterized.Parameters(name = "{0}") public static Iterable constructorFeeder() { - return QueryRunnerTestHelper.transformToConstructionFeeder(TopNQueryRunnerTest.queryRunners()); + return QueryRunnerTestHelper.transformToConstructionFeeder(TopNQueryRunnerTest.queryRunners(true)); } private final QueryRunner> runner; diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index c60874e7111d..8a124435faa5 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -73,6 +73,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -291,7 +292,7 @@ private static IncrementalIndex makeIncrementalIndex( Bucket theBucket, AggregatorFactory[] aggs, HadoopDruidIndexerConfig config, - Iterable oldDimOrder, + @Nullable Iterable oldDimOrder, Map oldCapabilities ) { @@ -313,7 +314,7 @@ private static IncrementalIndex makeIncrementalIndex( .setUseMaxMemoryEstimates(tuningConfig.isUseMaxMemoryEstimates()) .build(); - if (oldDimOrder != null && !indexSchema.getDimensionsSpec().hasCustomDimensions()) { + if (oldDimOrder != null && !indexSchema.getDimensionsSpec().hasFixedDimensions()) { newIndex.loadDimensionIterable(oldDimOrder, oldCapabilities); } @@ -472,7 +473,7 @@ protected void reduce(final BytesWritable key, Iterable values, f private void flushIndexToContextAndClose(BytesWritable key, IncrementalIndex index, Context context) throws IOException, InterruptedException { - final List dimensions = index.getDimensionNames(); + final List dimensions = index.getDimensionNames(false); Iterator rows = index.iterator(); while (rows.hasNext()) { context.progress(); @@ -709,7 +710,7 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) ++lineCount; if (!index.canAppendRow()) { - allDimensionNames.addAll(index.getDimensionOrder()); + allDimensionNames.addAll(index.getDimensionNames(false)); log.info(index.getOutOfRowsReason()); log.info( @@ -752,7 +753,7 @@ public void doRun() bucket, combiningAggs, config, - allDimensionNames, + index.getDimensionOrder(), persistIndex.getColumnFormats() ); startTime = System.currentTimeMillis(); @@ -760,7 +761,7 @@ public void doRun() } } - allDimensionNames.addAll(index.getDimensionOrder()); + allDimensionNames.addAll(index.getDimensionNames(false)); log.info("%,d lines completed.", lineCount); diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/DatasourcePathSpec.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/DatasourcePathSpec.java index c56dfaeb0891..3f107b937db9 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/DatasourcePathSpec.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/DatasourcePathSpec.java @@ -128,7 +128,7 @@ public Job addInputPaths(HadoopDruidIndexerConfig config, Job job) throws IOExce DatasourceIngestionSpec updatedIngestionSpec = ingestionSpec; if (updatedIngestionSpec.getDimensions() == null) { List dims; - if (config.getParser().getParseSpec().getDimensionsSpec().hasCustomDimensions()) { + if (config.getParser().getParseSpec().getDimensionsSpec().hasFixedDimensions()) { dims = config.getParser().getParseSpec().getDimensionsSpec().getDimensionNames(); } else { Set dimSet = Sets.newHashSet( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 5a17c4379f18..3d5c92c543ec 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -620,10 +620,19 @@ public static Function, Set> addCompactionStateToS if (storeCompactionState) { TuningConfig tuningConfig = ingestionSpec.getTuningConfig(); GranularitySpec granularitySpec = ingestionSpec.getDataSchema().getGranularitySpec(); - // We do not need to store dimensionExclusions and spatialDimensions since auto compaction does not support them - DimensionsSpec dimensionsSpec = ingestionSpec.getDataSchema().getDimensionsSpec() == null - ? null - : new DimensionsSpec(ingestionSpec.getDataSchema().getDimensionsSpec().getDimensions()); + DimensionsSpec dimensionsSpec; + if (ingestionSpec.getDataSchema().getDimensionsSpec() == null) { + dimensionsSpec = null; + } else { + // We do not need to store spatial dimensions, since by this point they've been converted to regular dimensions. + // We also do not need to store dimensionExclusions, only dimensions that exist. + final DimensionsSpec inputDimensionsSpec = ingestionSpec.getDataSchema().getDimensionsSpec(); + dimensionsSpec = DimensionsSpec + .builder() + .setDimensions(inputDimensionsSpec.getDimensions()) + .setForceSegmentSortByTime(inputDimensionsSpec.isForceSegmentSortByTimeConfigured()) + .build(); + } // We only need to store filter since that is the only field auto compaction support Map transformSpec = ingestionSpec.getDataSchema().getTransformSpec() == null || TransformSpec.NONE.equals(ingestionSpec.getDataSchema().getTransformSpec()) ? null diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 8659eb0f397e..ab7736b047e1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -42,6 +42,7 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.Checks; import org.apache.druid.indexer.Property; @@ -70,8 +71,9 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.query.Order; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; @@ -109,6 +111,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import java.util.Set; import java.util.TreeMap; import java.util.function.Supplier; @@ -753,8 +756,8 @@ static class ExistingSegmentAnalyzer private Granularity queryGranularity; // For processDimensionsSpec: - private final BiMap uniqueDims = HashBiMap.create(); - private final Map dimensionSchemaMap = new HashMap<>(); + private final BiMap uniqueDims = HashBiMap.create(); // dimension name -> position in sort order + private final Map dimensionSchemaMap = new HashMap<>(); // dimension name -> schema // For processMetricsSpec: private final Set> aggregatorFactoryLists = new HashSet<>(); @@ -831,19 +834,34 @@ public DimensionsSpec getDimensionsSpec() } final BiMap orderedDims = uniqueDims.inverse(); + + // Include __time as a dimension only if required, i.e., if it appears in the sort order after position 0. + final Integer timePosition = uniqueDims.get(ColumnHolder.TIME_COLUMN_NAME); + final boolean includeTimeAsDimension = timePosition != null && timePosition > 0; + final List dimensionSchemas = IntStream.range(0, orderedDims.size()) .mapToObj(i -> { final String dimName = orderedDims.get(i); - return Preconditions.checkNotNull( - dimensionSchemaMap.get(dimName), - "Cannot find dimension[%s] from dimensionSchemaMap", - dimName - ); + if (ColumnHolder.TIME_COLUMN_NAME.equals(dimName) && !includeTimeAsDimension) { + return null; + } else { + return Preconditions.checkNotNull( + dimensionSchemaMap.get(dimName), + "Cannot find dimension[%s] from dimensionSchemaMap", + dimName + ); + } }) + .filter(Objects::nonNull) .collect(Collectors.toList()); - return new DimensionsSpec(dimensionSchemas); + // Store forceSegmentSortByTime only if false, for compatibility with legacy compaction states. + final Boolean forceSegmentSortByTime = includeTimeAsDimension ? false : null; + return DimensionsSpec.builder() + .setDimensions(dimensionSchemas) + .setForceSegmentSortByTime(forceSegmentSortByTime) + .build(); } public AggregatorFactory[] getMetricsSpec() @@ -923,27 +941,30 @@ private void processDimensionsSpec(final QueryableIndex index) return; } - final Map dimensionHandlerMap = index.getDimensionHandlers(); + final List sortOrder = new ArrayList<>(); - for (String dimension : index.getAvailableDimensions()) { - final ColumnHolder columnHolder = Preconditions.checkNotNull( - index.getColumnHolder(dimension), - "Cannot find column for dimension[%s]", - dimension - ); + for (final OrderBy orderBy : index.getOrdering()) { + final String dimension = orderBy.getColumnName(); + if (orderBy.getOrder() != Order.ASCENDING) { + throw DruidException.defensive("Order[%s] for dimension[%s] not supported", orderBy.getOrder(), dimension); + } + sortOrder.add(dimension); + } - if (!uniqueDims.containsKey(dimension)) { - Preconditions.checkNotNull( - dimensionHandlerMap.get(dimension), - "Cannot find dimensionHandler for dimension[%s]", - dimension - ); - - uniqueDims.put(dimension, uniqueDims.size()); - dimensionSchemaMap.put( - dimension, - columnHolder.getColumnFormat().getColumnSchema(dimension) - ); + for (String dimension : Iterables.concat(sortOrder, index.getAvailableDimensions())) { + uniqueDims.computeIfAbsent(dimension, ignored -> uniqueDims.size()); + + if (!dimensionSchemaMap.containsKey(dimension)) { + // Possible for sortOrder to contain a dimension that doesn't exist (i.e. if it's 100% nulls). + // In this case, omit it from dimensionSchemaMap for now. We'll skip it later if *no* segment has an existing + // column for it. + final ColumnHolder columnHolder = index.getColumnHolder(dimension); + if (columnHolder != null) { + dimensionSchemaMap.put( + dimension, + columnHolder.getColumnFormat().getColumnSchema(dimension) + ); + } } } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java index 3a04b71e2f1e..a5f95c2e15f7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/InputRowSchemas.java @@ -79,7 +79,7 @@ public static ColumnsFilter createColumnsFilter( final Set transformOutputNames = transformSpec.getTransforms().stream().map(Transform::getName).collect(Collectors.toSet()); - if (dimensionsSpec.hasCustomDimensions()) { + if (dimensionsSpec.hasFixedDimensions()) { // We need an inclusion-based filter. final Set inclusions = new HashSet<>(); 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 5018795cbb4f..4e5e8634f18a 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 @@ -196,7 +196,9 @@ public SamplerResponse sample( Map parsed = new LinkedHashMap<>(); parsed.put(ColumnHolder.TIME_COLUMN_NAME, row.getTimestampFromEpoch()); - columnNames.forEach(k -> parsed.put(k, row.getRaw(k))); + columnNames.stream() + .filter(k -> !ColumnHolder.TIME_COLUMN_NAME.equals(k)) + .forEach(k -> parsed.put(k, row.getRaw(k))); Number sortKey = row.getMetric(SamplerInputRow.SAMPLER_ORDERING_COLUMN); if (sortKey != null) { 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 6fbf9e5deed4..307a44dbf7ec 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 @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import com.google.common.io.Files; import com.google.common.util.concurrent.Futures; @@ -34,6 +35,7 @@ import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.NewSpatialDimensionSchema; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.StringDimensionSchema; @@ -61,6 +63,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -76,6 +79,7 @@ import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnType; @@ -1865,6 +1869,161 @@ public void testRunWithAutoCastDimensions() throws Exception Assert.assertEquals(rows, rowsFromSegment); } + @Test + public void testRunWithAutoCastDimensionsSortByDimension() throws Exception + { + // Compaction will produce one segment sorted by [x, __time], even though input rows are sorted by __time. + final List rows = ImmutableList.of( + "2014-01-01T00:00:10Z,a,10,100,1\n", + "2014-01-01T00:00:10Z,b,20,110,2\n", + "2014-01-01T00:00:10Z,c,30,120,3\n", + "2014-01-01T00:01:20Z,a,10,100,1\n", + "2014-01-01T00:01:20Z,b,20,110,2\n", + "2014-01-01T00:01:20Z,c,30,120,3\n" + ); + final ParseSpec spec = new CSVParseSpec( + new TimestampSpec("ts", "auto", null), + DimensionsSpec.builder() + .setDimensions(Arrays.asList( + new AutoTypeColumnSchema("x", ColumnType.LONG), + new LongDimensionSchema("__time"), + new AutoTypeColumnSchema("ts", ColumnType.STRING), + new AutoTypeColumnSchema("dim", null), + new AutoTypeColumnSchema("y", ColumnType.LONG) + )) + .setForceSegmentSortByTime(false) + .build(), + "|", + Arrays.asList("ts", "dim", "x", "y", "val"), + false, + 0 + ); + Pair indexTaskResult = runIndexTask(null, null, spec, rows, false); + verifySchema(indexTaskResult.rhs); + + final Builder builder = new Builder( + DATA_SOURCE, + segmentCacheManagerFactory + ); + + final CompactionTask compactionTask = builder + .interval(Intervals.of("2014-01-01/2014-01-02")) + .build(); + + final Pair resultPair = runTask(compactionTask); + verifySchema(resultPair.rhs); + + Assert.assertTrue(resultPair.lhs.isSuccess()); + + final DataSegmentsWithSchemas dataSegmentsWithSchemas = resultPair.rhs; + final List segments = new ArrayList<>(dataSegmentsWithSchemas.getSegments()); + Assert.assertEquals(1, segments.size()); + + final DataSegment compactSegment = Iterables.getOnlyElement(segments); + Assert.assertEquals( + Intervals.of("2014-01-01T00:00:00/2014-01-01T01:00:00"), + compactSegment.getInterval() + ); + Map expectedLongSumMetric = new HashMap<>(); + expectedLongSumMetric.put("name", "val"); + expectedLongSumMetric.put("type", "longSum"); + expectedLongSumMetric.put("fieldName", "val"); + Assert.assertEquals( + getDefaultCompactionState( + Granularities.HOUR, + Granularities.MINUTE, + ImmutableList.of(Intervals.of("2014-01-01T00:00:00/2014-01-01T01:00:00")), + DimensionsSpec.builder() + .setDimensions(Arrays.asList( + // check explicitly that time ordering is preserved + new AutoTypeColumnSchema("x", ColumnType.LONG), + new LongDimensionSchema("__time"), + new AutoTypeColumnSchema("ts", ColumnType.STRING), + new AutoTypeColumnSchema("dim", null), + new AutoTypeColumnSchema("y", ColumnType.LONG) + )) + .setForceSegmentSortByTime(false) + .build(), + expectedLongSumMetric + ), + compactSegment.getLastCompactionState() + ); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals( + new NumberedOverwriteShardSpec(32768, 0, 3, (short) 1, (short) 1), + compactSegment.getShardSpec() + ); + } else { + Assert.assertEquals(new NumberedShardSpec(0, 1), compactSegment.getShardSpec()); + } + + final File cacheDir = temporaryFolder.newFolder(); + final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(cacheDir); + + List rowsFromSegment = new ArrayList<>(); + final File segmentFile = segmentCacheManager.getSegmentFiles(compactSegment); + + final QueryableIndex queryableIndex = testUtils.getTestIndexIO().loadIndex(segmentFile); + final WindowedStorageAdapter adapter = new WindowedStorageAdapter( + new QueryableIndexStorageAdapter(queryableIndex), + compactSegment.getInterval() + ); + Assert.assertEquals( + ImmutableList.of( + OrderBy.ascending("x"), + OrderBy.ascending("__time"), + OrderBy.ascending("ts"), + OrderBy.ascending("dim"), + OrderBy.ascending("y") + ), + queryableIndex.getOrdering() + ); + + try (final CursorHolder cursorHolder = + adapter.getAdapter() + .makeCursorHolder(CursorBuildSpec.builder().setInterval(compactSegment.getInterval()).build())) { + final Cursor cursor = cursorHolder.asCursor(); + cursor.reset(); + final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); + Assert.assertEquals(ColumnType.STRING, factory.getColumnCapabilities("ts").toColumnType()); + Assert.assertEquals(ColumnType.STRING, factory.getColumnCapabilities("dim").toColumnType()); + Assert.assertEquals(ColumnType.LONG, factory.getColumnCapabilities("x").toColumnType()); + Assert.assertEquals(ColumnType.LONG, factory.getColumnCapabilities("y").toColumnType()); + while (!cursor.isDone()) { + final ColumnValueSelector selector1 = factory.makeColumnValueSelector("ts"); + final DimensionSelector selector2 = factory.makeDimensionSelector(new DefaultDimensionSpec("dim", "dim")); + final DimensionSelector selector3 = factory.makeDimensionSelector(new DefaultDimensionSpec("x", "x")); + final DimensionSelector selector4 = factory.makeDimensionSelector(new DefaultDimensionSpec("y", "y")); + final DimensionSelector selector5 = factory.makeDimensionSelector(new DefaultDimensionSpec("val", "val")); + + rowsFromSegment.add( + StringUtils.format( + "%s,%s,%s,%s,%s", + selector1.getObject(), + selector2.getObject(), + selector3.getObject(), + selector4.getObject(), + selector5.getObject() + ) + ); + + cursor.advance(); + } + } + + Assert.assertEquals( + ImmutableList.of( + "2014-01-01T00:00:10Z,a,10,100,1", + "2014-01-01T00:01:20Z,a,10,100,1", + "2014-01-01T00:00:10Z,b,20,110,2", + "2014-01-01T00:01:20Z,b,20,110,2", + "2014-01-01T00:00:10Z,c,30,120,3", + "2014-01-01T00:01:20Z,c,30,120,3" + ), + rowsFromSegment + ); + } + private Pair runIndexTask() throws Exception { return runIndexTask(null, null, false); 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 3c144929546a..84a1457ad914 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 @@ -1874,6 +1874,7 @@ public Metadata getMetadata() aggregatorFactories.toArray(new AggregatorFactory[0]), null, null, + null, null ); } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java index 553d5172b5fa..e529aa2d218d 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionsSpec.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -29,7 +30,9 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.apache.druid.guice.annotations.PublicApi; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParserUtils; +import org.apache.druid.segment.column.ColumnHolder; import javax.annotation.Nullable; import java.util.ArrayList; @@ -44,14 +47,34 @@ @PublicApi public class DimensionsSpec { + /** + * Parameter name for allowing any sort order. Also used as an MSQ context parameter, for some consistency between + * MSQ and native ingest configuration. + */ + public static final String PARAMETER_FORCE_TIME_SORT = "forceSegmentSortByTime"; + + /** + * Warning about non-time ordering to include in error messages when {@link #PARAMETER_FORCE_TIME_SORT} is + * not set. + */ + public static final String WARNING_NON_TIME_SORT_ORDER = StringUtils.format( + "Warning: support for segments not sorted by[%s] is experimental. Such segments are not readable by older " + + "version of Druid, and certain queries cannot run on them. See " + + "https://druid.apache.org/docs/latest/ingestion/partitioning#sorting for details before using this option.", + ColumnHolder.TIME_COLUMN_NAME + ); + + public static final boolean DEFAULT_FORCE_TIME_SORT = true; + private final List dimensions; private final Set dimensionExclusions; private final Map dimensionSchemaMap; private final boolean includeAllDimensions; + private final Boolean forceSegmentSortByTime; private final boolean useSchemaDiscovery; - public static final DimensionsSpec EMPTY = new DimensionsSpec(null, null, null, false, null); + public static final DimensionsSpec EMPTY = new DimensionsSpec(null, null, null, false, null, false); public static List getDefaultSchemas(List dimNames) { @@ -80,7 +103,7 @@ public static Builder builder() public DimensionsSpec(List dimensions) { - this(dimensions, null, null, false, null); + this(dimensions, null, null, false, null, null); } @JsonCreator @@ -89,7 +112,8 @@ private DimensionsSpec( @JsonProperty("dimensionExclusions") List dimensionExclusions, @Deprecated @JsonProperty("spatialDimensions") List spatialDimensions, @JsonProperty("includeAllDimensions") boolean includeAllDimensions, - @JsonProperty("useSchemaDiscovery") Boolean useSchemaDiscovery + @JsonProperty("useSchemaDiscovery") Boolean useSchemaDiscovery, + @JsonProperty(PARAMETER_FORCE_TIME_SORT) Boolean forceSegmentSortByTime ) { this.dimensions = dimensions == null @@ -120,6 +144,7 @@ private DimensionsSpec( this.includeAllDimensions = includeAllDimensions; this.useSchemaDiscovery = useSchemaDiscovery != null && useSchemaDiscovery; + this.forceSegmentSortByTime = forceSegmentSortByTime; } @JsonProperty @@ -146,6 +171,26 @@ public boolean useSchemaDiscovery() return useSchemaDiscovery; } + @JsonProperty(PARAMETER_FORCE_TIME_SORT) + @JsonInclude(JsonInclude.Include.NON_NULL) + public Boolean isForceSegmentSortByTimeConfigured() + { + return forceSegmentSortByTime; + } + + /** + * Returns {@link #isForceSegmentSortByTimeConfigured()} if nonnull, otherwise + * {@link #DEFAULT_FORCE_TIME_SORT}. + */ + public boolean isForceSegmentSortByTime() + { + if (forceSegmentSortByTime != null) { + return forceSegmentSortByTime; + } else { + return DEFAULT_FORCE_TIME_SORT; + } + } + @Deprecated @JsonIgnore public List getSpatialDimensions() @@ -191,9 +236,13 @@ public DimensionSchema getSchema(String dimension) return dimensionSchemaMap.get(dimension); } - public boolean hasCustomDimensions() + /** + * Whether this spec represents a set of fixed dimensions. Will be false if schema discovery is enabled, even if + * some dimensions are explicitly defined. + */ + public boolean hasFixedDimensions() { - return !(dimensions == null || dimensions.isEmpty()); + return dimensions != null && !dimensions.isEmpty() && !useSchemaDiscovery && !includeAllDimensions; } @PublicApi @@ -204,7 +253,8 @@ public DimensionsSpec withDimensions(List dims) ImmutableList.copyOf(dimensionExclusions), null, includeAllDimensions, - useSchemaDiscovery + useSchemaDiscovery, + forceSegmentSortByTime ); } @@ -215,7 +265,8 @@ public DimensionsSpec withDimensionExclusions(Set dimExs) ImmutableList.copyOf(Sets.union(dimensionExclusions, dimExs)), null, includeAllDimensions, - useSchemaDiscovery + useSchemaDiscovery, + forceSegmentSortByTime ); } @@ -227,7 +278,8 @@ public DimensionsSpec withSpatialDimensions(List spatial ImmutableList.copyOf(dimensionExclusions), spatials, includeAllDimensions, - useSchemaDiscovery + useSchemaDiscovery, + forceSegmentSortByTime ); } @@ -269,7 +321,9 @@ public boolean equals(Object o) return includeAllDimensions == that.includeAllDimensions && useSchemaDiscovery == that.useSchemaDiscovery && Objects.equals(dimensions, that.dimensions) - && Objects.equals(dimensionExclusions, that.dimensionExclusions); + && Objects.equals(dimensionExclusions, that.dimensionExclusions) + && Objects.equals(dimensionSchemaMap, that.dimensionSchemaMap) + && Objects.equals(forceSegmentSortByTime, that.forceSegmentSortByTime); } @Override @@ -278,7 +332,9 @@ public int hashCode() return Objects.hash( dimensions, dimensionExclusions, + dimensionSchemaMap, includeAllDimensions, + forceSegmentSortByTime, useSchemaDiscovery ); } @@ -291,6 +347,9 @@ public String toString() ", dimensionExclusions=" + dimensionExclusions + ", includeAllDimensions=" + includeAllDimensions + ", useSchemaDiscovery=" + useSchemaDiscovery + + (forceSegmentSortByTime != null + ? ", forceSegmentSortByTime=" + forceSegmentSortByTime + : "") + '}'; } @@ -301,6 +360,7 @@ public static final class Builder private List spatialDimensions; private boolean includeAllDimensions; private boolean useSchemaDiscovery; + private Boolean forceSegmentSortByTime; public Builder setDimensions(List dimensions) { @@ -339,6 +399,12 @@ public Builder useSchemaDiscovery(boolean useSchemaDiscovery) return this; } + public Builder setForceSegmentSortByTime(Boolean forceSegmentSortByTime) + { + this.forceSegmentSortByTime = forceSegmentSortByTime; + return this; + } + public DimensionsSpec build() { return new DimensionsSpec( @@ -346,7 +412,8 @@ public DimensionsSpec build() dimensionExclusions, spatialDimensions, includeAllDimensions, - useSchemaDiscovery + useSchemaDiscovery, + forceSegmentSortByTime ); } } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java index 79dcdb7ced3c..51ef56928d40 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java @@ -31,7 +31,6 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ListIndexed; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -87,18 +86,6 @@ public int getDimensionCardinality(String column) return DimensionDictionarySelector.CARDINALITY_UNKNOWN; } - @Override - public DateTime getMinTime() - { - return getInterval().getStart(); - } - - @Override - public DateTime getMaxTime() - { - return getInterval().getEnd().minus(1); - } - @Nullable @Override public Comparable getMinValue(String column) @@ -128,12 +115,6 @@ public int getNumRows() return frame.numRows(); } - @Override - public DateTime getMaxIngestedEventTime() - { - return getMaxTime(); - } - @Override @Nullable public Metadata getMetadata() diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java index 7c97530e37f1..ee1a01f17fe5 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java @@ -28,6 +28,7 @@ import org.apache.druid.frame.segment.row.RowFrameCursorHolderFactory; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.vector.VectorValueMatcher; @@ -36,10 +37,8 @@ import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.CursorHolderFactory; -import org.apache.druid.segment.Cursors; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; import org.apache.druid.segment.SimpleAscendingOffset; -import org.apache.druid.segment.SimpleDescendingOffset; import org.apache.druid.segment.SimpleSettableOffset; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.vector.FilteredVectorOffset; @@ -51,6 +50,7 @@ import org.apache.druid.utils.CloseableUtils; import javax.annotation.Nullable; +import java.util.Collections; import java.util.List; /** @@ -81,24 +81,19 @@ public ColumnarFrameCursorHolderFactory( public CursorHolder makeCursorHolder(CursorBuildSpec spec) { final Closer closer = Closer.create(); - // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary - final List ordering; - final boolean descending; - if (Cursors.preferDescendingTimeOrdering(spec)) { - ordering = Cursors.descendingTimeOrder(); - descending = true; - } else { - ordering = Cursors.ascendingTimeOrder(); - descending = false; - } + + // Frames are not self-describing as to their sort order, so we can't determine the sort order by looking at + // the Frame object. We could populate this with information from the relevant ClusterBy, but that's not available + // at this point in the code. It could be plumbed in at some point. For now, use an empty list. + final List ordering = Collections.emptyList(); + return new CursorHolder() { @Override public boolean canVectorize() { return (spec.getFilter() == null || spec.getFilter().canVectorizeMatcher(signature)) - && spec.getVirtualColumns().canVectorize(signature) - && !descending; + && spec.getVirtualColumns().canVectorize(signature); } @Override @@ -107,16 +102,13 @@ public Cursor asCursor() final FrameQueryableIndex index = new FrameQueryableIndex(frame, signature, columnReaders); final ColumnCache columnCache = new ColumnCache(index, closer); final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval()); - final boolean descendingTimeOrder = Cursors.preferDescendingTimeOrdering(spec); - final SimpleSettableOffset baseOffset = descendingTimeOrder - ? new SimpleDescendingOffset(frame.numRows()) - : new SimpleAscendingOffset(frame.numRows()); + final SimpleSettableOffset baseOffset = new SimpleAscendingOffset(frame.numRows()); final QueryableIndexColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( - spec.getVirtualColumns(), - descendingTimeOrder, - baseOffset, - columnCache + spec.getVirtualColumns(), + Order.NONE, + baseOffset, + columnCache ); final SimpleSettableOffset offset; diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java index 86bf7eb51263..d3a03ca83bbd 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java @@ -24,6 +24,7 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.read.columnar.FrameColumnReader; +import org.apache.druid.query.OrderBy; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.QueryableIndex; @@ -100,6 +101,12 @@ public Indexed getAvailableDimensions() return new ListIndexed<>(signature.getColumnNames()); } + @Override + public List getOrdering() + { + return Collections.emptyList(); + } + @Override public BitmapFactory getBitmapFactoryForDimensions() { diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java index 2d0d6567d5f1..f84eb62a4fa8 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java @@ -34,12 +34,11 @@ import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.CursorHolderFactory; -import org.apache.druid.segment.Cursors; import org.apache.druid.segment.SimpleAscendingOffset; -import org.apache.druid.segment.SimpleDescendingOffset; import org.apache.druid.segment.SimpleSettableOffset; import javax.annotation.Nullable; +import java.util.Collections; import java.util.List; /** @@ -69,16 +68,11 @@ public RowFrameCursorHolderFactory( @Override public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary - final List ordering; - final boolean descending; - if (Cursors.preferDescendingTimeOrdering(spec)) { - ordering = Cursors.descendingTimeOrder(); - descending = true; - } else { - ordering = Cursors.ascendingTimeOrder(); - descending = false; - } + // Frames are not self-describing as to their sort order, so we can't determine the sort order by looking at + // the Frame object. We could populate this with information from the relevant ClusterBy, but that's not available + // at this point in the code. It could be plumbed in at some point. For now, use an empty list. + final List ordering = Collections.emptyList(); + return new CursorHolder() { @Nullable @@ -87,10 +81,7 @@ public Cursor asCursor() { final Filter filterToUse = FrameCursorUtils.buildFilter(spec.getFilter(), spec.getInterval()); - final SimpleSettableOffset baseOffset = descending - ? new SimpleDescendingOffset(frame.numRows()) - : new SimpleAscendingOffset(frame.numRows()); - + final SimpleSettableOffset baseOffset = new SimpleAscendingOffset(frame.numRows()); final ColumnSelectorFactory columnSelectorFactory = spec.getVirtualColumns().wrap( diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java index 3c8c89b2c265..05150ea830fc 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java @@ -69,7 +69,6 @@ public static Granularity nullToAll(Granularity granularity) return granularity == null ? Granularities.ALL : granularity; } - /** * Decorates {@link CursorBuildSpec} with a grouping column and virtual column equivalent to the {@link Granularity} * for a {@link Query}, if that query has granularity other than {@link Granularities#ALL}. If the query has 'ALL' diff --git a/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java index b9d3418779a3..fa2defce7f03 100644 --- a/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java +++ b/processing/src/main/java/org/apache/druid/query/CursorGranularizer.java @@ -22,11 +22,13 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnHolder; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -45,25 +47,39 @@ public class CursorGranularizer { @Nullable public static CursorGranularizer create( - final StorageAdapter storageAdapter, final Cursor cursor, + @Nullable final TimeBoundaryInspector timeBoundaryInspector, + final Order timeOrder, final Granularity granularity, - final Interval queryInterval, - final boolean descending + final Interval queryInterval ) { - final DateTime minTime = storageAdapter.getMinTime(); - final DateTime maxTime = storageAdapter.getMaxTime(); + if (!Granularities.ALL.equals(granularity) && timeOrder == Order.NONE) { + throw DruidException + .forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build("Cannot use granularity[%s] on non-time-sorted data.", granularity); + } - final Interval storageAdapterInterval = new Interval(minTime, granularity.bucketEnd(maxTime)); - final Interval clippedQueryInterval = queryInterval.overlap(storageAdapterInterval); + final Interval clippedQueryInterval; + + if (timeBoundaryInspector != null) { + clippedQueryInterval = queryInterval.overlap( + new Interval( + timeBoundaryInspector.getMinTime(), + granularity.bucketEnd(timeBoundaryInspector.getMaxTime()) + ) + ); + } else { + clippedQueryInterval = queryInterval; + } if (clippedQueryInterval == null) { return null; } Iterable bucketIterable = granularity.getIterable(clippedQueryInterval); - if (descending) { + if (timeOrder == Order.DESCENDING) { bucketIterable = Lists.reverse(ImmutableList.copyOf(bucketIterable)); } final Interval firstBucket = granularity.bucket(clippedQueryInterval.getStart()); @@ -78,7 +94,7 @@ public static CursorGranularizer create( timeSelector = cursor.getColumnSelectorFactory().makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME); } - return new CursorGranularizer(cursor, bucketIterable, timeSelector, descending); + return new CursorGranularizer(cursor, bucketIterable, timeSelector, timeOrder == Order.DESCENDING); } private final Cursor cursor; diff --git a/processing/src/main/java/org/apache/druid/query/FluentQueryRunner.java b/processing/src/main/java/org/apache/druid/query/FluentQueryRunner.java index e8875761ad42..dcef6fa06514 100644 --- a/processing/src/main/java/org/apache/druid/query/FluentQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/FluentQueryRunner.java @@ -104,31 +104,4 @@ public FluentQueryRunner map(final Function, QueryRunner> m { return from(mapFn.apply(baseRunner)); } - - /** - * Sets the toString of the QueryRunner. This is used because QueryRunner objects are often used as parameters for - * tests and the toString() value of the QueryRunners are used for the name of the test. - * - * This method doesn't return a FluentQueryRunner because it breaks the fluency. - * - * @param toStringValue the value that the resulting QueryRunner should return from its toString method. - * @return a QueryRunner that will return toStringValue from its toString method - */ - public QueryRunner setToString(String toStringValue) - { - return new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) - { - return FluentQueryRunner.this.run(queryPlus, responseContext); - } - - @Override - public String toString() - { - return toStringValue; - } - }; - } } diff --git a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java index 31751fb13f16..c24f6f4f90af 100644 --- a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java @@ -33,8 +33,9 @@ import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.Result; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.segment.MaxIngestedEventTimeInspector; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; +import org.joda.time.Interval; import java.util.Iterator; @@ -79,11 +80,13 @@ public QueryToolChest, DataSourceMetadataQ private static class DataSourceMetadataQueryRunner implements QueryRunner> { - private final StorageAdapter adapter; + private final Interval segmentInterval; + private final MaxIngestedEventTimeInspector inspector; public DataSourceMetadataQueryRunner(Segment segment) { - this.adapter = segment.asStorageAdapter(); + this.segmentInterval = segment.asStorageAdapter().getInterval(); + this.inspector = segment.as(MaxIngestedEventTimeInspector.class); } @Override @@ -105,15 +108,9 @@ public Sequence> run( @Override public Iterator> make() { - if (adapter == null) { - throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." - ); - } - return legacyQuery.buildResult( - adapter.getInterval().getStart(), - adapter.getMaxIngestedEventTime() + segmentInterval.getStart(), + (inspector != null ? inspector.getMaxIngestedEventTime() : null) ).iterator(); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java index adf43e8cc16c..dcde45aeaae9 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -32,6 +32,9 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; + +import javax.annotation.Nullable; /** * @@ -86,11 +89,14 @@ public QueryToolChest getToolchest() private static class GroupByQueryRunner implements QueryRunner { private final StorageAdapter adapter; + @Nullable + private final TimeBoundaryInspector timeBoundaryInspector; private final GroupingEngine groupingEngine; public GroupByQueryRunner(Segment segment, final GroupingEngine groupingEngine) { this.adapter = segment.asStorageAdapter(); + this.timeBoundaryInspector = segment.as(TimeBoundaryInspector.class); this.groupingEngine = groupingEngine; } @@ -102,7 +108,12 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r throw new ISE("Got a [%s] which isn't a %s", query.getClass(), GroupByQuery.class); } - return groupingEngine.process((GroupByQuery) query, adapter, (GroupByQueryMetrics) queryPlus.getQueryMetrics()); + return groupingEngine.process( + (GroupByQuery) query, + adapter, + timeBoundaryInspector, + (GroupByQueryMetrics) queryPlus.getQueryMetrics() + ); } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index 75d7ad50d8af..12b186afc72f 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -80,6 +80,7 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; @@ -424,7 +425,7 @@ public Sequence mergeResults( /** * Merges a variety of single-segment query runners into a combined runner. Used by * {@link GroupByQueryRunnerFactory#mergeRunners(QueryProcessingPool, Iterable)}. In - * that sense, it is intended to go along with {@link #process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)} (the runners created + * that sense, it is intended to go along with {@link #process} (the runners created * by that method will be fed into this method). * * This is primarily called on the data servers, to merge the results from processing on the segments. This method can @@ -464,14 +465,16 @@ public QueryRunner mergeRunners( * * This method is only called on data servers, like Historicals (not the Broker). * - * @param query the groupBy query - * @param storageAdapter storage adatper for the segment in question + * @param query the groupBy query + * @param storageAdapter storage adatper for the segment in question + * @param timeBoundaryInspector time boundary inspector for the segment in question * * @return result sequence for the storage adapter */ public Sequence process( GroupByQuery query, StorageAdapter storageAdapter, + @Nullable TimeBoundaryInspector timeBoundaryInspector, @Nullable GroupByQueryMetrics groupByQueryMetrics ) { @@ -515,6 +518,7 @@ public Sequence process( result = VectorGroupByEngine.process( query, storageAdapter, + timeBoundaryInspector, cursorHolder, bufferHolder.get(), fudgeTimestamp, @@ -526,6 +530,7 @@ public Sequence process( result = GroupByQueryEngine.process( query, storageAdapter, + timeBoundaryInspector, cursorHolder, buildSpec, bufferHolder.get(), diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java index 9f2507724621..c6a32d364163 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java @@ -39,7 +39,6 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; -import org.apache.druid.query.groupby.GroupByQueryMetrics; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorPlus; @@ -55,6 +54,7 @@ import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; @@ -77,8 +77,7 @@ *

* This code runs on anything that processes {@link StorageAdapter} directly, typically data servers like Historicals. *

- * Used for non-vectorized processing by - * {@link GroupingEngine#process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)}. + * Used for non-vectorized processing by {@link GroupingEngine#process}. * * This code runs on data servers, like Historicals and Peons, and also Brokers, if they operate on local datasources like * inlined data wherein the broker needs to process some portion of data like the data server @@ -97,6 +96,7 @@ private GroupByQueryEngine() public static Sequence process( final GroupByQuery query, final StorageAdapter storageAdapter, + @Nullable final TimeBoundaryInspector timeBoundaryInspector, final CursorHolder cursorHolder, final CursorBuildSpec buildSpec, final ByteBuffer processingBuffer, @@ -110,11 +110,11 @@ public static Sequence process( return Sequences.empty(); } final CursorGranularizer granularizer = CursorGranularizer.create( - storageAdapter, cursor, + timeBoundaryInspector, + cursorHolder.getTimeOrder(), query.getGranularity(), - buildSpec.getInterval(), - false + buildSpec.getInterval() ); if (granularizer == null) { return Sequences.empty(); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java index 4d0a30547c26..c6f52e0bd218 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java @@ -27,12 +27,12 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.Order; import org.apache.druid.query.aggregation.AggregatorAdapters; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; -import org.apache.druid.query.groupby.GroupByQueryMetrics; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.AggregateResult; @@ -46,6 +46,7 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorCursor; @@ -65,8 +66,7 @@ * Contains logic to process a groupBy query on a single {@link StorageAdapter} in a vectorized manner. * This code runs on anything that processes {@link StorageAdapter} directly, typically data servers like Historicals. *

- * Used for vectorized processing by - * {@link GroupingEngine#process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)}. + * Used for vectorized processing by {@link GroupingEngine#process}. * * @see org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngine for non-vectorized version of this logic */ @@ -80,6 +80,7 @@ private VectorGroupByEngine() public static Sequence process( final GroupByQuery query, final StorageAdapter storageAdapter, + @Nullable TimeBoundaryInspector timeBoundaryInspector, final CursorHolder cursorHolder, final ByteBuffer processingBuffer, @Nullable final DateTime fudgeTimestamp, @@ -145,7 +146,9 @@ public void close() config, processingConfig, storageAdapter, + timeBoundaryInspector, cursor, + cursorHolder.getTimeOrder(), interval, dimensions, processingBuffer, @@ -216,7 +219,7 @@ static class VectorGroupByEngineIterator implements CloseableIterator private final VectorGrouper vectorGrouper; @Nullable - private final VectorCursorGranularizer granulizer; + private final VectorCursorGranularizer granularizer; // Granularity-bucket iterator and current bucket. private final Iterator bucketIterator; @@ -239,7 +242,9 @@ static class VectorGroupByEngineIterator implements CloseableIterator final GroupByQueryConfig querySpecificConfig, final DruidProcessingConfig processingConfig, final StorageAdapter storageAdapter, + @Nullable TimeBoundaryInspector timeBoundaryInspector, final VectorCursor cursor, + final Order timeOrder, final Interval queryInterval, final List selectors, final ByteBuffer processingBuffer, @@ -257,10 +262,16 @@ static class VectorGroupByEngineIterator implements CloseableIterator this.keySize = selectors.stream().mapToInt(GroupByVectorColumnSelector::getGroupingKeySize).sum(); this.keySpace = WritableMemory.allocate(keySize * cursor.getMaxVectorSize()); this.vectorGrouper = makeGrouper(); - this.granulizer = VectorCursorGranularizer.create(storageAdapter, cursor, query.getGranularity(), queryInterval); + this.granularizer = VectorCursorGranularizer.create( + cursor, + timeBoundaryInspector, + timeOrder, + query.getGranularity(), + queryInterval + ); - if (granulizer != null) { - this.bucketIterator = granulizer.getBucketIterable().iterator(); + if (granularizer != null) { + this.bucketIterator = granularizer.getBucketIterable().iterator(); } else { this.bucketIterator = Collections.emptyIterator(); } @@ -368,20 +379,20 @@ private CloseableGrouperIterator initNewDelegate() final int startOffset; if (partiallyAggregatedRows < 0) { - granulizer.setCurrentOffsets(bucketInterval); - startOffset = granulizer.getStartOffset(); + granularizer.setCurrentOffsets(bucketInterval); + startOffset = granularizer.getStartOffset(); } else { - startOffset = granulizer.getStartOffset() + partiallyAggregatedRows; + startOffset = granularizer.getStartOffset() + partiallyAggregatedRows; } - if (granulizer.getEndOffset() > startOffset) { + if (granularizer.getEndOffset() > startOffset) { // Write keys to the keySpace. int keyOffset = 0; for (final GroupByVectorColumnSelector selector : selectors) { // Update selectorInternalFootprint now, but check it later. (We reset on the first vector that causes us // to go past the limit.) selectorInternalFootprint += - selector.writeKeys(keySpace, keySize, keyOffset, startOffset, granulizer.getEndOffset()); + selector.writeKeys(keySpace, keySize, keyOffset, startOffset, granularizer.getEndOffset()); keyOffset += selector.getGroupingKeySize(); } @@ -390,7 +401,7 @@ private CloseableGrouperIterator initNewDelegate() final AggregateResult result = vectorGrouper.aggregateVector( keySpace, startOffset, - granulizer.getEndOffset() + granularizer.getEndOffset() ); if (result.isOk()) { @@ -408,7 +419,7 @@ private CloseableGrouperIterator initNewDelegate() if (partiallyAggregatedRows >= 0) { break; - } else if (!granulizer.advanceCursorWithinBucket()) { + } else if (!granularizer.advanceCursorWithinBucket()) { // Advance bucketInterval. bucketInterval = bucketIterator.hasNext() ? bucketIterator.next() : null; break; 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 893480f54532..ca2c2725c6f6 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,6 +23,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; @@ -119,8 +120,20 @@ public Sequence process( responseContext.addRowScanCount(0); final long limit = calculateRemainingScanRowsLimit(query, responseContext); final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, queryMetrics)); - if (Order.NONE != query.getTimeOrder()) { - Cursors.requireTimeOrdering(cursorHolder, query.getTimeOrder()); + if (Order.NONE != query.getTimeOrder() + && Cursors.getTimeOrdering(cursorHolder.getOrdering()) != query.getTimeOrder()) { + final String failureReason = StringUtils.format( + "Cannot order by[%s] with direction[%s] on cursor with order[%s].", + ColumnHolder.TIME_COLUMN_NAME, + query.getTimeOrder(), + cursorHolder.getOrdering() + ); + + cursorHolder.close(); + + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build("%s", failureReason); } return new BaseSequence<>( new BaseSequence.IteratorMaker>() diff --git a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java index c522e70c90b1..d5af263a1c7a 100644 --- a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java @@ -27,6 +27,7 @@ import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.DefaultBitmapResultFactory; +import org.apache.druid.query.Order; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.extraction.IdentityExtractionFn; @@ -34,6 +35,7 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.search.CursorOnlyStrategy.CursorBasedExecutor; import org.apache.druid.segment.ColumnSelectorColumnIndexSelector; +import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DeprecatedQueryableIndexColumnSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; @@ -98,7 +100,8 @@ public List getExecutionPlan(SearchQuery query, Segment seg // Note: if some filters support bitmap indexes but others are not, the current implementation always employs // the cursor-based plan. This can be more optimized. One possible optimization is generating a bitmap index // from the non-bitmap-support filter, and then use it to compute the filtered result by intersecting bitmaps. - if (filter == null || filter.getBitmapColumnIndex(selector) != null) { + if ((filter == null || filter.getBitmapColumnIndex(selector) != null) + && Cursors.getTimeOrdering(index.getOrdering()) == Order.ASCENDING) { final ImmutableBitmap timeFilteredBitmap = makeTimeFilteredBitmap( index, segment, diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQuery.java index 17db30455e0f..47412f7d7d50 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQuery.java @@ -200,6 +200,16 @@ boolean isMaxTime() return bound.equalsIgnoreCase(MAX_TIME); } + boolean needsMinTime() + { + return !isMaxTime(); + } + + boolean needsMaxTime() + { + return !isMinTime(); + } + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 38b5ff6eb930..ec9e27c6f38e 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -19,8 +19,8 @@ package org.apache.druid.query.timeboundary; -import com.google.common.base.Function; import com.google.inject.Inject; +import it.unimi.dsi.fastutil.Pair; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -36,17 +36,20 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.Result; -import org.apache.druid.query.TableDataSource; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.CursorHolderFactory; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.vector.VectorCursor; +import org.apache.druid.segment.vector.VectorValueSelector; import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -93,47 +96,13 @@ public QueryToolChest, TimeBoundaryQuery> getToo private static class TimeBoundaryQueryRunner implements QueryRunner> { private final StorageAdapter adapter; - private final Function> skipToFirstMatching; + @Nullable + private final TimeBoundaryInspector timeBoundaryInspector; public TimeBoundaryQueryRunner(Segment segment) { this.adapter = segment.asStorageAdapter(); - this.skipToFirstMatching = new Function>() - { - @SuppressWarnings("ArgumentParameterSwap") - @Override - public Result apply(Cursor cursor) - { - if (cursor.isDone()) { - return null; - } - final BaseLongColumnValueSelector timestampColumnSelector = - cursor.getColumnSelectorFactory().makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME); - final DateTime timestamp = DateTimes.utc(timestampColumnSelector.getLong()); - return new Result<>(adapter.getInterval().getStart(), timestamp); - } - }; - } - - @Nullable - private DateTime getTimeBoundary(StorageAdapter adapter, TimeBoundaryQuery legacyQuery, boolean descending) - { - final CursorBuildSpec.CursorBuildSpecBuilder bob = CursorBuildSpec.builder(makeCursorBuildSpec(legacyQuery)); - if (descending) { - bob.setPreferredOrdering(Cursors.descendingTimeOrder()); - } else { - bob.setPreferredOrdering(Cursors.ascendingTimeOrder()); - } - - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(bob.build())) { - Cursors.requireTimeOrdering(cursorHolder, descending ? Order.DESCENDING : Order.ASCENDING); - final Cursor cursor = cursorHolder.asCursor(); - if (cursor == null) { - return null; - } - final Result result = skipToFirstMatching.apply(cursor); - return result == null ? null : result.getValue(); - } + this.timeBoundaryInspector = segment.as(TimeBoundaryInspector.class); } @Override @@ -164,24 +133,18 @@ public Iterator> make() DateTime minTime = null; DateTime maxTime = null; - if (canUseAdapterMinMaxTime(query, adapter)) { - if (!query.isMaxTime()) { - minTime = adapter.getMinTime(); + if (canUseTimeBoundaryInspector(query, timeBoundaryInspector)) { + if (query.needsMinTime()) { + minTime = timeBoundaryInspector.getMinTime(); } - if (!query.isMinTime()) { - maxTime = adapter.getMaxTime(); + if (query.needsMaxTime()) { + maxTime = timeBoundaryInspector.getMaxTime(); } } else { - if (!query.isMaxTime()) { - minTime = getTimeBoundary(adapter, query, false); - } - - if (!query.isMinTime()) { - if (query.isMaxTime() || minTime != null) { - maxTime = getTimeBoundary(adapter, query, true); - } - } + final Pair timeBoundary = getTimeBoundary(query, adapter); + minTime = timeBoundary.left(); + maxTime = timeBoundary.right(); } return query.buildResult( @@ -202,21 +165,22 @@ public void cleanup(Iterator> toClean) } /** - * Whether a particular {@link TimeBoundaryQuery} can use {@link StorageAdapter#getMinTime()} and/or - * {@link StorageAdapter#getMaxTime()}. If false, must use {@link StorageAdapter#makeCursorHolder(CursorBuildSpec)}. + * Whether a particular {@link TimeBoundaryQuery} can use {@link TimeBoundaryInspector#getMinTime()} and/or + * {@link TimeBoundaryInspector#getMaxTime()}. + * + * If false, must use {@link StorageAdapter#makeCursorHolder(CursorBuildSpec)}. */ - private static boolean canUseAdapterMinMaxTime(final TimeBoundaryQuery query, final StorageAdapter adapter) + private static boolean canUseTimeBoundaryInspector( + final TimeBoundaryQuery query, + @Nullable final TimeBoundaryInspector timeBoundaryInspector + ) { - if (query.getFilter() != null) { - // We have to check which rows actually match the filter. + if (timeBoundaryInspector == null || !timeBoundaryInspector.isMinMaxExact()) { return false; } - if (!(query.getDataSource() instanceof TableDataSource)) { - // In general, minTime / maxTime are only guaranteed to match data for regular tables. - // - // One example: an INNER JOIN can act as a filter and remove some rows. Another example: RowBasedStorageAdapter - // (used by e.g. inline data) uses nominal interval, not actual data, for minTime / maxTime. + if (query.getFilter() != null) { + // We have to check which rows actually match the filter. return false; } @@ -225,8 +189,8 @@ private static boolean canUseAdapterMinMaxTime(final TimeBoundaryQuery query, fi xs -> new IAE("Should only have one interval, got[%s]", xs) ); - if (!queryInterval.contains(adapter.getInterval())) { - // Query interval does not contain adapter interval. Need to create a cursor to see the first + if (!queryInterval.contains(timeBoundaryInspector.getMinMaxInterval())) { + // Query interval does not contain segment interval. Need to create a cursor to see the first // timestamp within the query interval. return false; } @@ -244,4 +208,169 @@ public static CursorBuildSpec makeCursorBuildSpec(TimeBoundaryQuery query) .setQueryContext(query.context()) .build(); } + + private static Pair getTimeBoundary( + final TimeBoundaryQuery query, + final CursorHolderFactory cursorHolderFactory + ) + { + DateTime minTime = null, maxTime = null; + + if (query.needsMinTime()) { + final CursorBuildSpec cursorSpec = + CursorBuildSpec.builder(makeCursorBuildSpec(query)) + .setPreferredOrdering(Cursors.ascendingTimeOrder()) + .build(); + + try (final CursorHolder cursorHolder = cursorHolderFactory.makeCursorHolder(cursorSpec)) { + if (cursorHolder.getTimeOrder() == Order.ASCENDING) { + // Time-ordered cursor, use the first timestamp. + minTime = getFirstTimestamp(query, cursorHolder); + } else { + // Non-time-ordered cursor. Walk and find both minTime, maxTime. Return immediately. + return getTimeBoundaryFullScan(query, cursorHolder); + } + } + } + + if (query.needsMaxTime()) { + final CursorBuildSpec cursorSpec = + CursorBuildSpec.builder(makeCursorBuildSpec(query)) + .setPreferredOrdering(Cursors.descendingTimeOrder()) + .build(); + + try (final CursorHolder cursorHolder = cursorHolderFactory.makeCursorHolder(cursorSpec)) { + if (cursorHolder.getTimeOrder() == Order.DESCENDING) { + // Time-ordered cursor, use the first timestamp. + maxTime = getFirstTimestamp(query, cursorHolder); + } else { + // Non-time-ordered cursor. Walk and find both minTime, maxTime. Return immediately. + return getTimeBoundaryFullScan(query, cursorHolder); + } + } + } + + return Pair.of(minTime, maxTime); + } + + /** + * Retrieve the first timestamp from {@link CursorHolder}. + */ + @Nullable + private static DateTime getFirstTimestamp( + final TimeBoundaryQuery query, + final CursorHolder cursorHolder + ) + { + if (query.context().getVectorize().shouldVectorize(cursorHolder.canVectorize())) { + return getFirstTimestampVectorized(cursorHolder.asVectorCursor()); + } else { + return getFirstTimestampNonVectorized(cursorHolder.asCursor()); + } + } + + /** + * Retrieve the first timestamp from {@link CursorHolder#asVectorCursor()}. + */ + @Nullable + private static DateTime getFirstTimestampVectorized(@Nullable final VectorCursor cursor) + { + if (cursor == null || cursor.isDone()) { + return null; + } + final VectorValueSelector timestampSelector = + cursor.getColumnSelectorFactory().makeValueSelector(ColumnHolder.TIME_COLUMN_NAME); + final long[] timeVector = timestampSelector.getLongVector(); + return DateTimes.utc(timeVector[0]); + } + + /** + * Retrieve the first timestamp from {@link CursorHolder#asCursor()}. + */ + @Nullable + private static DateTime getFirstTimestampNonVectorized(@Nullable final Cursor cursor) + { + if (cursor == null || cursor.isDone()) { + return null; + } + final BaseLongColumnValueSelector timestampSelector = + cursor.getColumnSelectorFactory().makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME); + return DateTimes.utc(timestampSelector.getLong()); + } + + /** + * Retrieve min/max timestamps from {@link CursorHolder} using a full scan. + */ + private static Pair getTimeBoundaryFullScan( + final TimeBoundaryQuery query, + final CursorHolder cursorHolder + ) + { + if (query.context().getVectorize().shouldVectorize(cursorHolder.canVectorize())) { + return getTimeBoundaryFullScanVectorized(query, cursorHolder.asVectorCursor()); + } else { + return getTimeBoundaryFullScanNonVectorized(query, cursorHolder.asCursor()); + } + } + + /** + * Retrieve min/max timestamps from {@link CursorHolder#asVectorCursor()} using a full scan. + */ + private static Pair getTimeBoundaryFullScanVectorized( + final TimeBoundaryQuery query, + @Nullable final VectorCursor cursor + ) + { + if (cursor == null || cursor.isDone()) { + return Pair.of(null, null); + } + + final VectorValueSelector timeSelector = + cursor.getColumnSelectorFactory().makeValueSelector(ColumnHolder.TIME_COLUMN_NAME); + + long minTime = Long.MAX_VALUE, maxTime = Long.MIN_VALUE; + while (!cursor.isDone()) { + final long[] timeVector = timeSelector.getLongVector(); + for (int i = 0; i < cursor.getCurrentVectorSize(); i++) { + final long timestamp = timeVector[i]; + minTime = Math.min(minTime, timestamp); + maxTime = Math.max(maxTime, timestamp); + } + cursor.advance(); + } + + return Pair.of( + query.needsMinTime() ? DateTimes.utc(minTime) : null, + query.needsMaxTime() ? DateTimes.utc(maxTime) : null + ); + } + + /** + * Retrieve min/max timestamps from {@link CursorHolder#asCursor()} using a full scan. + */ + private static Pair getTimeBoundaryFullScanNonVectorized( + final TimeBoundaryQuery query, + @Nullable final Cursor cursor + ) + { + if (cursor == null || cursor.isDone()) { + return Pair.of(null, null); + } + + final BaseLongColumnValueSelector timeSelector = + cursor.getColumnSelectorFactory().makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME); + + long minTime = Long.MAX_VALUE, maxTime = Long.MIN_VALUE; + while (!cursor.isDone()) { + final long timestamp = timeSelector.getLong(); + minTime = Math.min(minTime, timestamp); + maxTime = Math.max(maxTime, timestamp); + cursor.advance(); + } + + return Pair.of( + query.needsMinTime() ? DateTimes.utc(minTime) : null, + query.needsMaxTime() ? DateTimes.utc(maxTime) : null + ); + } } diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index e23f8e19e6df..95cba2e647f8 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -33,7 +33,6 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.CursorGranularizer; -import org.apache.druid.query.Order; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.Aggregator; @@ -47,6 +46,7 @@ import org.apache.druid.segment.Cursors; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorCursor; @@ -88,6 +88,7 @@ public TimeseriesQueryEngine( public Sequence> process( final TimeseriesQuery query, final StorageAdapter adapter, + @Nullable TimeBoundaryInspector timeBoundaryInspector, @Nullable final TimeseriesQueryMetrics timeseriesQueryMetrics ) { @@ -100,16 +101,14 @@ public Sequence> process( final Interval interval = Iterables.getOnlyElement(query.getIntervals()); final Granularity gran = query.getGranularity(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics)); - Cursors.requireTimeOrdering(cursorHolder, query.isDescending() ? Order.DESCENDING : Order.ASCENDING); try { final Sequence> result; if (query.context().getVectorize().shouldVectorize(cursorHolder.canVectorize())) { - result = processVectorized(query, adapter, cursorHolder, interval, gran); + result = processVectorized(query, cursorHolder, timeBoundaryInspector, interval, gran); } else { - result = processNonVectorized(query, adapter, cursorHolder, interval, gran); + result = processNonVectorized(query, cursorHolder, timeBoundaryInspector, interval, gran); } final int limit = query.getLimit(); @@ -127,8 +126,8 @@ public Sequence> process( private Sequence> processVectorized( final TimeseriesQuery query, - final StorageAdapter adapter, final CursorHolder cursorHolder, + @Nullable final TimeBoundaryInspector timeBoundaryInspector, final Interval queryInterval, final Granularity gran ) @@ -145,8 +144,9 @@ private Sequence> processVectorized( final Closer closer = Closer.create(); try { final VectorCursorGranularizer granularizer = VectorCursorGranularizer.create( - adapter, cursor, + timeBoundaryInspector, + cursorHolder.getTimeOrder(), gran, queryInterval ); @@ -243,8 +243,8 @@ private Sequence> processVectorized( private Sequence> processNonVectorized( final TimeseriesQuery query, - final StorageAdapter adapter, final CursorHolder cursorHolder, + @Nullable TimeBoundaryInspector timeBoundaryInspector, final Interval queryInterval, final Granularity gran ) @@ -256,11 +256,11 @@ private Sequence> processNonVectorized( return Sequences.empty(); } final CursorGranularizer granularizer = CursorGranularizer.create( - adapter, cursor, + timeBoundaryInspector, + cursorHolder.getTimeOrder(), gran, - queryInterval, - query.isDescending() + queryInterval ); if (granularizer == null) { return Sequences.empty(); diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java index fe3d420e5662..788e72015e29 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -34,6 +34,9 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; + +import javax.annotation.Nullable; /** */ @@ -59,7 +62,7 @@ public TimeseriesQueryRunnerFactory( @Override public QueryRunner> createRunner(final Segment segment) { - return new TimeseriesQueryRunner(engine, segment.asStorageAdapter()); + return new TimeseriesQueryRunner(engine, segment.asStorageAdapter(), segment.as(TimeBoundaryInspector.class)); } @Override @@ -81,11 +84,18 @@ private static class TimeseriesQueryRunner implements QueryRunner> run( throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TimeseriesQuery.class); } - return engine.process((TimeseriesQuery) input, adapter, (TimeseriesQueryMetrics) queryPlus.getQueryMetrics()); + return engine.process( + (TimeseriesQuery) input, + adapter, + timeBoundaryInspector, + (TimeseriesQueryMetrics) queryPlus.getQueryMetrics() + ); } } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index ac130d3fa860..c6561faeee3b 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -35,6 +35,7 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.SegmentMissingException; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.Types; @@ -66,7 +67,8 @@ public TopNQueryEngine(NonBlockingPool bufferPool) public Sequence> query( final TopNQuery query, final StorageAdapter adapter, - final @Nullable TopNQueryMetrics queryMetrics + @Nullable final TimeBoundaryInspector timeBoundaryInspector, + @Nullable final TopNQueryMetrics queryMetrics ) { if (adapter == null) { @@ -84,11 +86,11 @@ public Sequence> query( return Sequences.withBaggage(Sequences.empty(), cursorHolder); } final CursorGranularizer granularizer = CursorGranularizer.create( - adapter, cursor, + timeBoundaryInspector, + cursorHolder.getTimeOrder(), query.getGranularity(), - buildSpec.getInterval(), - false + buildSpec.getInterval() ); if (granularizer == null) { return Sequences.withBaggage(Sequences.empty(), cursorHolder); diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java index 821d37e3a199..a81d9258c866 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java @@ -34,10 +34,12 @@ import org.apache.druid.query.Result; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.Segment; +import org.apache.druid.segment.TimeBoundaryInspector; import java.nio.ByteBuffer; /** + * */ public class TopNQueryRunnerFactory implements QueryRunnerFactory, TopNQuery> { @@ -74,7 +76,12 @@ public Sequence> run( } TopNQuery query = (TopNQuery) input.getQuery(); - return queryEngine.query(query, segment.asStorageAdapter(), (TopNQueryMetrics) input.getQueryMetrics()); + return queryEngine.query( + query, + segment.asStorageAdapter(), + segment.as(TimeBoundaryInspector.class), + (TopNQueryMetrics) input.getQueryMetrics() + ); } }; diff --git a/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java index d899e1f08884..f262a4ea4864 100644 --- a/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java +++ b/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java @@ -20,12 +20,14 @@ package org.apache.druid.query.vector; import com.google.common.collect.Iterables; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.query.Order; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.segment.vector.VectorValueSelector; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -42,17 +44,32 @@ public class VectorCursorGranularizer { @Nullable public static VectorCursorGranularizer create( - final StorageAdapter storageAdapter, final VectorCursor cursor, + @Nullable final TimeBoundaryInspector timeBoundaryInspector, + final Order timeOrder, final Granularity granularity, final Interval queryInterval ) { - final DateTime minTime = storageAdapter.getMinTime(); - final DateTime maxTime = storageAdapter.getMaxTime(); + if (!Granularities.ALL.equals(granularity) && timeOrder == Order.NONE) { + throw DruidException + .forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build("Cannot use granularity[%s] on non-time-sorted data.", granularity); + } + + final Interval clippedQueryInterval; - final Interval storageAdapterInterval = new Interval(minTime, granularity.bucketEnd(maxTime)); - final Interval clippedQueryInterval = queryInterval.overlap(storageAdapterInterval); + if (timeBoundaryInspector != null) { + clippedQueryInterval = queryInterval.overlap( + new Interval( + timeBoundaryInspector.getMinTime(), + granularity.bucketEnd(timeBoundaryInspector.getMaxTime()) + ) + ); + } else { + clippedQueryInterval = queryInterval; + } if (clippedQueryInterval == null) { return null; diff --git a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java index f6cbbdeffe8d..a45c4b2d3e00 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java @@ -19,6 +19,7 @@ package org.apache.druid.segment; +import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.OrderBy; import org.apache.druid.query.QueryContext; @@ -28,6 +29,7 @@ import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.Collections; import java.util.List; public class CursorBuildSpec @@ -52,7 +54,6 @@ public static CursorBuildSpecBuilder builder(CursorBuildSpec spec) private final VirtualColumns virtualColumns; @Nullable private final List aggregators; - @Nullable private final List orderByColumns; private final QueryContext queryContext; @@ -66,18 +67,18 @@ public CursorBuildSpec( @Nullable List groupingColumns, VirtualColumns virtualColumns, @Nullable List aggregators, - @Nullable List preferredOrdering, + List preferredOrdering, QueryContext queryContext, @Nullable QueryMetrics queryMetrics ) { this.filter = filter; - this.interval = interval; + this.interval = Preconditions.checkNotNull(interval, "interval"); this.groupingColumns = groupingColumns; - this.virtualColumns = virtualColumns; + this.virtualColumns = Preconditions.checkNotNull(virtualColumns, "virtualColumns"); this.aggregators = aggregators; - this.orderByColumns = preferredOrdering; - this.queryContext = queryContext; + this.orderByColumns = Preconditions.checkNotNull(preferredOrdering, "preferredOrdering"); + this.queryContext = Preconditions.checkNotNull(queryContext, "queryContext"); this.queryMetrics = queryMetrics; } @@ -139,7 +140,6 @@ public List getAggregators() *

* If not specified, the cursor will advance in the native order of the underlying data. */ - @Nullable public List getPreferredOrdering() { return orderByColumns; @@ -177,8 +177,7 @@ public static class CursorBuildSpecBuilder private VirtualColumns virtualColumns = VirtualColumns.EMPTY; @Nullable private List aggregators; - @Nullable - private List preferredOrdering; + private List preferredOrdering = Collections.emptyList(); private QueryContext queryContext = QueryContext.empty(); @Nullable @@ -251,9 +250,9 @@ public CursorBuildSpecBuilder setAggregators(@Nullable List a /** * @see CursorBuildSpec#getPreferredOrdering() */ - public CursorBuildSpecBuilder setPreferredOrdering(@Nullable List orderBy) + public CursorBuildSpecBuilder setPreferredOrdering(List preferredOrdering) { - this.preferredOrdering = orderBy; + this.preferredOrdering = preferredOrdering; return this; } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java index d0384e265b34..a70fd8757e13 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java @@ -20,7 +20,9 @@ package org.apache.druid.segment; import org.apache.druid.java.util.common.UOE; +import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.vector.VectorCursor; import javax.annotation.Nullable; @@ -42,7 +44,10 @@ public interface CursorHolder extends Closeable @Nullable default VectorCursor asVectorCursor() { - throw new UOE("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor' on %s.", this.getClass().getName()); + throw new UOE( + "Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor' on %s.", + this.getClass().getName() + ); } /** @@ -55,13 +60,30 @@ default boolean canVectorize() /** * Returns cursor ordering, which may or may not match {@link CursorBuildSpec#getPreferredOrdering()}. If returns - * null then the cursor has no defined ordering + * an empty list then the cursor has no defined ordering. + * + * Cursors associated with this holder return rows in this ordering, using the natural comparator for the column type. + * Includes {@link ColumnHolder#TIME_COLUMN_NAME} if appropriate. */ default List getOrdering() { return Collections.emptyList(); } + /** + * If {@link #getOrdering()} starts with {@link ColumnHolder#TIME_COLUMN_NAME}, returns the time ordering; otherwise + * returns {@link Order#NONE}. + */ + default Order getTimeOrder() + { + final List ordering = getOrdering(); + if (!ordering.isEmpty() && ColumnHolder.TIME_COLUMN_NAME.equals(ordering.get(0).getColumnName())) { + return ordering.get(0).getOrder(); + } else { + return Order.NONE; + } + } + /** * Release any resources acquired by cursors. */ diff --git a/processing/src/main/java/org/apache/druid/segment/Cursors.java b/processing/src/main/java/org/apache/druid/segment/Cursors.java index 814681ab8b47..36b37ce31469 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursors.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursors.java @@ -19,8 +19,6 @@ package org.apache.druid.segment; -import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; import org.apache.druid.segment.column.ColumnHolder; @@ -54,34 +52,15 @@ public static boolean preferDescendingTimeOrdering(CursorBuildSpec buildSpec) } /** - * Check if the first {@link OrderBy} of {@link CursorHolder#getOrdering()} is {@link ColumnHolder#TIME_COLUMN_NAME} - * in the specified {@link Order} + * Return the {@link Order} of the {@link ColumnHolder#TIME_COLUMN_NAME}, based on a + * {@link CursorHolder#getOrdering()} or {@link Metadata#getOrdering()}. */ - public static boolean isTimeOrdered(CursorHolder holder, Order order) + public static Order getTimeOrdering(final List ordering) { - final List ordering = holder.getOrdering(); - if (ordering.isEmpty()) { - return false; - } - final OrderBy orderBy = ordering.get(0); - return ColumnHolder.TIME_COLUMN_NAME.equals(orderBy.getColumnName()) && orderBy.getOrder() == order; - } - - /** - * Require the first {@link OrderBy} of {@link CursorHolder#getOrdering()} is {@link ColumnHolder#TIME_COLUMN_NAME}. - * Throws {@link DruidException} if the order does - */ - public static void requireTimeOrdering(CursorHolder holder, Order order) - { - if (!isTimeOrdered(holder, order)) { - final String failureReason = StringUtils.format( - "Cursor must be ordered by [%s] with direction [%s] but was [%s] instead.", - ColumnHolder.TIME_COLUMN_NAME, - order, - holder.getOrdering() - ); - holder.close(); - throw DruidException.defensive(failureReason); + if (!ordering.isEmpty() && ColumnHolder.TIME_COLUMN_NAME.equals(ordering.get(0).getColumnName())) { + return ordering.get(0).getOrder(); + } else { + return Order.NONE; } } diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java index b4b0ddfbb60b..3fcfe2dd78a7 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java @@ -25,7 +25,6 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.filter.AndFilter; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -88,18 +87,6 @@ public int getDimensionCardinality(String column) return baseStorageAdapter.getDimensionCardinality(column); } - @Override - public DateTime getMinTime() - { - return baseStorageAdapter.getMinTime(); - } - - @Override - public DateTime getMaxTime() - { - return baseStorageAdapter.getMaxTime(); - } - @Nullable @Override public Comparable getMinValue(String column) @@ -127,12 +114,6 @@ public int getNumRows() return 0; } - @Override - public DateTime getMaxIngestedEventTime() - { - return baseStorageAdapter.getMaxIngestedEventTime(); - } - @Nullable @Override public Metadata getMetadata() diff --git a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexMaxIngestedEventTimeInspector.java b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexMaxIngestedEventTimeInspector.java new file mode 100644 index 000000000000..5d4d10a37e64 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexMaxIngestedEventTimeInspector.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment; + +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.joda.time.DateTime; + +public class IncrementalIndexMaxIngestedEventTimeInspector implements MaxIngestedEventTimeInspector +{ + private final IncrementalIndex index; + + public IncrementalIndexMaxIngestedEventTimeInspector(IncrementalIndex index) + { + this.index = index; + } + + @Override + public DateTime getMaxIngestedEventTime() + { + return index.getMaxIngestedEventTime(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java index b270b54c4d1d..62ecda675ac4 100644 --- a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java @@ -24,6 +24,8 @@ import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; +import javax.annotation.Nullable; + /** */ public class IncrementalIndexSegment implements Segment @@ -61,6 +63,19 @@ public StorageAdapter asStorageAdapter() return new IncrementalIndexStorageAdapter(index); } + @Nullable + @Override + public T as(final Class clazz) + { + if (TimeBoundaryInspector.class.equals(clazz)) { + return (T) new IncrementalIndexTimeBoundaryInspector(index); + } else if (MaxIngestedEventTimeInspector.class.equals(clazz)) { + return (T) new IncrementalIndexMaxIngestedEventTimeInspector(index); + } else { + return Segment.super.as(clazz); + } + } + @Override public void close() { diff --git a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexTimeBoundaryInspector.java b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexTimeBoundaryInspector.java new file mode 100644 index 000000000000..cfe8aa7d762c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexTimeBoundaryInspector.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment; + +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.joda.time.DateTime; + +/** + * {@link TimeBoundaryInspector} for {@link IncrementalIndex}. + */ +public class IncrementalIndexTimeBoundaryInspector implements TimeBoundaryInspector +{ + private final IncrementalIndex index; + + public IncrementalIndexTimeBoundaryInspector(IncrementalIndex index) + { + this.index = index; + } + + @Override + public DateTime getMinTime() + { + final DateTime minTime = index.getMinTime(); + return minTime == null ? DateTimes.MIN : minTime; + } + + @Override + public DateTime getMaxTime() + { + final DateTime maxTime = index.getMaxTime(); + return maxTime == null ? DateTimes.MAX : maxTime; + } + + @Override + public boolean isMinMaxExact() + { + return !index.isEmpty(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/IndexIO.java b/processing/src/main/java/org/apache/druid/segment/IndexIO.java index 966de4052066..4db49535ae60 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexIO.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexIO.java @@ -140,8 +140,8 @@ public void validateTwoSegments(final IndexableAdapter adapter1, final Indexable ); } { - final Set dimNames1 = Sets.newHashSet(adapter1.getDimensionNames()); - final Set dimNames2 = Sets.newHashSet(adapter2.getDimensionNames()); + final Set dimNames1 = Sets.newHashSet(adapter1.getDimensionNames(true)); + final Set dimNames2 = Sets.newHashSet(adapter2.getDimensionNames(true)); if (!dimNames1.equals(dimNames2)) { throw new SegmentValidationException( "Dimension names differ. Expected [%s] found [%s]", @@ -240,8 +240,8 @@ private static void validateRowValues( if (dims1.size() != dims2.size()) { throw new SegmentValidationException("Dim lengths not equal %s vs %s", dims1, dims2); } - final List dim1Names = adapter1.getDimensionNames(); - final List dim2Names = adapter2.getDimensionNames(); + final List dim1Names = adapter1.getDimensionNames(false); + final List dim2Names = adapter2.getDimensionNames(false); int dimCount = dims1.size(); for (int i = 0; i < dimCount; ++i) { final String dim1Name = dim1Names.get(i); diff --git a/processing/src/main/java/org/apache/druid/segment/IndexMerger.java b/processing/src/main/java/org/apache/druid/segment/IndexMerger.java index c3e0e2d4dd9c..f397b02814c4 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMerger.java @@ -20,7 +20,6 @@ package org.apache.druid.segment; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.inject.ImplementedBy; @@ -29,6 +28,7 @@ import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.utils.CollectionUtils; @@ -53,12 +53,18 @@ public interface IndexMerger int INVALID_ROW = -1; int UNLIMITED_MAX_COLUMNS_TO_MERGE = -1; + /** + * Returns merged dimensions, not including {@link ColumnHolder#TIME_COLUMN_NAME}, from a set of indexes. + */ static List getMergedDimensionsFromQueryableIndexes( List indexes, @Nullable DimensionsSpec dimensionsSpec ) { - return getMergedDimensions(toIndexableAdapters(indexes), dimensionsSpec); + return getMergedDimensionsWithTime(toIndexableAdapters(indexes), dimensionsSpec) + .stream() + .filter(column -> !ColumnHolder.TIME_COLUMN_NAME.equals(column)) + .collect(Collectors.toList()); } static List toIndexableAdapters(List indexes) @@ -66,12 +72,16 @@ static List toIndexableAdapters(List indexes) return indexes.stream().map(QueryableIndexIndexableAdapter::new).collect(Collectors.toList()); } - static List getMergedDimensions( + /** + * Returns a merged list of dimensions, including {@link ColumnHolder#TIME_COLUMN_NAME}, that will be used + * to build {@link TimeAndDimsPointer}. + */ + static List getMergedDimensionsWithTime( List indexes, @Nullable DimensionsSpec dimensionsSpec ) { - if (indexes.size() == 0) { + if (indexes.isEmpty()) { return ImmutableList.of(); } List commonDimOrder = getLongestSharedDimOrder(indexes, dimensionsSpec); @@ -84,6 +94,9 @@ static List getMergedDimensions( } } + /** + * Gets the longest shared dimension order, including both {@link ColumnHolder#TIME_COLUMN_NAME} and dimensions. + */ @Nullable static List getLongestSharedDimOrder( List indexes, @@ -93,10 +106,10 @@ static List getLongestSharedDimOrder( int maxSize = 0; Iterable orderingCandidate = null; for (IndexableAdapter index : indexes) { - int iterSize = index.getDimensionNames().size(); + int iterSize = index.getDimensionNames(true).size(); if (iterSize > maxSize) { maxSize = iterSize; - orderingCandidate = index.getDimensionNames(); + orderingCandidate = index.getDimensionNames(true); } } @@ -113,11 +126,16 @@ static List getLongestSharedDimOrder( log.info("Cannot fall back on dimension ordering from ingestionSpec as it does not exist"); return null; } - List candidate = new ArrayList<>(dimensionsSpec.getDimensionNames()); + List candidate = new ArrayList<>(); + if (!dimensionsSpec.getDimensionNames().contains(ColumnHolder.TIME_COLUMN_NAME)) { + candidate.add(ColumnHolder.TIME_COLUMN_NAME); + } + candidate.addAll(dimensionsSpec.getDimensionNames()); // Remove all dimensions that does not exist within the indexes from the candidate - Set allValidDimensions = indexes.stream() - .flatMap(indexableAdapter -> indexableAdapter.getDimensionNames().stream()) - .collect(Collectors.toSet()); + Set allValidDimensions = + indexes.stream() + .flatMap(indexableAdapter -> indexableAdapter.getDimensionNames(true).stream()) + .collect(Collectors.toSet()); candidate.retainAll(allValidDimensions); // Sanity check that there is no extra/missing columns if (candidate.size() != allValidDimensions.size()) { @@ -141,7 +159,7 @@ static boolean isDimensionOrderingValid(List indexes, Iterable { for (IndexableAdapter index : indexes) { Iterator candidateIter = orderingCandidate.iterator(); - for (String matchDim : index.getDimensionNames()) { + for (String matchDim : index.getDimensionNames(true)) { boolean matched = false; while (candidateIter.hasNext()) { String nextDim = candidateIter.next(); @@ -158,21 +176,15 @@ static boolean isDimensionOrderingValid(List indexes, Iterable return true; } + /** + * Returns {@link ColumnHolder#TIME_COLUMN_NAME} followed by a merged list of dimensions in lexicographic order. + */ static List getLexicographicMergedDimensions(List indexes) { - return mergeIndexed( - Lists.transform( - indexes, - new Function>() - { - @Override - public Iterable apply(@Nullable IndexableAdapter input) - { - return input.getDimensionNames(); - } - } - ) - ); + final List retVal = new ArrayList<>(); + retVal.add(ColumnHolder.TIME_COLUMN_NAME); + retVal.addAll(mergeIndexed(Lists.transform(indexes, input -> input.getDimensionNames(false)))); + return retVal; } static > ArrayList mergeIndexed(List> indexedLists) diff --git a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java index 05c721e5c560..c60a4acf3519 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java @@ -28,6 +28,7 @@ import com.google.inject.Inject; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.io.ZeroCopyByteArrayOutputStream; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.FileUtils; @@ -133,7 +134,7 @@ private File makeIndexFiles( final @Nullable AggregatorFactory[] metricAggs, final File outDir, final ProgressIndicator progress, - final List mergedDimensions, // should have both explicit and implicit dimensions + final List mergedDimensionsWithTime, // has both explicit and implicit dimensions, as well as __time final DimensionsSpecInspector dimensionsSpecInspector, final List mergedMetrics, final Function, TimeAndDimsIterator> rowMergerFn, @@ -147,6 +148,12 @@ private File makeIndexFiles( List metadataList = Lists.transform(adapters, IndexableAdapter::getMetadata); + // Merged dimensions without __time. + List mergedDimensions = + mergedDimensionsWithTime.stream() + .filter(dim -> !ColumnHolder.TIME_COLUMN_NAME.equals(dim)) + .collect(Collectors.toList()); + final Metadata segmentMetadata; if (metricAggs != null) { AggregatorFactory[] combiningMetricAggs = new AggregatorFactory[metricAggs.length]; @@ -164,6 +171,18 @@ private File makeIndexFiles( ); } + if (segmentMetadata != null + && segmentMetadata.getOrdering() != null + && segmentMetadata.getOrdering() + .stream() + .noneMatch(orderBy -> ColumnHolder.TIME_COLUMN_NAME.equals(orderBy.getColumnName()))) { + throw DruidException.defensive( + "sortOrder[%s] must include[%s]", + segmentMetadata.getOrdering(), + ColumnHolder.TIME_COLUMN_NAME + ); + } + Closer closer = Closer.create(); try { final FileSmoosher v9Smoosher = new FileSmoosher(outDir); @@ -220,7 +239,7 @@ private File makeIndexFiles( progress.progress(); final TimeAndDimsIterator timeAndDimsIterator = makeMergedTimeAndDimsIterator( adapters, - mergedDimensions, + mergedDimensionsWithTime, mergedMetrics, rowMergerFn, handlers, @@ -827,7 +846,7 @@ private void mergeFormat( { final Map columnFormats = new HashMap<>(); for (IndexableAdapter adapter : adapters) { - for (String dimension : adapter.getDimensionNames()) { + for (String dimension : adapter.getDimensionNames(false)) { ColumnFormat format = adapter.getFormat(dimension); columnFormats.compute(dimension, (d, existingFormat) -> existingFormat == null ? format : format.merge(existingFormat)); } @@ -1082,8 +1101,7 @@ private List> getMergePhases(List index private int getIndexColumnCount(IndexableAdapter indexableAdapter) { - // +1 for the __time column - return 1 + indexableAdapter.getDimensionNames().size() + indexableAdapter.getMetricNames().size(); + return indexableAdapter.getDimensionNames(true).size() + indexableAdapter.getMetricNames().size(); } private int getIndexColumnCount(List indexableAdapters) @@ -1106,7 +1124,7 @@ private File merge( @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { - final List mergedDimensions = IndexMerger.getMergedDimensions(indexes, dimensionsSpec); + final List mergedDimensionsWithTime = IndexMerger.getMergedDimensionsWithTime(indexes, dimensionsSpec); final List mergedMetrics = IndexMerger.mergeIndexed( indexes.stream().map(IndexableAdapter::getMetricNames).collect(Collectors.toList()) @@ -1157,7 +1175,7 @@ private File merge( sortedMetricAggs, outDir, progress, - mergedDimensions, + mergedDimensionsWithTime, new DimensionsSpecInspector(storeEmptyColumns, dimensionsSpec), mergedMetrics, rowMergerFn, @@ -1183,7 +1201,7 @@ private Map makeDimensionHandlers( private TimeAndDimsIterator makeMergedTimeAndDimsIterator( final List indexes, - final List mergedDimensions, + final List mergedDimensionsWithTime, final List mergedMetrics, final Function, TimeAndDimsIterator> rowMergerFn, final Map handlers, @@ -1194,9 +1212,10 @@ private TimeAndDimsIterator makeMergedTimeAndDimsIterator( for (int i = 0; i < indexes.size(); ++i) { final IndexableAdapter adapter = indexes.get(i); TransformableRowIterator target = adapter.getRows(); - if (!mergedDimensions.equals(adapter.getDimensionNames()) || !mergedMetrics.equals(adapter.getMetricNames())) { + if (!mergedDimensionsWithTime.equals(adapter.getDimensionNames(true)) + || !mergedMetrics.equals(adapter.getMetricNames())) { target = makeRowIteratorWithReorderedColumns( - mergedDimensions, + mergedDimensionsWithTime, mergedMetrics, handlers, adapter, @@ -1209,7 +1228,7 @@ private TimeAndDimsIterator makeMergedTimeAndDimsIterator( } private TransformableRowIterator makeRowIteratorWithReorderedColumns( - List reorderedDimensions, + List reorderedDimensionsWithTime, List reorderedMetrics, Map originalHandlers, IndexableAdapter originalAdapter, @@ -1217,14 +1236,14 @@ private TransformableRowIterator makeRowIteratorWithReorderedColumns( ) { RowPointer reorderedRowPointer = reorderRowPointerColumns( - reorderedDimensions, + reorderedDimensionsWithTime, reorderedMetrics, originalHandlers, originalAdapter, originalIterator.getPointer() ); TimeAndDimsPointer reorderedMarkedRowPointer = reorderRowPointerColumns( - reorderedDimensions, + reorderedDimensionsWithTime, reorderedMetrics, originalHandlers, originalAdapter, @@ -1247,17 +1266,22 @@ public TimeAndDimsPointer getMarkedPointer() } private static T reorderRowPointerColumns( - List reorderedDimensions, + List reorderedDimensionsWithTime, List reorderedMetrics, Map originalHandlers, IndexableAdapter originalAdapter, T originalRowPointer ) { - ColumnValueSelector[] reorderedDimensionSelectors = reorderedDimensions + int reorderedTimePosition = reorderedDimensionsWithTime.indexOf(ColumnHolder.TIME_COLUMN_NAME); + if (reorderedTimePosition < 0) { + throw DruidException.defensive("Missing column[%s]", ColumnHolder.TIME_COLUMN_NAME); + } + ColumnValueSelector[] reorderedDimensionSelectors = reorderedDimensionsWithTime .stream() + .filter(column -> !ColumnHolder.TIME_COLUMN_NAME.equals(column)) .map(dimName -> { - int dimIndex = originalAdapter.getDimensionNames().indexOf(dimName); + int dimIndex = originalAdapter.getDimensionNames(false).indexOf(dimName); if (dimIndex >= 0) { return originalRowPointer.getDimensionSelector(dimIndex); } else { @@ -1266,7 +1290,9 @@ private static T reorderRowPointerColumns( }) .toArray(ColumnValueSelector[]::new); List reorderedHandlers = - reorderedDimensions.stream().map(originalHandlers::get).collect(Collectors.toList()); + reorderedDimensionsWithTime.stream() + .filter(column -> !ColumnHolder.TIME_COLUMN_NAME.equals(column)) + .map(originalHandlers::get).collect(Collectors.toList()); ColumnValueSelector[] reorderedMetricSelectors = reorderedMetrics .stream() .map(metricName -> { @@ -1282,6 +1308,7 @@ private static T reorderRowPointerColumns( //noinspection unchecked return (T) new RowPointer( originalRowPointer.timestampSelector, + reorderedTimePosition, reorderedDimensionSelectors, reorderedHandlers, reorderedMetricSelectors, @@ -1292,6 +1319,7 @@ private static T reorderRowPointerColumns( //noinspection unchecked return (T) new TimeAndDimsPointer( originalRowPointer.timestampSelector, + reorderedTimePosition, reorderedDimensionSelectors, reorderedHandlers, reorderedMetricSelectors, diff --git a/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java index 6f520dd2f9c7..023ca7001924 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java @@ -23,6 +23,7 @@ import org.apache.druid.segment.column.CapabilitiesBasedFormat; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnFormat; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.BitmapValues; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.nested.FieldTypeInfo; @@ -45,7 +46,12 @@ public interface IndexableAdapter int getNumRows(); - List getDimensionNames(); + /** + * Returns names of dimension columns. + * + * @param includeTime whether to include {@link ColumnHolder#TIME_COLUMN_NAME}. + */ + List getDimensionNames(boolean includeTime); List getMetricNames(); diff --git a/processing/src/main/java/org/apache/druid/segment/MaxIngestedEventTimeInspector.java b/processing/src/main/java/org/apache/druid/segment/MaxIngestedEventTimeInspector.java new file mode 100644 index 000000000000..67fc7536f118 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/MaxIngestedEventTimeInspector.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.query.datasourcemetadata.DataSourceMetadataQuery; +import org.apache.druid.query.datasourcemetadata.DataSourceMetadataResultValue; +import org.joda.time.DateTime; + +/** + * Inspector used by {@link DataSourceMetadataQuery} to populate + * {@link DataSourceMetadataResultValue#getMaxIngestedEventTime()}. + */ +public interface MaxIngestedEventTimeInspector +{ + /** + * Highest known {@link InputRow#getTimestamp()} for rows that went into creating this index. + */ + DateTime getMaxIngestedEventTime(); +} 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 c3ac554a0faf..e21c673e2796 100644 --- a/processing/src/main/java/org/apache/druid/segment/Metadata.java +++ b/processing/src/main/java/org/apache/druid/segment/Metadata.java @@ -19,10 +19,13 @@ package org.apache.druid.segment; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.annotations.PublicApi; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.aggregation.AggregatorFactory; import javax.annotation.Nullable; @@ -35,6 +38,7 @@ import java.util.concurrent.ConcurrentHashMap; /** + * */ @PublicApi public class Metadata @@ -50,13 +54,16 @@ public class Metadata private final Granularity queryGranularity; @Nullable private final Boolean rollup; + @Nullable + private final List ordering; public Metadata( @JsonProperty("container") @Nullable Map container, @JsonProperty("aggregators") @Nullable AggregatorFactory[] aggregators, @JsonProperty("timestampSpec") @Nullable TimestampSpec timestampSpec, @JsonProperty("queryGranularity") @Nullable Granularity queryGranularity, - @JsonProperty("rollup") @Nullable Boolean rollup + @JsonProperty("rollup") @Nullable Boolean rollup, + @JsonProperty("ordering") @Nullable List ordering ) { this.container = container == null ? new ConcurrentHashMap<>() : container; @@ -64,6 +71,7 @@ public Metadata( this.timestampSpec = timestampSpec; this.queryGranularity = queryGranularity; this.rollup = rollup; + this.ordering = ordering; } @JsonProperty @@ -100,6 +108,22 @@ public Boolean isRollup() return rollup; } + /** + * Ordering for the segment associated with this object. Nonnull for segments written in current versions of Druid, + * but would null for older segments. Null may be interpreted as {@link Cursors#ascendingTimeOrder()}, since prior + * to this field being added, segments were always time-ordered. + * + * When dealing with {@link QueryableIndex}, it is generally better to use {@link QueryableIndex#getOrdering()}, since + * that method never returns null. + */ + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getOrdering() + { + return ordering; + } + public Metadata putAll(@Nullable Map other) { if (other != null) { @@ -130,6 +154,7 @@ public static Metadata merge( List timestampSpecsToMerge = new ArrayList<>(); List gransToMerge = new ArrayList<>(); List rollupToMerge = new ArrayList<>(); + List> orderingsToMerge = new ArrayList<>(); for (Metadata metadata : toBeMerged) { if (metadata != null) { @@ -149,6 +174,8 @@ public static Metadata merge( if (rollupToMerge != null) { rollupToMerge.add(metadata.isRollup()); } + + orderingsToMerge.add(metadata.getOrdering()); mergedContainer.putAll(metadata.container); } else { //if metadata and hence aggregators and queryGranularity for some segment being merged are unknown then @@ -176,6 +203,8 @@ public static Metadata merge( null : Granularity.mergeGranularities(gransToMerge); + final List mergedOrdering = mergeOrderings(orderingsToMerge); + Boolean rollup = null; if (rollupToMerge != null && !rollupToMerge.isEmpty()) { rollup = rollupToMerge.get(0); @@ -197,7 +226,8 @@ public static Metadata merge( mergedAggregators, mergedTimestampSpec, mergedGranularity, - rollup + rollup, + mergedOrdering ); } @@ -215,7 +245,8 @@ public boolean equals(Object o) Arrays.equals(aggregators, metadata.aggregators) && Objects.equals(timestampSpec, metadata.timestampSpec) && Objects.equals(queryGranularity, metadata.queryGranularity) && - Objects.equals(rollup, metadata.rollup); + Objects.equals(rollup, metadata.rollup) && + Objects.equals(ordering, metadata.ordering); } @Override @@ -233,6 +264,48 @@ public String toString() ", timestampSpec=" + timestampSpec + ", queryGranularity=" + queryGranularity + ", rollup=" + rollup + + ", ordering=" + ordering + '}'; } + + /** + * Merge {@link #getOrdering()} from different metadatas. + * + * When an input sort order is null, we assume it is {@link Cursors#ascendingTimeOrder()}, as this was the only + * sort order possible prior to the introduction of the "ordering" field. + */ + public static List mergeOrderings(List> orderingsToMerge) + { + if (orderingsToMerge.isEmpty()) { + throw new IAE("orderingsToMerge is empty"); + } + + final List mergedOrdering = new ArrayList<>(); + + while (true) { + final int position = mergedOrdering.size(); + OrderBy orderBy = null; + + // Iterate through each sort order, check that the columns at "position" are all the same. If not, return + // the mergedOrdering as-is. + for (List ordering : orderingsToMerge) { + if (ordering == null) { + // null ordering is treated as [__time]. + ordering = Cursors.ascendingTimeOrder(); + } + + if (position < ordering.size()) { + if (orderBy == null) { + orderBy = ordering.get(position); + } else if (!orderBy.equals(ordering.get(position))) { + return mergedOrdering; + } + } else { + return mergedOrdering; + } + } + + mergedOrdering.add(orderBy); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java index b4cd503681c6..3766db4a4b19 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java @@ -20,6 +20,7 @@ package org.apache.druid.segment; import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.query.OrderBy; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.Indexed; @@ -42,9 +43,17 @@ public interface QueryableIndex extends Closeable, ColumnInspector { Interval getDataInterval(); int getNumRows(); + /** + * List of dimensions, not including {@link ColumnHolder#TIME_COLUMN_NAME}. + */ Indexed getAvailableDimensions(); BitmapFactory getBitmapFactoryForDimensions(); @Nullable Metadata getMetadata(); + + /** + * Map of column name to {@link DimensionHandler}, whose contents and iteration order matches + * {@link #getAvailableDimensions()}. + */ Map getDimensionHandlers(); List getColumnNames(); @@ -63,6 +72,11 @@ default ColumnCapabilities getColumnCapabilities(String column) return columnHolder.getCapabilities(); } + /** + * Returns the ordering of rows in this index. + */ + List getOrdering(); + /** * The close method shouldn't actually be here as this is nasty. We will adjust it in the future. * @throws IOException if an exception was thrown closing the index diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java index 887f74368aa8..22da9a075054 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java @@ -19,6 +19,7 @@ package org.apache.druid.segment; +import org.apache.druid.query.Order; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.segment.column.BaseColumn; @@ -43,7 +44,7 @@ public class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory, RowIdSupplier { private final VirtualColumns virtualColumns; - private final boolean descending; + private final Order timeOrder; protected final ReadableOffset offset; // Share Column objects, since they cache decompressed buffers internally, and we can avoid recomputation if the @@ -56,13 +57,13 @@ public class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactor public QueryableIndexColumnSelectorFactory( VirtualColumns virtualColumns, - boolean descending, + Order timeOrder, ReadableOffset offset, ColumnCache columnCache ) { this.virtualColumns = virtualColumns; - this.descending = descending; + this.timeOrder = timeOrder; this.offset = offset; this.columnCache = columnCache; this.dimensionSelectorCache = new HashMap<>(); @@ -107,8 +108,12 @@ private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensi return DimensionSelector.constant(null, extractionFn); } - if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME)) { - return new SingleScanTimeDimensionSelector(makeColumnValueSelector(dimension), extractionFn, descending); + if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME) && timeOrder != Order.NONE) { + return new SingleScanTimeDimensionSelector( + makeColumnValueSelector(dimension), + extractionFn, + timeOrder + ); } ColumnCapabilities capabilities = columnHolder.getCapabilities(); diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java index 0f0646839ac7..53412bf9da02 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.io.Closer; @@ -29,6 +30,7 @@ import org.apache.druid.query.BaseQuery; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.DefaultBitmapResultFactory; +import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; @@ -36,13 +38,16 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterBundle; +import org.apache.druid.query.filter.RangeFilter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.NumericColumn; import org.apache.druid.segment.data.Offset; import org.apache.druid.segment.data.ReadableOffset; +import org.apache.druid.segment.filter.AndFilter; import org.apache.druid.segment.historical.HistoricalCursor; import org.apache.druid.segment.vector.BitmapVectorOffset; import org.apache.druid.segment.vector.FilteredVectorOffset; @@ -57,7 +62,6 @@ import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; @@ -75,7 +79,6 @@ public class QueryableIndexCursorHolder implements CursorHolder @Nullable private final QueryMetrics metrics; private final List ordering; - private final boolean descending; private final QueryContext queryContext; private final int vectorSize; private final Supplier resourcesSupplier; @@ -90,18 +93,28 @@ public QueryableIndexCursorHolder( this.virtualColumns = cursorBuildSpec.getVirtualColumns(); this.aggregatorFactories = cursorBuildSpec.getAggregators(); this.filter = cursorBuildSpec.getFilter(); - // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary - if (Cursors.preferDescendingTimeOrdering(cursorBuildSpec)) { - this.ordering = Collections.singletonList(OrderBy.descending(ColumnHolder.TIME_COLUMN_NAME)); - this.descending = true; + + final List indexOrdering = index.getOrdering(); + if (Cursors.preferDescendingTimeOrdering(cursorBuildSpec) + && Cursors.getTimeOrdering(indexOrdering) == Order.ASCENDING) { + this.ordering = Cursors.descendingTimeOrder(); } else { - this.ordering = Collections.singletonList(OrderBy.ascending(ColumnHolder.TIME_COLUMN_NAME)); - this.descending = false; + this.ordering = indexOrdering; } + this.queryContext = cursorBuildSpec.getQueryContext(); this.vectorSize = cursorBuildSpec.getQueryContext().getVectorSize(); this.metrics = cursorBuildSpec.getQueryMetrics(); - this.resourcesSupplier = Suppliers.memoize(() -> new CursorResources(index, virtualColumns, filter, metrics)); + this.resourcesSupplier = Suppliers.memoize( + () -> new CursorResources( + index, + virtualColumns, + Cursors.getTimeOrdering(ordering), + interval, + filter, + metrics + ) + ); } @Override @@ -130,7 +143,7 @@ public boolean canVectorize() } // vector cursors can't iterate backwards yet - return !descending; + return Cursors.getTimeOrdering(ordering) != Order.DESCENDING; } @Override @@ -148,51 +161,60 @@ public Cursor asCursor() final long maxDataTimestamp = resources.maxDataTimestamp; final NumericColumn timestamps = resources.timestamps; final ColumnCache columnCache = resources.columnCache; + final Order timeOrder = resources.timeOrder; - // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter + // if filterBundle is null, the filter itself is also null. otherwise check to see if the filter // can use an index if (filterBundle == null || filterBundle.getIndex() == null) { - baseOffset = descending ? new SimpleDescendingOffset(numRows) : new SimpleAscendingOffset(numRows); + baseOffset = + timeOrder == Order.DESCENDING ? new SimpleDescendingOffset(numRows) : new SimpleAscendingOffset(numRows); } else { - baseOffset = BitmapOffset.of(filterBundle.getIndex().getBitmap(), descending, index.getNumRows()); + baseOffset = + BitmapOffset.of(filterBundle.getIndex().getBitmap(), timeOrder == Order.DESCENDING, index.getNumRows()); } final long timeStart = Math.max(interval.getStartMillis(), minDataTimestamp); final long timeEnd = interval.getEndMillis(); - if (descending) { + if (timeOrder == Order.ASCENDING) { for (; baseOffset.withinBounds(); baseOffset.increment()) { - if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) { + if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) { break; } } - } else { + } else if (timeOrder == Order.DESCENDING) { for (; baseOffset.withinBounds(); baseOffset.increment()) { - if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) { + if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) { break; } } } - final Offset offset = descending ? - new DescendingTimestampCheckingOffset( - baseOffset, - timestamps, - timeStart, - minDataTimestamp >= timeStart - ) : - new AscendingTimestampCheckingOffset( - baseOffset, - timestamps, - timeEnd, - maxDataTimestamp < timeEnd - ); + final Offset offset; + if (timeOrder == Order.ASCENDING) { + offset = new AscendingTimestampCheckingOffset( + baseOffset, + timestamps, + timeEnd, + maxDataTimestamp < timeEnd + ); + } else if (timeOrder == Order.DESCENDING) { + offset = new DescendingTimestampCheckingOffset( + baseOffset, + timestamps, + timeStart, + minDataTimestamp >= timeStart + ); + } else { + // Time filter is moved into filterBundle in the non-time-ordered case. + offset = baseOffset; + } final Offset baseCursorOffset = offset.clone(); final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( virtualColumns, - descending, + Cursors.getTimeOrdering(ordering), baseCursorOffset.getBaseReadableOffset(), columnCache ); @@ -203,7 +225,7 @@ public Cursor asCursor() .valueMatcher( columnSelectorFactory, baseCursorOffset, - descending + timeOrder == Order.DESCENDING ); final FilteredOffset filteredOffset = new FilteredOffset(baseCursorOffset, matcher); return new QueryableIndexCursor(filteredOffset, columnSelectorFactory); @@ -212,6 +234,10 @@ public Cursor asCursor() } } + /** + * Compute filter to use for cursor creation. For non-time-ordered segments, this includes the query interval + * as a filter. + */ @Nullable @Override public VectorCursor asVectorCursor() @@ -222,6 +248,7 @@ public VectorCursor asVectorCursor() final long maxDataTimestamp = resources.maxDataTimestamp; final NumericColumn timestamps = resources.timestamps; final ColumnCache columnCache = resources.columnCache; + final Order timeOrder = resources.timeOrder; // Wrap the remainder of cursor setup in a try, so if an error is encountered while setting it up, we don't // leak columns in the ColumnCache. @@ -234,17 +261,18 @@ public VectorCursor asVectorCursor() metrics.vectorized(true); } - + // startOffset, endOffset must match the "interval" if timeOrdered. Otherwise, the "interval" filtering is embedded + // within the filterBundle. final int startOffset; final int endOffset; - if (interval.getStartMillis() > minDataTimestamp) { + if (timeOrder != Order.NONE && interval.getStartMillis() > minDataTimestamp) { startOffset = timeSearch(timestamps, interval.getStartMillis(), 0, index.getNumRows()); } else { startOffset = 0; } - if (interval.getEndMillis() <= maxDataTimestamp) { + if (timeOrder != Order.NONE && interval.getEndMillis() <= maxDataTimestamp) { endOffset = timeSearch(timestamps, interval.getEndMillis(), startOffset, index.getNumRows()); } else { endOffset = index.getNumRows(); @@ -318,6 +346,7 @@ private VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( * @param timestamp the timestamp to search for * @param startIndex first index to search, inclusive * @param endIndex last index to search, exclusive + * * @return first index that has a timestamp equal to, or greater, than "timestamp" */ @VisibleForTesting @@ -625,24 +654,21 @@ private static final class CursorResources implements Closeable private final long maxDataTimestamp; private final int numRows; @Nullable - private final Filter filter; - @Nullable private final FilterBundle filterBundle; private final NumericColumn timestamps; + private final Order timeOrder; private final ColumnCache columnCache; - @Nullable - private final QueryMetrics> metrics; private CursorResources( QueryableIndex index, VirtualColumns virtualColumns, + Order timeOrder, + Interval interval, @Nullable Filter filter, @Nullable QueryMetrics> metrics ) { this.closer = Closer.create(); - this.filter = filter; - this.metrics = metrics; this.columnCache = new ColumnCache(index, closer); final ColumnSelectorColumnIndexSelector bitmapIndexSelector = new ColumnSelectorColumnIndexSelector( index.getBitmapFactoryForDimensions(), @@ -651,10 +677,22 @@ private CursorResources( ); try { this.numRows = index.getNumRows(); - this.filterBundle = makeFilterBundle(bitmapIndexSelector, numRows); this.timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME); this.minDataTimestamp = DateTimes.utc(timestamps.getLongSingleValueRow(0)).getMillis(); this.maxDataTimestamp = DateTimes.utc(timestamps.getLongSingleValueRow(timestamps.length() - 1)).getMillis(); + this.filterBundle = makeFilterBundle( + computeFilterWithIntervalIfNeeded( + timeOrder, + this.minDataTimestamp, + this.maxDataTimestamp, + interval, + filter + ), + bitmapIndexSelector, + numRows, + metrics + ); + this.timeOrder = timeOrder; } catch (Throwable t) { throw CloseableUtils.closeAndWrapInCatch(t, closer); @@ -666,49 +704,93 @@ public void close() throws IOException { closer.close(); } + } - @Nullable - private FilterBundle makeFilterBundle( - ColumnSelectorColumnIndexSelector bitmapIndexSelector, - int numRows - ) - { - final BitmapFactory bitmapFactory = bitmapIndexSelector.getBitmapFactory(); - final BitmapResultFactory bitmapResultFactory; - if (metrics != null) { - bitmapResultFactory = metrics.makeBitmapResultFactory(bitmapFactory); - metrics.reportSegmentRows(numRows); + /** + * Create a {@link FilterBundle} for a cursor hold instance. + * + * The provided filter must include the query-level interface if needed. To compute this properly, use + * {@link #computeFilterWithIntervalIfNeeded}. + */ + @Nullable + private static FilterBundle makeFilterBundle( + @Nullable final Filter filter, + final ColumnSelectorColumnIndexSelector bitmapIndexSelector, + final int numRows, + @Nullable final QueryMetrics metrics + ) + { + final BitmapFactory bitmapFactory = bitmapIndexSelector.getBitmapFactory(); + final BitmapResultFactory bitmapResultFactory; + if (metrics != null) { + bitmapResultFactory = metrics.makeBitmapResultFactory(bitmapFactory); + metrics.reportSegmentRows(numRows); + } else { + bitmapResultFactory = new DefaultBitmapResultFactory(bitmapFactory); + } + if (filter == null) { + return null; + } + final long bitmapConstructionStartNs = System.nanoTime(); + final FilterBundle filterBundle = filter.makeFilterBundle( + bitmapIndexSelector, + bitmapResultFactory, + numRows, + numRows, + false + ); + if (metrics != null) { + final long buildTime = System.nanoTime() - bitmapConstructionStartNs; + metrics.reportBitmapConstructionTime(buildTime); + final FilterBundle.BundleInfo info = filterBundle.getInfo(); + metrics.filterBundle(info); + log.debug("Filter partitioning (%sms):%s", TimeUnit.NANOSECONDS.toMillis(buildTime), info); + if (filterBundle.getIndex() != null) { + metrics.reportPreFilteredRows(filterBundle.getIndex().getBitmap().size()); } else { - bitmapResultFactory = new DefaultBitmapResultFactory(bitmapFactory); + metrics.reportPreFilteredRows(0); } - if (filter == null) { - return null; - } - final long bitmapConstructionStartNs = System.nanoTime(); - final FilterBundle filterBundle = filter.makeFilterBundle( - bitmapIndexSelector, - bitmapResultFactory, - numRows, - numRows, - false + } else if (log.isDebugEnabled()) { + final FilterBundle.BundleInfo info = filterBundle.getInfo(); + final long buildTime = System.nanoTime() - bitmapConstructionStartNs; + log.debug("Filter partitioning (%sms):%s", TimeUnit.NANOSECONDS.toMillis(buildTime), info); + } + return filterBundle; + } + + /** + * Returns the query-level {@link Filter} plus, if needed, a {@link RangeFilter} for + * {@link ColumnHolder#TIME_COLUMN_NAME}. The time filter is added if time order is {@link Order#NONE} and + * the provided {@link Interval} is not contained entirely within [minDataTimestamp, maxDataTimestamp]. + */ + @Nullable + private static Filter computeFilterWithIntervalIfNeeded( + final Order timeOrder, + final long minDataTimestamp, + final long maxDataTimestamp, + final Interval interval, + @Nullable final Filter filter + ) + { + if (timeOrder == Order.NONE + && (minDataTimestamp < interval.getStartMillis() || maxDataTimestamp >= interval.getEndMillis())) { + final RangeFilter timeFilter = new RangeFilter( + ColumnHolder.TIME_COLUMN_NAME, + ColumnType.LONG, + minDataTimestamp < interval.getStartMillis() ? interval.getStartMillis() : null, + maxDataTimestamp >= interval.getEndMillis() ? interval.getEndMillis() : null, + false, + true, + null ); - if (metrics != null) { - final long buildTime = System.nanoTime() - bitmapConstructionStartNs; - metrics.reportBitmapConstructionTime(buildTime); - final FilterBundle.BundleInfo info = filterBundle.getInfo(); - metrics.filterBundle(info); - log.debug("Filter partitioning (%sms):%s", TimeUnit.NANOSECONDS.toMillis(buildTime), info); - if (filterBundle.getIndex() != null) { - metrics.reportPreFilteredRows(filterBundle.getIndex().getBitmap().size()); - } else { - metrics.reportPreFilteredRows(0); - } - } else if (log.isDebugEnabled()) { - final FilterBundle.BundleInfo info = filterBundle.getInfo(); - final long buildTime = System.nanoTime() - bitmapConstructionStartNs; - log.debug("Filter partitioning (%sms):%s", TimeUnit.NANOSECONDS.toMillis(buildTime), info); + + if (filter == null) { + return timeFilter; + } else { + return new AndFilter(ImmutableList.of(filter, timeFilter)); } - return filterBundle; + } else { + return filter; } } } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java index f97f4a1bade7..0060b276c4b2 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java @@ -21,7 +21,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.Order; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnCapabilities; @@ -60,13 +63,49 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter private final QueryableIndex input; private final ImmutableList availableDimensions; private final Metadata metadata; + private final int timePositionForComparator; public QueryableIndexIndexableAdapter(QueryableIndex input) { this.input = input; numRows = input.getNumRows(); availableDimensions = ImmutableList.copyOf(input.getAvailableDimensions()); + if (availableDimensions.contains(ColumnHolder.TIME_COLUMN_NAME)) { + throw DruidException.defensive("Unexpectedly encountered dimension[%s]", ColumnHolder.TIME_COLUMN_NAME); + } this.metadata = input.getMetadata(); + + final List inputOrdering = input.getOrdering(); + + int foundTimePosition = -1; + int i = 0; + + // Some sort columns may not exist in the index, for example if they are omitted due to being 100% nulls. + // Locate the __time column in the sort order, skipping any nonexistent columns. This will be the position of + // the __time column within the dimension handlers. + for (final OrderBy orderBy : inputOrdering) { + final String columnName = orderBy.getColumnName(); + + if (orderBy.getOrder() != Order.ASCENDING) { + throw DruidException.defensive("Order[%s] for column[%s] is not supported", orderBy.getOrder(), columnName); + } + + if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { + foundTimePosition = i; + break; + } else if (input.getDimensionHandlers().containsKey(columnName)) { + i++; + } + } + + if (foundTimePosition >= 0) { + this.timePositionForComparator = foundTimePosition; + } else { + // Sort order is set, but does not contain __time. Indexable adapters involve all columns in TimeAndDimsPointer + // comparators, so we need to put the __time column somewhere. Put it immediately after the ones in the + // sort order. + this.timePositionForComparator = inputOrdering.size(); + } } public QueryableIndex getQueryableIndex() @@ -87,16 +126,23 @@ public int getNumRows() } @Override - public List getDimensionNames() + public List getDimensionNames(final boolean includeTime) { - return availableDimensions; + if (includeTime) { + final List retVal = new ArrayList<>(availableDimensions.size() + 1); + retVal.add(ColumnHolder.TIME_COLUMN_NAME); + retVal.addAll(availableDimensions); + return retVal; + } else { + return availableDimensions; + } } @Override public List getMetricNames() { final Set columns = Sets.newLinkedHashSet(input.getColumnNames()); - final HashSet dimensions = Sets.newHashSet(getDimensionNames()); + final HashSet dimensions = Sets.newHashSet(availableDimensions); return ImmutableList.copyOf(Sets.difference(columns, dimensions)); } @@ -262,7 +308,7 @@ class RowIteratorImpl implements TransformableRowIterator final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( VirtualColumns.EMPTY, - false, + timePositionForComparator == 0 ? Order.ASCENDING : Order.NONE, offset, columnCache ); @@ -292,6 +338,7 @@ class RowIteratorImpl implements TransformableRowIterator rowPointer = new RowPointer( rowTimestampSelector, + timePositionForComparator, rowDimensionValueSelectors, dimensionHandlers, rowMetricSelectors, @@ -309,6 +356,7 @@ class RowIteratorImpl implements TransformableRowIterator .toArray(SettableColumnValueSelector[]::new); markedRowPointer = new TimeAndDimsPointer( markedTimestampSelector, + timePositionForComparator, markedDimensionValueSelectors, dimensionHandlers, markedMetricSelectors, diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java index b8d4d2d16cf9..ccbe0475c33c 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java @@ -31,6 +31,7 @@ import java.util.function.Function; /** + * */ public class QueryableIndexSegment implements Segment { @@ -39,12 +40,14 @@ public class QueryableIndexSegment implements Segment private final QueryableIndex index; private final QueryableIndexStorageAdapter storageAdapter; + private final TimeBoundaryInspector timeBoundaryInspector; private final SegmentId segmentId; public QueryableIndexSegment(QueryableIndex index, final SegmentId segmentId) { this.index = index; this.storageAdapter = new QueryableIndexStorageAdapter(index); + this.timeBoundaryInspector = QueryableIndexTimeBoundaryInspector.create(index); this.segmentId = segmentId; } @@ -86,7 +89,14 @@ public T as(@Nonnull Class clazz) { final Function fn = AS_MAP.get(clazz); if (fn != null) { - return (T) fn.apply(this); + final T fnApply = (T) fn.apply(this); + if (fnApply != null) { + return fnApply; + } + } + + if (TimeBoundaryInspector.class.equals(clazz)) { + return (T) timeBoundaryInspector; } return Segment.super.as(clazz); diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index 7d860465cb6f..e2c88071607c 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -19,13 +19,11 @@ package org.apache.druid.segment; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.NumericColumn; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.joda.time.DateTime; @@ -46,12 +44,6 @@ public class QueryableIndexStorageAdapter implements StorageAdapter private final QueryableIndex index; - @Nullable - private volatile DateTime minTime; - - @Nullable - private volatile DateTime maxTime; - public QueryableIndexStorageAdapter(QueryableIndex index) { this.index = index; @@ -107,28 +99,6 @@ public int getNumRows() return index.getNumRows(); } - @Override - public DateTime getMinTime() - { - if (minTime == null) { - // May be called a few times in parallel when first populating minTime, but this is benign, so allow it. - populateMinMaxTime(); - } - - return minTime; - } - - @Override - public DateTime getMaxTime() - { - if (maxTime == null) { - // May be called a few times in parallel when first populating maxTime, but this is benign, so allow it. - populateMinMaxTime(); - } - - return maxTime; - } - @Override @Nullable public Comparable getMinValue(String dimension) @@ -183,14 +153,4 @@ public Metadata getMetadata() { return index.getMetadata(); } - - private void populateMinMaxTime() - { - // Compute and cache minTime, maxTime. - final ColumnHolder columnHolder = index.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME); - try (NumericColumn column = (NumericColumn) columnHolder.getColumn()) { - this.minTime = DateTimes.utc(column.getLongSingleValueRow(0)); - this.maxTime = DateTimes.utc(column.getLongSingleValueRow(column.length() - 1)); - } - } } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexTimeBoundaryInspector.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexTimeBoundaryInspector.java new file mode 100644 index 000000000000..28292fc24391 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexTimeBoundaryInspector.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment; + +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.query.Order; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.NumericColumn; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.joda.time.DateTime; + +/** + * {@link TimeBoundaryInspector} for {@link QueryableIndex} that are sorted by {@link ColumnHolder#TIME_COLUMN_NAME}. + */ +public class QueryableIndexTimeBoundaryInspector implements TimeBoundaryInspector +{ + private final QueryableIndex index; + private final boolean timeOrdered; + + private QueryableIndexTimeBoundaryInspector(final QueryableIndex index, final boolean timeOrdered) + { + this.index = index; + this.timeOrdered = timeOrdered; + } + + public static QueryableIndexTimeBoundaryInspector create(final QueryableIndex index) + { + return new QueryableIndexTimeBoundaryInspector( + index, + Cursors.getTimeOrdering(index.getOrdering()) == Order.ASCENDING + ); + } + + @MonotonicNonNull + private volatile DateTime minTime; + + @MonotonicNonNull + private volatile DateTime maxTime; + + @Override + public DateTime getMinTime() + { + if (minTime == null) { + // May be called a few times in parallel when first populating minTime, but this is benign, so allow it. + populateMinMaxTime(); + } + + return minTime; + } + + @Override + public DateTime getMaxTime() + { + if (maxTime == null) { + // May be called a few times in parallel when first populating maxTime, but this is benign, so allow it. + populateMinMaxTime(); + } + + return maxTime; + } + + @Override + public boolean isMinMaxExact() + { + return timeOrdered; + } + + private void populateMinMaxTime() + { + if (timeOrdered) { + // Compute and cache minTime, maxTime. + final ColumnHolder columnHolder = index.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME); + try (NumericColumn column = (NumericColumn) columnHolder.getColumn()) { + this.minTime = DateTimes.utc(column.getLongSingleValueRow(0)); + this.maxTime = DateTimes.utc(column.getLongSingleValueRow(column.length() - 1)); + } + } else { + // Use metadata. (Will be inexact.) + this.minTime = index.getDataInterval().getStart(); + this.maxTime = index.getDataInterval().getEnd(); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java index f4330709a44b..eaed700f59cf 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java @@ -34,7 +34,6 @@ import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.utils.CloseableUtils; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -57,7 +56,8 @@ public class RowBasedStorageAdapter implements StorageAdapter public RowBasedStorageAdapter( final Sequence rowSequence, final RowAdapter rowAdapter, - final RowSignature rowSignature) + final RowSignature rowSignature + ) { this.rowSequence = Preconditions.checkNotNull(rowSequence, "rowSequence"); this.rowAdapter = Preconditions.checkNotNull(rowAdapter, "rowAdapter"); @@ -105,18 +105,6 @@ public int getDimensionCardinality(String column) return DimensionDictionarySelector.CARDINALITY_UNKNOWN; } - @Override - public DateTime getMinTime() - { - return getInterval().getStart(); - } - - @Override - public DateTime getMaxTime() - { - return getInterval().getEnd().minus(1); - } - @Nullable @Override public Comparable getMinValue(String column) @@ -154,12 +142,6 @@ public int getNumRows() throw new UnsupportedOperationException("Cannot retrieve number of rows"); } - @Override - public DateTime getMaxIngestedEventTime() - { - return getMaxTime(); - } - @Override public Metadata getMetadata() { @@ -169,7 +151,9 @@ public Metadata getMetadata() @Override public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary + // It's in principle incorrect for sort order to be __time based here, but for historical reasons, we're keeping + // this in place for now. The handling of "interval" in "RowBasedCursor", which has been in place for some time, + // suggests we think the data is always sorted by time. final List ordering; final boolean descending; if (Cursors.preferDescendingTimeOrdering(spec)) { diff --git a/processing/src/main/java/org/apache/druid/segment/RowCombiningTimeAndDimsIterator.java b/processing/src/main/java/org/apache/druid/segment/RowCombiningTimeAndDimsIterator.java index 5120ece6a956..919f53a41a09 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowCombiningTimeAndDimsIterator.java +++ b/processing/src/main/java/org/apache/druid/segment/RowCombiningTimeAndDimsIterator.java @@ -128,6 +128,7 @@ final class RowCombiningTimeAndDimsIterator implements TimeAndDimsIterator if (markedRowPointer != null) { return new TimeAndDimsPointer( markedRowPointer.timestampSelector, + markedRowPointer.timePosition, markedRowPointer.dimensionSelectors, markedRowPointer.getDimensionHandlers(), combinedMetricSelectors, diff --git a/processing/src/main/java/org/apache/druid/segment/RowPointer.java b/processing/src/main/java/org/apache/druid/segment/RowPointer.java index dcb7ab6b2040..7a67b7482878 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowPointer.java +++ b/processing/src/main/java/org/apache/druid/segment/RowPointer.java @@ -41,6 +41,7 @@ public final class RowPointer extends TimeAndDimsPointer public RowPointer( ColumnValueSelector timestampSelector, + int timePosition, ColumnValueSelector[] dimensionSelectors, List dimensionHandlers, ColumnValueSelector[] metricSelectors, @@ -48,7 +49,7 @@ public RowPointer( IntSupplier rowNumPointer ) { - super(timestampSelector, dimensionSelectors, dimensionHandlers, metricSelectors, metricNames); + super(timestampSelector, timePosition, dimensionSelectors, dimensionHandlers, metricSelectors, metricNames); this.rowNumPointer = rowNumPointer; } @@ -62,6 +63,7 @@ RowPointer withDimensionSelectors(ColumnValueSelector[] newDimensionSelectors) { return new RowPointer( timestampSelector, + timePosition, newDimensionSelectors, getDimensionHandlers(), metricSelectors, diff --git a/processing/src/main/java/org/apache/druid/segment/Segment.java b/processing/src/main/java/org/apache/druid/segment/Segment.java index fafd3f500ac4..44be82a56ee7 100644 --- a/processing/src/main/java/org/apache/druid/segment/Segment.java +++ b/processing/src/main/java/org/apache/druid/segment/Segment.java @@ -20,6 +20,8 @@ package org.apache.druid.segment; import org.apache.druid.guice.annotations.PublicApi; +import org.apache.druid.query.datasourcemetadata.DataSourceMetadataResultValue; +import org.apache.druid.segment.join.table.IndexedTable; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -36,11 +38,14 @@ public interface Segment extends Closeable { SegmentId getId(); + Interval getDataInterval(); + @Nullable QueryableIndex asQueryableIndex(); + StorageAdapter asStorageAdapter(); - + /** * Request an implementation of a particular interface. * @@ -49,14 +54,22 @@ public interface Segment extends Closeable * expected to be requested by callers that have specific knowledge of extra features provided by specific * segment types. For example, an extension might provide a custom Segment type that can offer both * StorageAdapter and some new interface. That extension can also offer a Query that uses that new interface. - * - * Implementations which accept classes other than {@link QueryableIndex} or {@link StorageAdapter} are limited - * to using those classes within the extension. This means that one extension cannot rely on the `Segment.as` + * + * Implementations which accept classes other than {@link QueryableIndex} or {@link StorageAdapter} are limited + * to using those classes within the extension. This means that one extension cannot rely on the `Segment.as` * behavior of another extension. * * @param clazz desired interface - * @param desired interface + * @param desired interface + * * @return instance of clazz, or null if the interface is not supported by this segment + * + * @see StorageAdapter storage adapter for queries. Never null. + * @see QueryableIndex index object, if this is a memory-mapped regular segment. + * @see IndexedTable table object, if this is a joinable indexed table. + * @see TimeBoundaryInspector inspector for min/max timestamps, if supported by this segment. + * @see MaxIngestedEventTimeInspector inspector for {@link DataSourceMetadataResultValue#getMaxIngestedEventTime()} + * @see CloseableShapeshifter stepping stone to {@link org.apache.druid.query.rowsandcols.RowsAndColumns}. */ @SuppressWarnings({"unused", "unchecked"}) @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java index 013a634fdc4b..507737a2c5ba 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java @@ -27,6 +27,7 @@ import com.google.common.collect.Maps; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.query.OrderBy; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.Indexed; import org.joda.time.Interval; @@ -102,6 +103,18 @@ public Indexed getAvailableDimensions() return availableDimensions; } + @Override + public List getOrdering() + { + final Metadata metadata = getMetadata(); + if (metadata != null && metadata.getOrdering() != null) { + return metadata.getOrdering(); + } else { + // When sort order isn't set in metadata.drd, assume the segment is sorted by __time. + return Cursors.ascendingTimeOrder(); + } + } + @Override public BitmapFactory getBitmapFactoryForDimensions() { diff --git a/processing/src/main/java/org/apache/druid/segment/SingleScanTimeDimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/SingleScanTimeDimensionSelector.java index 681738360c56..136c4ef271e1 100644 --- a/processing/src/main/java/org/apache/druid/segment/SingleScanTimeDimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/SingleScanTimeDimensionSelector.java @@ -20,6 +20,8 @@ package org.apache.druid.segment; import com.google.common.base.Preconditions; +import org.apache.druid.error.DruidException; +import org.apache.druid.query.Order; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.filter.DruidObjectPredicate; import org.apache.druid.query.filter.DruidPredicateFactory; @@ -43,7 +45,7 @@ public class SingleScanTimeDimensionSelector implements DimensionSelector { private final ExtractionFn extractionFn; private final BaseLongColumnValueSelector selector; - private final boolean descending; + private final Order timeOrder; private final List timeValues = new ArrayList<>(); private final SingleIndexedInt row = new SingleIndexedInt(); @@ -56,13 +58,17 @@ public class SingleScanTimeDimensionSelector implements DimensionSelector public SingleScanTimeDimensionSelector( BaseLongColumnValueSelector selector, @Nullable ExtractionFn extractionFn, - boolean descending + Order timeOrder ) { Preconditions.checkNotNull(extractionFn, "time dimension must provide an extraction function"); this.extractionFn = extractionFn; this.selector = selector; - this.descending = descending; + this.timeOrder = timeOrder; + + if (timeOrder == Order.NONE) { + throw DruidException.defensive("Cannot use timeOrder[%s]", timeOrder); + } } @Override @@ -130,7 +136,7 @@ private int getDimensionValueIndex() // we can also avoid creating a dimension value and corresponding index // and use the current one } else if (timestamp != currentTimestamp) { - if (descending ? timestamp > currentTimestamp : timestamp < currentTimestamp) { + if (timeOrder == Order.ASCENDING ? timestamp < currentTimestamp : timestamp > currentTimestamp) { // re-using this selector for multiple scans would cause the same rows to return different IDs // we might want to re-visit if we ever need to do multiple scans with this dimension selector throw new IllegalStateException("cannot re-use time dimension selector for multiple scans"); @@ -198,6 +204,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("selector", selector); inspector.visit("extractionFn", extractionFn); - inspector.visit("descending", descending); + inspector.visit("timeOrder", timeOrder); } } diff --git a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java index fba947f73d2a..3ad6c9300027 100644 --- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java @@ -20,10 +20,12 @@ package org.apache.druid.segment; import com.google.common.collect.Iterables; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.OrderBy; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.vector.VectorCursor; @@ -35,6 +37,7 @@ import java.util.Optional; /** + * */ @PublicApi public interface StorageAdapter extends CursorFactory, ColumnInspector, CursorHolderFactory @@ -49,7 +52,8 @@ public interface StorageAdapter extends CursorFactory, ColumnInspector, CursorHo @Override default CursorHolder makeCursorHolder(CursorBuildSpec spec) { - // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary + // For backwards compatibility, the default implementation assumes the underlying rows are sorted by __time. + // Built-in implementations of StorageAdapter must override this method. final List ordering; final boolean descending; if (Cursors.preferDescendingTimeOrdering(spec)) { @@ -67,7 +71,7 @@ public boolean canVectorize() return StorageAdapter.this.canVectorize( spec.getFilter(), spec.getVirtualColumns(), - Cursors.preferDescendingTimeOrdering(spec) + descending ); } @@ -115,7 +119,15 @@ public void close() } Interval getInterval(); + + /** + * Returns the names of all dimension columns, not including {@link ColumnHolder#TIME_COLUMN_NAME}. + */ Indexed getAvailableDimensions(); + + /** + * Returns the names of all metric columns. + */ Iterable getAvailableMetrics(); /** @@ -147,28 +159,26 @@ default RowSignature getRowSignature() int getDimensionCardinality(String column); /** - * Metadata-only operation that returns a lower bound on - * {@link org.apache.druid.segment.column.ColumnHolder#TIME_COLUMN_NAME} values for this adapter. May be earlier than - * the actual minimum data timestamp. - * - * For {@link QueryableIndexStorageAdapter} and {@link org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter} - * specifically, which back regular tables (i.e. {@link org.apache.druid.query.TableDataSource}), this method - * contract is tighter: it does return the actual minimum data timestamp. This fact is leveraged by - * {@link org.apache.druid.query.timeboundary.TimeBoundaryQuery} to return results using metadata only. + * Use {@link TimeBoundaryInspector#getMinTime()} instead. */ - DateTime getMinTime(); + @Deprecated + default DateTime getMinTime() + { + throw DruidException.defensive( + "getMinTime is no longer supported, use Segment.as(MinMaxValueInspector.class) instead" + ); + } /** - * Metadata-only operation that returns an upper bound on - * {@link org.apache.druid.segment.column.ColumnHolder#TIME_COLUMN_NAME} values for this adapter. May be later than - * the actual maximum data timestamp. - * - * For {@link QueryableIndexStorageAdapter} and {@link org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter} - * specifically, which back regular tables (i.e. {@link org.apache.druid.query.TableDataSource}), this method - * contract is tighter: it does return the actual maximum data timestamp. This fact is leveraged by - * {@link org.apache.druid.query.timeboundary.TimeBoundaryQuery} to return results using metadata only. + * Use {@link TimeBoundaryInspector#getMaxTime()} instead. */ - DateTime getMaxTime(); + @Deprecated + default DateTime getMaxTime() + { + throw DruidException.defensive( + "getMaxTime is no longer supported, use Segment.as(MinMaxValueInspector.class) instead" + ); + } /** * Returns the minimum value of the provided column, if known through an index, dictionary, or cache. Returns null @@ -203,7 +213,17 @@ default RowSignature getRowSignature() ColumnCapabilities getColumnCapabilities(String column); int getNumRows(); - DateTime getMaxIngestedEventTime(); + + /** + * Use {@link MaxIngestedEventTimeInspector#getMaxIngestedEventTime()} instead. + */ + @Deprecated + default DateTime getMaxIngestedEventTime() + { + throw DruidException.defensive( + "getMaxIngestedEventTime is no longer supported, use Segment.as(MaxIngestedEventTimeInspector.class) instead" + ); + } @Nullable Metadata getMetadata(); diff --git a/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java b/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java index 56eeba56655b..4472ef099ae6 100644 --- a/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java +++ b/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java @@ -20,7 +20,9 @@ package org.apache.druid.segment; import com.google.common.base.Preconditions; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.column.ColumnHolder; import javax.annotation.Nonnull; import java.util.Arrays; @@ -49,6 +51,12 @@ public class TimeAndDimsPointer implements Comparable { final ColumnValueSelector timestampSelector; + /** + * Position of {@link ColumnHolder#TIME_COLUMN_NAME} in the sort order, relative to elements of + * {@link #dimensionSelectors}. For example, for the sort order [x, __time, y], {@link #dimensionSelectors} + * contains [x, y] and timePosition is 1. + */ + final int timePosition; /** * This collection of dimension selectors is stored as array rather than List in order to minimize indirection in hot * spots, in particular in {@link #compareTo}. @@ -76,6 +84,7 @@ public class TimeAndDimsPointer implements Comparable */ TimeAndDimsPointer( ColumnValueSelector timestampSelector, + int timePosition, ColumnValueSelector[] dimensionSelectors, List dimensionHandlers, ColumnValueSelector[] metricSelectors, @@ -83,6 +92,13 @@ public class TimeAndDimsPointer implements Comparable ) { this.timestampSelector = timestampSelector; + this.timePosition = timePosition; + Preconditions.checkArgument( + timePosition >= 0 && timePosition <= dimensionSelectors.length, + "timePosition[%s] is out of range 0 .. [%s]", + timePosition, + dimensionSelectors.length + ); Preconditions.checkArgument(dimensionSelectors.length == dimensionHandlers.size()); this.dimensionSelectors = dimensionSelectors; this.dimensionHandlers = dimensionHandlers; @@ -131,10 +147,23 @@ List getMetricNames() return metricNames; } + /** + * Replace {@link #dimensionSelectors} with a new set of selectors. The new selectors must be the same length. + * Time position ({@link #timePosition}) is unchanged. + */ TimeAndDimsPointer withDimensionSelectors(ColumnValueSelector[] newDimensionSelectors) { + if (dimensionSelectors.length != newDimensionSelectors.length) { + throw DruidException.defensive( + "Lengths of dimensionSelectors[%d] and newDimensionSelectors[%d] do not match", + dimensionSelectors.length, + newDimensionSelectors.length + ); + } + return new TimeAndDimsPointer( timestampSelector, + timePosition, newDimensionSelectors, dimensionHandlers, metricSelectors, @@ -148,19 +177,23 @@ TimeAndDimsPointer withDimensionSelectors(ColumnValueSelector[] newDimensionSele @Override public int compareTo(@Nonnull TimeAndDimsPointer rhs) { - long timestamp = getTimestamp(); - long rhsTimestamp = rhs.getTimestamp(); - int timestampDiff = Long.compare(timestamp, rhsTimestamp); - if (timestampDiff != 0) { - return timestampDiff; - } - for (int dimIndex = 0; dimIndex < dimensionSelectors.length; dimIndex++) { - int dimDiff = dimensionSelectorComparators[dimIndex].compare( - dimensionSelectors[dimIndex], - rhs.dimensionSelectors[dimIndex] - ); - if (dimDiff != 0) { - return dimDiff; + for (int index = 0; index < (dimensionSelectors.length + 1); index++) { + if (index == timePosition) { + final long timestamp = getTimestamp(); + final long rhsTimestamp = rhs.getTimestamp(); + final int timestampDiff = Long.compare(timestamp, rhsTimestamp); + if (timestampDiff != 0) { + return timestampDiff; + } + } else { + final int dimIndex = index < timePosition ? index : index - 1; + final int dimDiff = dimensionSelectorComparators[dimIndex].compare( + dimensionSelectors[dimIndex], + rhs.dimensionSelectors[dimIndex] + ); + if (dimDiff != 0) { + return dimDiff; + } } } return 0; diff --git a/processing/src/main/java/org/apache/druid/segment/TimeBoundaryInspector.java b/processing/src/main/java/org/apache/druid/segment/TimeBoundaryInspector.java new file mode 100644 index 000000000000..00d379696d46 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/TimeBoundaryInspector.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment; + +import org.apache.druid.segment.column.ColumnHolder; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +/** + * Returns min/max values of {@link ColumnHolder#TIME_COLUMN_NAME} for a segment. Typically retrieved from + * {@link Segment#as(Class)}. + * + * Only offered by segments that are able to provide more detailed min/max values via metadata inspection than are + * available from {@link Segment#getDataInterval()}. + */ +public interface TimeBoundaryInspector +{ + /** + * Lower bound on {@link ColumnHolder#TIME_COLUMN_NAME}. Matches the lowest timestamp in the dataset + * if {@link #isMinMaxExact()}. + */ + DateTime getMinTime(); + + /** + * Upper bound on {@link ColumnHolder#TIME_COLUMN_NAME}. Strict if {@link #isMinMaxExact()}. Matches the highest + * timestamp in the dataset if {@link #isMinMaxExact()}. + */ + DateTime getMaxTime(); + + /** + * Smallest interval that contains {@link #getMinTime()} and {@link #getMaxTime()}. The endpoint is one millisecond + * higher than {@link #getMaxTime()}. + */ + default Interval getMinMaxInterval() + { + return new Interval(getMinTime(), getMaxTime().plus(1)); + } + + /** + * Whether the lower and upper bounds returned by {@link #getMinTime()} and {@link #getMaxTime()} are actually + * found in the dataset. If true, the bounds are strict and can be used substitutes for aggregations + * {@code MIN(__time)} and {@code MAX(__time)}. + */ + boolean isMinMaxExact(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index a9f215892d1b..435771b4780b 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -49,7 +49,6 @@ import org.apache.druid.segment.join.PostJoinCursor; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.utils.CloseableUtils; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -150,11 +149,10 @@ public Cursor asCursor() ); } - @Nullable @Override public List getOrdering() { - return cursorHolderSupplier.get().getOrdering(); + return computeOrdering(cursorHolderSupplier.get().getOrdering()); } @Override @@ -204,18 +202,6 @@ public int getDimensionCardinality(String column) return DimensionDictionarySelector.CARDINALITY_UNKNOWN; } - @Override - public DateTime getMinTime() - { - return baseAdapter.getMinTime(); - } - - @Override - public DateTime getMaxTime() - { - return baseAdapter.getMaxTime(); - } - @Nullable @Override public Comparable getMinValue(String column) @@ -255,12 +241,6 @@ public int getNumRows() return 0; } - @Override - public DateTime getMaxIngestedEventTime() - { - return baseAdapter.getMaxIngestedEventTime(); - } - @Nullable @Override public Metadata getMetadata() @@ -579,6 +559,23 @@ private Filter rewriteFilterOnUnnestColumnIfPossible( } } + /** + * Computes ordering of a join {@link CursorHolder} based on the ordering of an underlying {@link CursorHolder}. + */ + private List computeOrdering(final List baseOrdering) + { + // Sorted the same way as the base segment, unless the unnested column shadows one of the base columns. + int limit = 0; + for (; limit < baseOrdering.size(); limit++) { + final String columnName = baseOrdering.get(limit).getColumnName(); + if (columnName.equals(outputColumnName) || columnName.equals(unnestColumn.getOutputName())) { + break; + } + } + + return limit == baseOrdering.size() ? baseOrdering : baseOrdering.subList(0, limit); + } + /** * Computes the capabilities of {@link #outputColumnName}, after unnesting. */ diff --git a/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java b/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java index a327ff6f1f15..19db47067b57 100644 --- a/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java +++ b/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java @@ -23,6 +23,7 @@ import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; @@ -79,6 +80,17 @@ public StorageAdapter asStorageAdapter() return storageAdapterWrapperFunction.apply(delegate.asStorageAdapter()); } + @Nullable + @Override + public T as(@Nonnull Class clazz) + { + if (TimeBoundaryInspector.class.equals(clazz)) { + return (T) WrappedTimeBoundaryInspector.create(delegate.as(TimeBoundaryInspector.class)); + } else { + return SegmentReference.super.as(clazz); + } + } + @Override public void close() throws IOException { diff --git a/processing/src/main/java/org/apache/druid/segment/WrappedTimeBoundaryInspector.java b/processing/src/main/java/org/apache/druid/segment/WrappedTimeBoundaryInspector.java new file mode 100644 index 000000000000..4b2a21eb11b6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/WrappedTimeBoundaryInspector.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment; + +import com.google.common.base.Preconditions; +import org.apache.druid.segment.column.ColumnHolder; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; + +/** + * Wrapper for {@link TimeBoundaryInspector} used by {@link Segment} implementations that may filter out rows + * from an underlying segment, but do not modify {@link ColumnHolder#TIME_COLUMN_NAME}. + */ +public class WrappedTimeBoundaryInspector implements TimeBoundaryInspector +{ + private final TimeBoundaryInspector delegate; + + private WrappedTimeBoundaryInspector(final TimeBoundaryInspector delegate) + { + this.delegate = Preconditions.checkNotNull(delegate, "delegate"); + } + + @Nullable + public static WrappedTimeBoundaryInspector create(@Nullable final TimeBoundaryInspector delegate) + { + if (delegate != null) { + return new WrappedTimeBoundaryInspector(delegate); + } else { + return null; + } + } + + @Override + public DateTime getMinTime() + { + return delegate.getMinTime(); + } + + @Override + public DateTime getMaxTime() + { + return delegate.getMaxTime(); + } + + @Override + public boolean isMinMaxExact() + { + // Always false, because rows may be filtered out. + return false; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java index 5265f4f2c448..cfcc99c10e6c 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java @@ -44,6 +44,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.common.parsers.UnparseableColumnsParseException; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; @@ -93,6 +94,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; /** * In-memory, row-based data structure used to hold data during ingestion. Realtime tasks query this index using @@ -227,6 +229,9 @@ public ColumnCapabilities getColumnCapabilities(String columnName) private final List> rowTransformers; private final VirtualColumns virtualColumns; private final AggregatorFactory[] metrics; + /** + * Metadata to be persisted along with this index, when it is eventually persisted. + */ private final Metadata metadata; protected final boolean preserveExistingMetrics; @@ -234,6 +239,12 @@ public ColumnCapabilities getColumnCapabilities(String columnName) private final DimensionsSpec dimensionsSpec; private final Map dimensionDescs; + /** + * Position of {@link ColumnHolder#TIME_COLUMN_NAME} in the sort order, relative to elements of + * {@link #dimensionDescs}. For example, for the sort order [x, __time, y], dimensionDescs contains [x, y] and + * timePosition is 1. + */ + protected final int timePosition; private final List dimensionDescsList; // dimension capabilities are provided by the indexers private final Map timeAndMetricsColumnCapabilities; @@ -280,13 +291,6 @@ protected IncrementalIndex( this.timeAndMetricsColumnFormats = new HashMap<>(); this.metricDescs = Maps.newLinkedHashMap(); this.dimensionDescs = Maps.newLinkedHashMap(); - this.metadata = new Metadata( - null, - getCombiningAggregators(metrics), - incrementalIndexSchema.getTimestampSpec(), - this.queryGranularity, - this.rollup - ); initAggs(metrics, inputRowHolder); @@ -319,8 +323,24 @@ protected IncrementalIndex( this.dimensionsSpec = incrementalIndexSchema.getDimensionsSpec(); this.dimensionDescsList = new ArrayList<>(); - for (DimensionSchema dimSchema : dimensionsSpec.getDimensions()) { - addNewDimension(dimSchema.getName(), dimSchema.getDimensionHandler()); + + int foundTimePosition = -1; + final List dimSchemas = dimensionsSpec.getDimensions(); + for (int i = 0; i < dimSchemas.size(); i++) { + final DimensionSchema dimSchema = dimSchemas.get(i); + if (ColumnHolder.TIME_COLUMN_NAME.equals(dimSchema.getName())) { + foundTimePosition = i; + } else { + addNewDimension(dimSchema.getName(), dimSchema.getDimensionHandler()); + } + } + + if (foundTimePosition == -1) { + // __time not found: that means it either goes at the end, or the beginning, based on + // forceSegmentSortByTime. + this.timePosition = dimensionsSpec.isForceSegmentSortByTime() ? 0 : dimensionDescsList.size(); + } else { + this.timePosition = foundTimePosition; } //__time capabilities @@ -334,6 +354,16 @@ protected IncrementalIndex( if (!spatialDimensions.isEmpty()) { this.rowTransformers.add(new SpatialDimensionRowTransformer(spatialDimensions)); } + + // Set metadata last, so dimensionOrder is populated + this.metadata = new Metadata( + null, + getCombiningAggregators(metrics), + incrementalIndexSchema.getTimestampSpec(), + this.queryGranularity, + this.rollup, + getDimensionOrder().stream().map(OrderBy::ascending).collect(Collectors.toList()) + ); } public abstract FactsHolder getFacts(); @@ -582,7 +612,7 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) // first, process dimension values present in the row dims = new Object[dimensionDescs.size()]; for (String dimension : rowDimensions) { - if (Strings.isNullOrEmpty(dimension)) { + if (Strings.isNullOrEmpty(dimension) || ColumnHolder.TIME_COLUMN_NAME.equals(dimension)) { continue; } boolean wasNewDim = false; @@ -789,13 +819,38 @@ public DimensionsSpec getDimensionsSpec() return dimensionsSpec; } - public List getDimensionNames() + /** + * Returns names of dimension columns. + * + * @param includeTime whether to include {@link ColumnHolder#TIME_COLUMN_NAME}. + */ + public List getDimensionNames(final boolean includeTime) { synchronized (dimensionDescs) { - return ImmutableList.copyOf(dimensionDescs.keySet()); + if (includeTime) { + final ImmutableList.Builder listBuilder = + ImmutableList.builderWithExpectedSize(dimensionDescs.size() + 1); + int i = 0; + if (i == timePosition) { + listBuilder.add(ColumnHolder.TIME_COLUMN_NAME); + } + for (String dimName : dimensionDescs.keySet()) { + listBuilder.add(dimName); + i++; + if (i == timePosition) { + listBuilder.add(ColumnHolder.TIME_COLUMN_NAME); + } + } + return listBuilder.build(); + } else { + return ImmutableList.copyOf(dimensionDescs.keySet()); + } } } + /** + * Returns a descriptor for each dimension. Does not inclue {@link ColumnHolder#TIME_COLUMN_NAME}. + */ public List getDimensions() { synchronized (dimensionDescs) { @@ -803,6 +858,9 @@ public List getDimensions() } } + /** + * Returns the descriptor for a particular dimension. + */ @Nullable public DimensionDesc getDimension(String dimension) { @@ -859,11 +917,12 @@ public Integer getDimensionIndex(String dimension) return dimSpec == null ? null : dimSpec.getIndex(); } + /** + * Returns names of time and dimension columns, in persist sort order. Includes {@link ColumnHolder#TIME_COLUMN_NAME}. + */ public List getDimensionOrder() { - synchronized (dimensionDescs) { - return ImmutableList.copyOf(dimensionDescs.keySet()); - } + return getDimensionNames(true); } public static ColumnCapabilitiesImpl makeDefaultCapabilitiesFromValueType(ColumnType type) @@ -887,19 +946,23 @@ public static ColumnCapabilitiesImpl makeDefaultCapabilitiesFromValueType(Column * Currently called to initialize IncrementalIndex dimension order during index creation * Index dimension ordering could be changed to initialize from DimensionsSpec after resolution of * https://github.com/apache/druid/issues/2011 + * + * @param oldDimensionOrder dimension order to initialize + * @param oldColumnFormats formats for the dimensions */ public void loadDimensionIterable( Iterable oldDimensionOrder, - Map oldColumnCapabilities + Map oldColumnFormats ) { synchronized (dimensionDescs) { - if (!dimensionDescs.isEmpty()) { - throw new ISE("Cannot load dimension order when existing order[%s] is not empty.", dimensionDescs.keySet()); + if (size() != 0) { + throw new ISE("Cannot load dimension order[%s] when existing index is not empty.", dimensionDescs.keySet()); } for (String dim : oldDimensionOrder) { - if (dimensionDescs.get(dim) == null) { - ColumnFormat format = oldColumnCapabilities.get(dim); + // Skip __time; its position is solely based on configuration at index creation time. + if (!ColumnHolder.TIME_COLUMN_NAME.equals(dim) && dimensionDescs.get(dim) == null) { + ColumnFormat format = oldColumnFormats.get(dim); addNewDimension(dim, format.getColumnHandler(dim)); } } @@ -925,9 +988,12 @@ public List getMetricNames() return ImmutableList.copyOf(metricDescs.keySet()); } + /** + * Returns all column names, including {@link ColumnHolder#TIME_COLUMN_NAME}. + */ public List getColumnNames() { - List columnNames = new ArrayList<>(getDimensionNames()); + List columnNames = new ArrayList<>(getDimensionNames(true)); columnNames.addAll(getMetricNames()); return columnNames; } @@ -1068,33 +1134,48 @@ protected ColumnSelectorFactory makeColumnSelectorFactory( protected final Comparator dimsComparator() { - return new IncrementalIndexRowComparator(dimensionDescsList); + return new IncrementalIndexRowComparator(timePosition, dimensionDescsList); } @VisibleForTesting static final class IncrementalIndexRowComparator implements Comparator { - private List dimensionDescs; + /** + * Position of {@link ColumnHolder#TIME_COLUMN_NAME} in the sort order. + */ + private final int timePosition; + private final List dimensionDescs; - public IncrementalIndexRowComparator(List dimDescs) + public IncrementalIndexRowComparator(int timePosition, List dimDescs) { + this.timePosition = timePosition; this.dimensionDescs = dimDescs; } @Override public int compare(IncrementalIndexRow lhs, IncrementalIndexRow rhs) { - int retVal = Longs.compare(lhs.timestamp, rhs.timestamp); - int numComparisons = Math.min(lhs.dims.length, rhs.dims.length); + int retVal = 0; + + // Number of dimension comparisons, not counting __time. + int numDimComparisons = Math.min(lhs.dims.length, rhs.dims.length); - int index = 0; - while (retVal == 0 && index < numComparisons) { - final Object lhsIdxs = lhs.dims[index]; - final Object rhsIdxs = rhs.dims[index]; + int dimIndex = 0; + while (retVal == 0 && dimIndex < numDimComparisons) { + if (dimIndex == timePosition) { + retVal = Longs.compare(lhs.timestamp, rhs.timestamp); + + if (retVal != 0) { + break; + } + } + + final Object lhsIdxs = lhs.dims[dimIndex]; + final Object rhsIdxs = rhs.dims[dimIndex]; if (lhsIdxs == null) { if (rhsIdxs == null) { - ++index; + ++dimIndex; continue; } return -1; @@ -1104,9 +1185,13 @@ public int compare(IncrementalIndexRow lhs, IncrementalIndexRow rhs) return 1; } - final DimensionIndexer indexer = dimensionDescs.get(index).getIndexer(); + final DimensionIndexer indexer = dimensionDescs.get(dimIndex).getIndexer(); retVal = indexer.compareUnsortedEncodedKeyComponents(lhsIdxs, rhsIdxs); - ++index; + ++dimIndex; + } + + if (retVal == 0 && dimIndex == numDimComparisons && timePosition >= numDimComparisons) { + retVal = Longs.compare(lhs.timestamp, rhs.timestamp); } if (retVal == 0) { @@ -1115,7 +1200,7 @@ public int compare(IncrementalIndexRow lhs, IncrementalIndexRow rhs) return 0; } Object[] largerDims = lengthDiff > 0 ? lhs.dims : rhs.dims; - return allNull(largerDims, numComparisons) ? 0 : lengthDiff; + return allNull(largerDims, numDimComparisons) ? 0 : lengthDiff; } return retVal; diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java index a61ff334bb9e..aae1c1335753 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java @@ -116,9 +116,9 @@ public int getNumRows() } @Override - public List getDimensionNames() + public List getDimensionNames(final boolean includeTime) { - return index.getDimensionNames(); + return index.getDimensionNames(includeTime); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java index 45d081c8b916..f43b81c40adc 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.incremental; +import org.apache.druid.query.Order; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.segment.ColumnSelectorFactory; @@ -43,20 +44,20 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro private final IncrementalIndexStorageAdapter adapter; private final IncrementalIndex index; private final VirtualColumns virtualColumns; - private final boolean descending; + private final Order timeOrder; private final IncrementalIndexRowHolder rowHolder; IncrementalIndexColumnSelectorFactory( IncrementalIndexStorageAdapter adapter, VirtualColumns virtualColumns, - boolean descending, + Order timeOrder, IncrementalIndexRowHolder rowHolder ) { this.adapter = adapter; this.index = adapter.index; this.virtualColumns = virtualColumns; - this.descending = descending; + this.timeOrder = timeOrder; this.rowHolder = rowHolder; } @@ -75,8 +76,12 @@ private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensi final String dimension = dimensionSpec.getDimension(); final ExtractionFn extractionFn = dimensionSpec.getExtractionFn(); - if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME)) { - return new SingleScanTimeDimensionSelector(makeColumnValueSelector(dimension), extractionFn, descending); + if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME) && timeOrder != Order.NONE) { + return new SingleScanTimeDimensionSelector( + makeColumnValueSelector(dimension), + extractionFn, + timeOrder + ); } final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimensionSpec.getDimension()); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java index f0966a828eb9..33c08d5920cb 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -21,6 +21,7 @@ import com.google.common.collect.Iterators; import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; @@ -34,6 +35,7 @@ import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -43,7 +45,6 @@ public class IncrementalIndexCursorHolder implements CursorHolder private final IncrementalIndex index; private final CursorBuildSpec spec; private final List ordering; - private final boolean descending; public IncrementalIndexCursorHolder( IncrementalIndexStorageAdapter storageAdapter, @@ -54,13 +55,16 @@ public IncrementalIndexCursorHolder( this.storageAdapter = storageAdapter; this.index = index; this.spec = spec; - // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary - if (Cursors.preferDescendingTimeOrdering(spec)) { - this.ordering = Cursors.descendingTimeOrder(); - this.descending = true; + if (index.timePosition == 0) { + if (Cursors.preferDescendingTimeOrdering(spec)) { + this.ordering = Cursors.descendingTimeOrder(); + } else { + this.ordering = Cursors.ascendingTimeOrder(); + } } else { - this.ordering = Cursors.ascendingTimeOrder(); - this.descending = false; + // In principle, we could report a sort order here for certain types of fact holders; for example the + // RollupFactsHolder would be sorted by dimensions. However, this is left for future work. + this.ordering = Collections.emptyList(); } } @@ -80,7 +84,7 @@ public Cursor asCursor() storageAdapter, index, spec.getVirtualColumns(), - descending, + Cursors.getTimeOrdering(ordering), spec.getFilter(), spec.getInterval() ); @@ -109,7 +113,7 @@ static class IncrementalIndexCursor implements Cursor IncrementalIndexStorageAdapter storageAdapter, IncrementalIndex index, VirtualColumns virtualColumns, - boolean descending, + Order timeOrder, @Nullable Filter filter, Interval actualInterval ) @@ -118,7 +122,7 @@ static class IncrementalIndexCursor implements Cursor columnSelectorFactory = new IncrementalIndexColumnSelectorFactory( storageAdapter, virtualColumns, - descending, + timeOrder, currEntry ); // Set maxRowIndex before creating the filterMatcher. See https://github.com/apache/druid/pull/6340 @@ -127,7 +131,7 @@ static class IncrementalIndexCursor implements Cursor numAdvanced = -1; facts = index.getFacts(); cursorIterable = facts.timeRangeIterable( - descending, + timeOrder == Order.DESCENDING, actualInterval.getStartMillis(), actualInterval.getEndMillis() ); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java index 70a589f39505..89e94961f6b2 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java @@ -142,7 +142,7 @@ public String toString() @Override public Object apply(@Nullable Object input) { - if (input == null || Array.getLength(input) == 0) { + if (input == null || (input.getClass().isArray() && Array.getLength(input) == 0)) { return Collections.singletonList("null"); } return Collections.singletonList(input); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java index 98ad5c75109f..0e7e82f01c3e 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.incremental; +import org.apache.druid.query.Order; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionHandler; @@ -67,7 +68,7 @@ private static RowPointer makeRowPointer( new IncrementalIndexColumnSelectorFactory( new IncrementalIndexStorageAdapter(incrementalIndex), VirtualColumns.EMPTY, - false, + incrementalIndex.timePosition == 0 ? Order.ASCENDING : Order.NONE, rowHolder ); ColumnValueSelector[] dimensionSelectors = incrementalIndex @@ -91,6 +92,7 @@ private static RowPointer makeRowPointer( return new RowPointer( rowHolder, + incrementalIndex.timePosition, dimensionSelectors, dimensionHandlers, metricSelectors, diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index ad55e33adaef..ab92a00897b9 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -32,12 +32,12 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ListIndexed; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; /** + * */ public class IncrementalIndexStorageAdapter implements StorageAdapter { @@ -125,7 +125,7 @@ public Interval getInterval() @Override public Indexed getAvailableDimensions() { - return new ListIndexed<>(index.getDimensionNames()); + return new ListIndexed<>(index.getDimensionNames(false)); } @Override @@ -155,18 +155,6 @@ public int getNumRows() return index.size(); } - @Override - public DateTime getMinTime() - { - return index.getMinTime(); - } - - @Override - public DateTime getMaxTime() - { - return index.getMaxTime(); - } - @Nullable @Override public Comparable getMinValue(String column) @@ -236,12 +224,6 @@ public ColumnCapabilities getSnapshotColumnCapabilities(String column) ); } - @Override - public DateTime getMaxIngestedEventTime() - { - return index.getMaxIngestedEventTime(); - } - @Override public CursorHolder makeCursorHolder(CursorBuildSpec spec) { diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java index 0103dc7b729a..b5e580f44f2f 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java @@ -21,11 +21,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Maps; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; @@ -46,6 +48,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collection; import java.util.Comparator; @@ -145,8 +148,13 @@ public class OnheapIncrementalIndex extends IncrementalIndex super(incrementalIndexSchema, preserveExistingMetrics, useMaxMemoryEstimates); this.maxRowCount = maxRowCount; this.maxBytesInMemory = maxBytesInMemory == 0 ? Long.MAX_VALUE : maxBytesInMemory; - this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(dimsComparator(), getDimensions()) - : new PlainFactsHolder(dimsComparator()); + if (incrementalIndexSchema.isRollup()) { + this.facts = new RollupFactsHolder(dimsComparator(), getDimensions(), timePosition == 0); + } else if (timePosition == 0) { + this.facts = new PlainTimeOrderedFactsHolder(dimsComparator()); + } else { + this.facts = new PlainNonTimeOrderedFactsHolder(dimsComparator()); + } maxBytesPerRowForAggregators = useMaxMemoryEstimates ? getMaxBytesPerRowForAggregators(incrementalIndexSchema) : 0; this.useMaxMemoryEstimates = useMaxMemoryEstimates; @@ -734,14 +742,19 @@ static final class RollupFactsHolder implements FactsHolder // Can't use Set because we need to be able to get from collection private final ConcurrentNavigableMap facts; private final List dimensionDescsList; + private final boolean timeOrdered; + private volatile long minTime = DateTimes.MAX.getMillis(); + private volatile long maxTime = DateTimes.MIN.getMillis(); RollupFactsHolder( Comparator incrementalIndexRowComparator, - List dimensionDescsList + List dimensionDescsList, + boolean timeOrdered ) { this.facts = new ConcurrentSkipListMap<>(incrementalIndexRowComparator); this.dimensionDescsList = dimensionDescsList; + this.timeOrdered = timeOrdered; } @Override @@ -754,13 +767,13 @@ public int getPriorIndex(IncrementalIndexRow key) @Override public long getMinTimeMillis() { - return facts.firstKey().getTimestamp(); + return minTime; } @Override public long getMaxTimeMillis() { - return facts.lastKey().getTimestamp(); + return maxTime; } @Override @@ -777,11 +790,18 @@ public Iterator iterator(boolean descending) @Override public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) { - IncrementalIndexRow start = new IncrementalIndexRow(timeStart, new Object[]{}, dimensionDescsList); - IncrementalIndexRow end = new IncrementalIndexRow(timeEnd, new Object[]{}, dimensionDescsList); - ConcurrentNavigableMap subMap = facts.subMap(start, end); - ConcurrentMap rangeMap = descending ? subMap.descendingMap() : subMap; - return rangeMap.keySet(); + if (timeOrdered) { + IncrementalIndexRow start = new IncrementalIndexRow(timeStart, new Object[]{}, dimensionDescsList); + IncrementalIndexRow end = new IncrementalIndexRow(timeEnd, new Object[]{}, dimensionDescsList); + ConcurrentNavigableMap subMap = facts.subMap(start, end); + ConcurrentMap rangeMap = descending ? subMap.descendingMap() : subMap; + return rangeMap.keySet(); + } else { + return Iterables.filter( + facts.keySet(), + row -> row.timestamp >= timeStart && row.timestamp < timeEnd + ); + } } @Override @@ -802,6 +822,8 @@ public int putIfAbsent(IncrementalIndexRow key, int rowIndex) { // setRowIndex() must be called before facts.putIfAbsent() for visibility of rowIndex from concurrent readers. key.setRowIndex(rowIndex); + minTime = Math.min(minTime, key.timestamp); + maxTime = Math.max(maxTime, key.timestamp); IncrementalIndexRow prev = facts.putIfAbsent(key, key); return prev == null ? IncrementalIndexRow.EMPTY_ROW_INDEX : prev.getRowIndex(); } @@ -813,13 +835,13 @@ public void clear() } } - static final class PlainFactsHolder implements FactsHolder + static final class PlainTimeOrderedFactsHolder implements FactsHolder { private final ConcurrentNavigableMap> facts; private final Comparator incrementalIndexRowComparator; - public PlainFactsHolder(Comparator incrementalIndexRowComparator) + public PlainTimeOrderedFactsHolder(Comparator incrementalIndexRowComparator) { this.facts = new ConcurrentSkipListMap<>(); this.incrementalIndexRowComparator = incrementalIndexRowComparator; @@ -918,4 +940,84 @@ public void clear() facts.clear(); } } + + static final class PlainNonTimeOrderedFactsHolder implements FactsHolder + { + private final Deque facts; + private final Comparator incrementalIndexRowComparator; + private volatile long minTime = DateTimes.MAX.getMillis(); + private volatile long maxTime = DateTimes.MIN.getMillis(); + + public PlainNonTimeOrderedFactsHolder(Comparator incrementalIndexRowComparator) + { + this.facts = new ArrayDeque<>(); + this.incrementalIndexRowComparator = incrementalIndexRowComparator; + } + + @Override + public int getPriorIndex(IncrementalIndexRow key) + { + // always return EMPTY_ROW_INDEX to indicate that no prior key cause we always add new row + return IncrementalIndexRow.EMPTY_ROW_INDEX; + } + + @Override + public long getMinTimeMillis() + { + return minTime; + } + + @Override + public long getMaxTimeMillis() + { + return maxTime; + } + + @Override + public Iterator iterator(boolean descending) + { + return descending ? facts.descendingIterator() : facts.iterator(); + } + + @Override + public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) + { + return Iterables.filter( + () -> iterator(descending), + row -> row.timestamp >= timeStart && row.timestamp < timeEnd + ); + } + + @Override + public Iterable keySet() + { + return facts; + } + + @Override + public Iterable persistIterable() + { + final List sortedFacts = new ArrayList<>(facts); + sortedFacts.sort(incrementalIndexRowComparator); + return sortedFacts; + } + + @Override + public int putIfAbsent(IncrementalIndexRow key, int rowIndex) + { + // setRowIndex() must be called before rows.add() for visibility of rowIndex from concurrent readers. + key.setRowIndex(rowIndex); + minTime = Math.min(minTime, key.timestamp); + maxTime = Math.max(maxTime, key.timestamp); + facts.add(key); + // always return EMPTY_ROW_INDEX to indicate that we always add new row + return IncrementalIndexRow.EMPTY_ROW_INDEX; + } + + @Override + public void clear() + { + facts.clear(); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index 6567baa140d7..89356a9c9cb5 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -55,7 +55,6 @@ private HashJoinEngine() public static Cursor makeJoinCursor( final Cursor leftCursor, final JoinableClause joinableClause, - final boolean descending, final Closer closer ) { @@ -65,7 +64,6 @@ public static Cursor makeJoinCursor( leftColumnSelectorFactory, joinableClause.getCondition(), joinableClause.getJoinType().isRighty(), - descending, closer ); diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java index cdcc3ad8d6b9..0cf54b8c1484 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java @@ -28,6 +28,8 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; +import org.apache.druid.segment.WrappedTimeBoundaryInspector; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.timeline.SegmentId; import org.apache.druid.utils.CloseableUtils; @@ -158,7 +160,10 @@ public T as(Class clazz) { if (CloseableShapeshifter.class.equals(clazz)) { return (T) new StorageAdapterRowsAndColumns(this.asStorageAdapter()); + } else if (TimeBoundaryInspector.class.equals(clazz)) { + return (T) WrappedTimeBoundaryInspector.create(baseSegment.as(TimeBoundaryInspector.class)); + } else { + return SegmentReference.super.as(clazz); } - return SegmentReference.super.as(clazz); } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 1276a60523f3..f3668ca45e14 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -29,7 +29,6 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.Cursors; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -42,7 +41,6 @@ import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey; import org.apache.druid.segment.join.filter.JoinFilterSplit; import org.apache.druid.utils.CloseableUtils; -import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -135,18 +133,6 @@ public int getDimensionCardinality(String column) } } - @Override - public DateTime getMinTime() - { - return baseAdapter.getMinTime(); - } - - @Override - public DateTime getMaxTime() - { - return baseAdapter.getMaxTime(); - } - @Nullable @Override public Comparable getMinValue(String column) @@ -192,12 +178,6 @@ public int getNumRows() throw new UnsupportedOperationException("Cannot retrieve number of rows from join segment"); } - @Override - public DateTime getMaxIngestedEventTime() - { - return baseAdapter.getMaxIngestedEventTime(); - } - @Override public Metadata getMetadata() { @@ -232,23 +212,27 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec) return baseAdapter.makeCursorHolder(newSpec); } - // adequate for time ordering, but needs to be updated if we support cursors ordered other time as the primary - final List ordering; - final boolean descending; - if (Cursors.preferDescendingTimeOrdering(spec)) { - ordering = Cursors.descendingTimeOrder(); - descending = true; - } else { - ordering = Cursors.ascendingTimeOrder(); - descending = false; - } - return new CursorHolder() { final Closer joinablesCloser = Closer.create(); - @Override - public Cursor asCursor() + /** + * Typically the same as {@link HashJoinSegmentStorageAdapter#joinFilterPreAnalysis}, but may differ when + * an unnest datasource is layered on top of a join datasource. + */ + final JoinFilterPreAnalysis actualPreAnalysis; + + /** + * Result of {@link JoinFilterAnalyzer#splitFilter} on {@link #actualPreAnalysis} and + * {@link HashJoinSegmentStorageAdapter#baseFilter}. + */ + final JoinFilterSplit joinFilterSplit; + + /** + * Cursor holder for {@link HashJoinSegmentStorageAdapter#baseAdapter}. + */ + final CursorHolder baseCursorHolder; + { // Filter pre-analysis key implied by the call to "makeCursorHolder". We need to sanity-check that it matches // the actual pre-analysis that was done. Note: we could now infer a rewrite config from the "makeCursorHolder" @@ -264,23 +248,21 @@ public Cursor asCursor() ); final JoinFilterPreAnalysisKey keyCached = joinFilterPreAnalysis.getKey(); - final JoinFilterPreAnalysis preAnalysis; if (keyIn.equals(keyCached)) { // Common case: key used during filter pre-analysis (keyCached) matches key implied by makeCursorHolder call // (keyIn). - preAnalysis = joinFilterPreAnalysis; + actualPreAnalysis = joinFilterPreAnalysis; } else { // Less common case: key differs. Re-analyze the filter. This case can happen when an unnest datasource is // layered on top of a join datasource. - preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis(keyIn); + actualPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis(keyIn); } - final JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter( - preAnalysis, + joinFilterSplit = JoinFilterAnalyzer.splitFilter( + actualPreAnalysis, baseFilter ); - if (joinFilterSplit.getBaseTableFilter().isPresent()) { cursorBuildSpecBuilder.setFilter(joinFilterSplit.getBaseTableFilter().get()); } @@ -295,8 +277,14 @@ public Cursor asCursor() ); cursorBuildSpecBuilder.setVirtualColumns(preJoinVirtualColumns); - final Cursor baseCursor = joinablesCloser.register(baseAdapter.makeCursorHolder(cursorBuildSpecBuilder.build())) - .asCursor(); + baseCursorHolder = + joinablesCloser.register(baseAdapter.makeCursorHolder(cursorBuildSpecBuilder.build())); + } + + @Override + public Cursor asCursor() + { + final Cursor baseCursor = baseCursorHolder.asCursor(); if (baseCursor == null) { return null; @@ -305,21 +293,20 @@ public Cursor asCursor() Cursor retVal = baseCursor; for (JoinableClause clause : clauses) { - retVal = HashJoinEngine.makeJoinCursor(retVal, clause, descending, joinablesCloser); + retVal = HashJoinEngine.makeJoinCursor(retVal, clause, joinablesCloser); } return PostJoinCursor.wrap( retVal, - VirtualColumns.fromIterable(preAnalysis.getPostJoinVirtualColumns()), + VirtualColumns.fromIterable(actualPreAnalysis.getPostJoinVirtualColumns()), joinFilterSplit.getJoinTableFilter().orElse(null) ); } - @Nullable @Override public List getOrdering() { - return ordering; + return computeOrdering(baseCursorHolder.getOrdering()); } @Override @@ -367,4 +354,20 @@ private Filter baseFilterAnd(@Nullable final Filter other) { return Filters.maybeAnd(Arrays.asList(baseFilter, other)).orElse(null); } + + /** + * Computes ordering of a join {@link CursorHolder} based on the ordering of an underlying {@link CursorHolder}. + */ + private List computeOrdering(final List baseOrdering) + { + // Sorted the same way as the base segment, unless a joined-in column shadows one of the base columns. + int limit = 0; + for (; limit < baseOrdering.size(); limit++) { + if (!isBaseColumn(baseOrdering.get(limit).getColumnName())) { + break; + } + } + + return limit == baseOrdering.size() ? baseOrdering : baseOrdering.subList(0, limit); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/Joinable.java b/processing/src/main/java/org/apache/druid/segment/join/Joinable.java index dab102d44932..6886065f1a6e 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/Joinable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/Joinable.java @@ -72,7 +72,6 @@ public interface Joinable extends ReferenceCountedObject * @param condition join condition for the matcher * @param remainderNeeded whether or not {@link JoinMatcher#matchRemainder()} will ever be called on the * matcher. If we know it will not, additional optimizations are often possible. - * @param descending true if join cursor is iterated in descending order * @param closer closer that will run after join cursor has completed to clean up any per query * resources the joinable uses * @@ -82,7 +81,6 @@ JoinMatcher makeJoinMatcher( ColumnSelectorFactory leftColumnSelectorFactory, JoinConditionAnalysis condition, boolean remainderNeeded, - boolean descending, Closer closer ); diff --git a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java index 638506e5901b..f0c9a34526d0 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java @@ -90,7 +90,6 @@ public JoinMatcher makeJoinMatcher( final ColumnSelectorFactory leftSelectorFactory, final JoinConditionAnalysis condition, final boolean remainderNeeded, - boolean descending, Closer closer ) { diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java index 716decb04d99..9591dc315f0c 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java @@ -30,6 +30,7 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.Cursors; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; @@ -222,11 +223,11 @@ public void close() throws IOException @Nullable @Override - public ColumnSelectorFactory makeColumnSelectorFactory(ReadableOffset offset, boolean descending, Closer closer) + public ColumnSelectorFactory makeColumnSelectorFactory(ReadableOffset offset, Closer closer) { return new QueryableIndexColumnSelectorFactory( VirtualColumns.EMPTY, - descending, + Cursors.getTimeOrdering(queryableIndex.getOrdering()), offset, new ColumnCache(queryableIndex, closer) ); diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java index 221fa67fc0f7..e2c01cd7fb27 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java @@ -85,7 +85,7 @@ public interface IndexedTable extends ReferenceCountedObject, Closeable * instead. */ @Nullable - default ColumnSelectorFactory makeColumnSelectorFactory(ReadableOffset offset, boolean descending, Closer closer) + default ColumnSelectorFactory makeColumnSelectorFactory(ReadableOffset offset, Closer closer) { return null; } diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java index f96e4260f8e3..3f3c75770f0b 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java @@ -46,7 +46,6 @@ import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.SimpleAscendingOffset; -import org.apache.druid.segment.SimpleDescendingOffset; import org.apache.druid.segment.SimpleSettableOffset; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; @@ -95,16 +94,11 @@ public class IndexedTableJoinMatcher implements JoinMatcher final ColumnSelectorFactory leftSelectorFactory, final JoinConditionAnalysis condition, final boolean remainderNeeded, - final boolean descending, final Closer closer ) { this.table = table; - if (descending) { - this.joinableOffset = new SimpleDescendingOffset(table.numRows()); - } else { - this.joinableOffset = new SimpleAscendingOffset(table.numRows()); - } + this.joinableOffset = new SimpleAscendingOffset(table.numRows()); reset(); if (condition.isAlwaysTrue()) { @@ -133,7 +127,7 @@ public class IndexedTableJoinMatcher implements JoinMatcher ); } - ColumnSelectorFactory selectorFactory = table.makeColumnSelectorFactory(joinableOffset, descending, closer); + ColumnSelectorFactory selectorFactory = table.makeColumnSelectorFactory(joinableOffset, closer); this.selectorFactory = selectorFactory != null ? selectorFactory : new IndexedTableColumnSelectorFactory(table, () -> currentRow, closer); diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java index 4e9c5b5b3524..10384b373a6b 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java @@ -77,7 +77,6 @@ public JoinMatcher makeJoinMatcher( final ColumnSelectorFactory leftColumnSelectorFactory, final JoinConditionAnalysis condition, final boolean remainderNeeded, - boolean descending, Closer closer ) { @@ -86,7 +85,6 @@ public JoinMatcher makeJoinMatcher( leftColumnSelectorFactory, condition, remainderNeeded, - descending, closer ); } diff --git a/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java b/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java index 72958e8049d3..bb6ad8aaca97 100644 --- a/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java @@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.query.OrderBy; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.QueryableIndex; @@ -36,7 +37,6 @@ import org.joda.time.Interval; import javax.annotation.Nullable; - import java.io.File; import java.util.List; import java.util.Map; @@ -102,6 +102,12 @@ public Map getDimensionHandlers() throw new UnsupportedOperationException(); } + @Override + public List getOrdering() + { + throw new UnsupportedOperationException(); + } + @Override public void close() { 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 979d213ee411..5aac2a2a9e8b 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 @@ -205,10 +205,14 @@ public static ColumnValueSelector makeExprEvalSelector( final String column = plan.getSingleInputName(); final ColumnType inputType = plan.getSingleInputType(); if (inputType.is(ValueType.LONG)) { + // Skip LRU cache when the underlying data is sorted by __time. Note: data is not always sorted by __time; when + // forceSegmentSortByTime: false, segments can be written in non-__time order. However, this + // information is not currently available here, so we assume the common case, which is __time-sortedness. + final boolean useLruCache = !ColumnHolder.TIME_COLUMN_NAME.equals(column); return new SingleLongInputCachingExpressionColumnValueSelector( columnSelectorFactory.makeColumnValueSelector(column), plan.getExpression(), - !ColumnHolder.TIME_COLUMN_NAME.equals(column), // __time doesn't need an LRU cache since it is sorted. + useLruCache, rowIdSupplier ); } else if (inputType.is(ValueType.STRING)) { diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java index b9abf2f81b14..2a8f525bf781 100644 --- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java @@ -53,9 +53,11 @@ import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.hamcrest.CoreMatchers; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -205,18 +207,6 @@ public void test_getColumnCapabilities_unknownColumn() Assert.assertNull(frameAdapter.getColumnCapabilities("nonexistent")); } - @Test - public void test_getMinTime() - { - Assert.assertEquals(queryableAdapter.getInterval().getStart(), frameAdapter.getMinTime()); - } - - @Test - public void test_getMaxTime() - { - Assert.assertEquals(queryableAdapter.getInterval().getEnd().minus(1), frameAdapter.getMaxTime()); - } - @Test public void test_getNumRows() { @@ -267,13 +257,14 @@ public CursorTests( this.interval = interval; this.virtualColumns = virtualColumns; this.descending = descending; - this.buildSpec = CursorBuildSpec.builder() - .setFilter(this.filter) - .setInterval(this.interval) - .setVirtualColumns(this.virtualColumns) - .setPreferredOrdering(descending ? Cursors.descendingTimeOrder() : null) - .setQueryContext(queryContext) - .build(); + this.buildSpec = + CursorBuildSpec.builder() + .setFilter(this.filter) + .setInterval(this.interval) + .setVirtualColumns(this.virtualColumns) + .setPreferredOrdering(descending ? Cursors.descendingTimeOrder() : Collections.emptyList()) + .setQueryContext(queryContext) + .build(); } @Parameterized.Parameters(name = "frameType = {0}, " @@ -363,44 +354,49 @@ public void tearDown() @Test public void test_makeCursor() { - assertCursorMatch(adapter -> adapter.makeCursorHolder(buildSpec)); - } + final RowSignature signature = frameAdapter.getRowSignature(); - @Test - public void test_makeVectorCursor() - { - assertVectorCursorsMatch(adapter -> adapter.makeCursorHolder(buildSpec)); - } + // Frame adapters don't know the order of the underlying frames, so they should ignore the "preferred ordering" + // of the cursor build spec. We test this by passing the frameAdapter a build spec with a preferred ordering, + // and passing the queryableAdapter the same build spec *without* a preferred ordering, and verifying they match. + final CursorBuildSpec queryableBuildSpec = + CursorBuildSpec.builder(buildSpec).setPreferredOrdering(Collections.emptyList()).build(); - private void assertCursorMatch(final Function call) - { - final RowSignature signature = frameAdapter.getRowSignature(); - try (final CursorHolder queryableMaker = call.apply(queryableAdapter); - final CursorHolder frameMaker = call.apply(frameAdapter)) { + try (final CursorHolder queryableCursorHolder = queryableAdapter.makeCursorHolder(queryableBuildSpec); + final CursorHolder frameCursorHolder = frameAdapter.makeCursorHolder(buildSpec)) { final Sequence> queryableRows = - FrameTestUtil.readRowsFromCursor(advanceAndReset(queryableMaker.asCursor()), signature); + FrameTestUtil.readRowsFromCursor(advanceAndReset(queryableCursorHolder.asCursor()), signature); final Sequence> frameRows = - FrameTestUtil.readRowsFromCursor(advanceAndReset(frameMaker.asCursor()), signature); + FrameTestUtil.readRowsFromCursor(advanceAndReset(frameCursorHolder.asCursor()), signature); FrameTestUtil.assertRowsEqual(queryableRows, frameRows); } } + @Test + public void test_makeVectorCursor() + { + // Conditions for frames to be vectorizable. + Assume.assumeThat(frameType, CoreMatchers.equalTo(FrameType.COLUMNAR)); + Assume.assumeFalse(descending); + assertVectorCursorsMatch(adapter -> adapter.makeCursorHolder(buildSpec)); + } + private void assertVectorCursorsMatch(final Function call) { final CursorHolder cursorHolder = call.apply(queryableAdapter); final CursorHolder frameCursorHolder = call.apply(frameAdapter); - if (frameCursorHolder.canVectorize()) { - final RowSignature signature = frameAdapter.getRowSignature(); - final Sequence> queryableRows = - FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(cursorHolder.asVectorCursor()), signature).withBaggage(cursorHolder); - final Sequence> frameRows = - FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(frameCursorHolder.asVectorCursor()), signature) - .withBaggage(frameCursorHolder); - FrameTestUtil.assertRowsEqual(queryableRows, frameRows); - } else { - cursorHolder.close(); - frameCursorHolder.close(); - } + + Assert.assertTrue("queryable cursor is vectorizable", cursorHolder.canVectorize()); + Assert.assertTrue("frame cursor is vectorizable", frameCursorHolder.canVectorize()); + + final RowSignature signature = frameAdapter.getRowSignature(); + final Sequence> queryableRows = + FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(cursorHolder.asVectorCursor()), signature) + .withBaggage(cursorHolder); + final Sequence> frameRows = + FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(frameCursorHolder.asVectorCursor()), signature) + .withBaggage(frameCursorHolder); + FrameTestUtil.assertRowsEqual(queryableRows, frameRows); } /** diff --git a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java index 8aeaf6260705..75119bfb8979 100644 --- a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java +++ b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java @@ -33,8 +33,11 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -55,6 +58,7 @@ public class CursorGranularizerTest extends InitializedNullHandlingTest public final TemporaryFolder temporaryFolder = new TemporaryFolder(); private StorageAdapter adapter; + private TimeBoundaryInspector timeBoundaryInspector; @Before public void setup() throws IOException @@ -147,7 +151,9 @@ public void setup() throws IOException ) .tmpDir(temporaryFolder.newFolder()); - adapter = new QueryableIndexStorageAdapter(bob.buildMMappedIndex()); + final QueryableIndex index = bob.buildMMappedIndex(); + adapter = new QueryableIndexStorageAdapter(index); + timeBoundaryInspector = QueryableIndexTimeBoundaryInspector.create(index); } @Test @@ -156,11 +162,11 @@ public void testGranularizeFullScan() try (CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); CursorGranularizer granularizer = CursorGranularizer.create( - adapter, cursor, + timeBoundaryInspector, + Order.ASCENDING, Granularities.HOUR, - adapter.getInterval(), - false + adapter.getInterval() ); final ColumnSelectorFactory selectorFactory = cursor.getColumnSelectorFactory(); @@ -209,11 +215,11 @@ public void testGranularizeFullScanDescending() try (CursorHolder cursorHolder = adapter.makeCursorHolder(descending)) { final Cursor cursor = cursorHolder.asCursor(); CursorGranularizer granularizer = CursorGranularizer.create( - adapter, cursor, + timeBoundaryInspector, + Order.DESCENDING, Granularities.HOUR, - adapter.getInterval(), - true + adapter.getInterval() ); final ColumnSelectorFactory selectorFactory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java index e562f72955ef..270a335bc77a 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java @@ -393,78 +393,128 @@ public void remove() } /** - * Check if a QueryRunner returned by {@link #makeQueryRunners(QueryRunnerFactory)} is vectorizable. + * Check if a QueryRunner returned by {@link #makeQueryRunners(QueryRunnerFactory, boolean)} is vectorizable. */ public static boolean isTestRunnerVectorizable(QueryRunner runner) { final String runnerName = runner.toString(); - return !("rtIndex".equals(runnerName) || "noRollupRtIndex".equals(runnerName)); + return !("rtIndex".equals(runnerName) + || "noRollupRtIndex".equals(runnerName) + || "nonTimeOrderedRtIndex".equals(runnerName) + || "nonTimeOrderedNoRollupRtIndex".equals(runnerName)); } - - public static > List> makeQueryRunners( - QueryRunnerFactory factory + /** + * Create test query runners. + * + * @param factory query runner factory + * @param includeNonTimeOrdered whether to include runners with non-time-ordered segments. Some test suites are not + * compatible with non-time-ordered segments. + */ + public static > List> makeQueryRunners( + QueryRunnerFactory factory, + boolean includeNonTimeOrdered ) { - BiFunction> maker = (name, seg) -> makeQueryRunner(factory, seg, name); - - return ImmutableList.of( - maker.apply( - "rtIndex", - new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(), SEGMENT_ID) - ), - maker.apply( - "noRollupRtIndex", - new IncrementalIndexSegment(TestIndex.getNoRollupIncrementalTestIndex(), SEGMENT_ID) - ), - maker.apply( - "mMappedTestIndex", - new QueryableIndexSegment(TestIndex.getMMappedTestIndex(), SEGMENT_ID) - ), - maker.apply( - "noRollupMMappedTestIndex", - new QueryableIndexSegment(TestIndex.getNoRollupMMappedTestIndex(), SEGMENT_ID) - ), - maker.apply( - "mergedRealtimeIndex", - new QueryableIndexSegment(TestIndex.mergedRealtimeIndex(), SEGMENT_ID) - ), - maker.apply( - "frontCodedMMappedTestIndex", - new QueryableIndexSegment(TestIndex.getFrontCodedMMappedTestIndex(), SEGMENT_ID) + BiFunction> maker = (name, seg) -> makeQueryRunner(factory, seg, name); + + final ImmutableList.Builder> retVal = ImmutableList.builder(); + + retVal.addAll( + Arrays.asList( + maker.apply( + "rtIndex", + new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(), SEGMENT_ID) + ), + maker.apply( + "noRollupRtIndex", + new IncrementalIndexSegment(TestIndex.getNoRollupIncrementalTestIndex(), SEGMENT_ID) + ), + maker.apply( + "mMappedTestIndex", + new QueryableIndexSegment(TestIndex.getMMappedTestIndex(), SEGMENT_ID) + ), + maker.apply( + "noRollupMMappedTestIndex", + new QueryableIndexSegment(TestIndex.getNoRollupMMappedTestIndex(), SEGMENT_ID) + ), + maker.apply( + "mergedRealtimeIndex", + new QueryableIndexSegment(TestIndex.mergedRealtimeIndex(), SEGMENT_ID) + ), + maker.apply( + "frontCodedMMappedTestIndex", + new QueryableIndexSegment(TestIndex.getFrontCodedMMappedTestIndex(), SEGMENT_ID) + ) ) ); + + if (includeNonTimeOrdered) { + retVal.addAll( + Arrays.asList( + maker.apply( + "nonTimeOrderedRtIndex", + new IncrementalIndexSegment(TestIndex.getNonTimeOrderedRealtimeTestIndex(), SEGMENT_ID) + ), + maker.apply( + "nonTimeOrderedNoRollupRtIndex", + new IncrementalIndexSegment(TestIndex.getNonTimeOrderedNoRollupRealtimeTestIndex(), SEGMENT_ID) + ), + maker.apply( + "nonTimeOrderedMMappedTestIndex", + new QueryableIndexSegment(TestIndex.getNonTimeOrderedMMappedTestIndex(), SEGMENT_ID) + ), + maker.apply( + "nonTimeOrderedNoRollupMMappedTestIndex", + new QueryableIndexSegment(TestIndex.getNonTimeOrderedNoRollupMMappedTestIndex(), SEGMENT_ID) + ) + ) + ); + } + + return retVal.build(); } - public static > List> makeQueryRunnersToMerge( - QueryRunnerFactory factory + /** + * Create test query runners. + * + * @param factory query runner factory + * @param includeNonTimeOrdered whether to include runners with non-time-ordered segments. Some test suites are not + * written to be compatible with non-time-ordered segments. + */ + public static > List> makeQueryRunnersToMerge( + final QueryRunnerFactory factory, + final boolean includeNonTimeOrdered ) { - return mapQueryRunnersToMerge(factory, makeQueryRunners(factory)); + return mapQueryRunnersToMerge(factory, makeQueryRunners(factory, includeNonTimeOrdered)); } - public static > ArrayList> mapQueryRunnersToMerge( + public static > ArrayList> mapQueryRunnersToMerge( QueryRunnerFactory factory, - List> runners + List> runners ) { - final ArrayList> retVal = new ArrayList<>(runners.size()); + final ArrayList> retVal = new ArrayList<>(runners.size()); final QueryToolChest toolchest = factory.getToolchest(); - for (QueryRunner baseRunner : runners) { + for (TestQueryRunner baseRunner : runners) { retVal.add( - FluentQueryRunner.create(baseRunner, toolchest) - .applyPreMergeDecoration() - .mergeResults(true) - .applyPostMergeDecoration() - .setToString(baseRunner.toString()) + new TestQueryRunner<>( + baseRunner.getName(), + FluentQueryRunner.create(baseRunner, toolchest) + .applyPreMergeDecoration() + .mergeResults(true) + .applyPostMergeDecoration(), + baseRunner.getSegment() + ) ); } return retVal; } - public static > QueryRunner makeQueryRunner( + public static > TestQueryRunner makeQueryRunner( QueryRunnerFactory factory, String resourceFileName, final String runnerName @@ -478,7 +528,7 @@ public static > QueryRunner makeQueryRunner( ); } - public static > QueryRunner makeQueryRunner( + public static > TestQueryRunner makeQueryRunner( QueryRunnerFactory factory, Segment adapter, final String runnerName @@ -487,33 +537,24 @@ public static > QueryRunner makeQueryRunner( return makeQueryRunner(factory, SEGMENT_ID, adapter, runnerName); } - public static > QueryRunner makeQueryRunner( - QueryRunnerFactory factory, - SegmentId segmentId, - Segment adapter, + public static > TestQueryRunner makeQueryRunner( + final QueryRunnerFactory factory, + final SegmentId segmentId, + final Segment adapter, final String runnerName ) { - //noinspection - return new BySegmentQueryRunner( - segmentId, - adapter.getDataInterval().getStart(), - factory.createRunner(adapter) - ) - { - @Override - public String toString() - { - // Tests that use these QueryRunners directly are parameterized and use the toString of their QueryRunner as - // the name of the test. It would be better if the usages were adjusted to actually parameterize with an extra - // name parameter, or use a different object or something like that, but for now, we have to overload toString - // to name it so that the parameterization continues to work. - return runnerName; - } - }; + return new TestQueryRunner<>( + runnerName, + new BySegmentQueryRunner<>( + segmentId, + adapter.getDataInterval().getStart(), + factory.createRunner(adapter) + ), + adapter + ); } - public static > QueryRunner makeQueryRunnerWithSegmentMapFn( QueryRunnerFactory factory, Segment adapter, diff --git a/processing/src/test/java/org/apache/druid/query/TestQueryRunner.java b/processing/src/test/java/org/apache/druid/query/TestQueryRunner.java new file mode 100644 index 000000000000..1a589d507010 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/TestQueryRunner.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; + +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.segment.Segment; + +/** + * Wrapper around a {@link QueryRunner} that adds a name and allows retrieval of the underlying segment. Used by + * {@link QueryRunnerTestHelper#makeQueryRunners(QueryRunnerFactory, boolean)} and related methods. + */ +public class TestQueryRunner implements QueryRunner +{ + private final String name; + private final QueryRunner runner; + private final Segment segment; + + public TestQueryRunner(String name, QueryRunner runner, Segment segment) + { + this.name = name; + this.runner = runner; + this.segment = segment; + } + + @Override + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + { + return runner.run(queryPlus, responseContext); + } + + @Override + public Sequence run(QueryPlus queryPlus) + { + return runner.run(queryPlus); + } + + public String getName() + { + return name; + } + + public Segment getSegment() + { + return segment; + } + + @Override + public String toString() + { + return name; + } +} diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java index 2e960e6186e3..2c44d080deb3 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java @@ -35,8 +35,10 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryEngine; import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; @@ -143,10 +145,20 @@ public void testTimeseriesQuery() final DefaultTimeseriesQueryMetrics defaultTimeseriesQueryMetrics = new DefaultTimeseriesQueryMetrics(); final Iterable> iiResults = - engine.process(query, new IncrementalIndexStorageAdapter(incrementalIndex), defaultTimeseriesQueryMetrics).toList(); + engine.process( + query, + new IncrementalIndexStorageAdapter(incrementalIndex), + new IncrementalIndexTimeBoundaryInspector(incrementalIndex), + defaultTimeseriesQueryMetrics + ).toList(); final Iterable> qiResults = - engine.process(query, new QueryableIndexStorageAdapter(queryableIndex), defaultTimeseriesQueryMetrics).toList(); + engine.process( + query, + new QueryableIndexStorageAdapter(queryableIndex), + QueryableIndexTimeBoundaryInspector.create(queryableIndex), + defaultTimeseriesQueryMetrics + ).toList(); TestHelper.assertExpectedResults(expectedResults, iiResults, "incremental index"); TestHelper.assertExpectedResults(expectedResults, qiResults, "queryable index"); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java index 6bbada405b93..1c9b4033cd9a 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java @@ -35,8 +35,10 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryEngine; import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; @@ -147,10 +149,20 @@ public void testTimeseriesQuery() final DefaultTimeseriesQueryMetrics defaultTimeseriesQueryMetrics = new DefaultTimeseriesQueryMetrics(); final Iterable> iiResults = - engine.process(query, new IncrementalIndexStorageAdapter(incrementalIndex), defaultTimeseriesQueryMetrics).toList(); + engine.process( + query, + new IncrementalIndexStorageAdapter(incrementalIndex), + new IncrementalIndexTimeBoundaryInspector(incrementalIndex), + defaultTimeseriesQueryMetrics + ).toList(); final Iterable> qiResults = - engine.process(query, new QueryableIndexStorageAdapter(queryableIndex), defaultTimeseriesQueryMetrics).toList(); + engine.process( + query, + new QueryableIndexStorageAdapter(queryableIndex), + QueryableIndexTimeBoundaryInspector.create(queryableIndex), + defaultTimeseriesQueryMetrics + ).toList(); TestHelper.assertExpectedResults(expectedResults, iiResults, "incremental index"); TestHelper.assertExpectedResults(expectedResults, qiResults, "queryable index"); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java index 7c5b5e908444..5de602b87720 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -173,7 +173,7 @@ public static void teardownClass() public static Collection constructorFeeder() { final List args = new ArrayList<>(); - for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(FACTORY)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(FACTORY, true)) { args.add(new Object[]{runner}); } return args; diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java index ad6157dc927d..69debeb902cb 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java @@ -40,7 +40,10 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.segment.TestHelper; +import org.junit.After; import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -126,6 +129,26 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( private QueryRunner runner; + @Before + public void setUp() + { + Assert.assertEquals( + "MERGE_BUFFER_POOL size, pre-test", + MERGE_BUFFER_POOL.maxSize(), + MERGE_BUFFER_POOL.getPoolSize() + ); + } + + @After + public void tearDown() + { + Assert.assertEquals( + "MERGE_BUFFER_POOL size, post-test", + MERGE_BUFFER_POOL.maxSize(), + MERGE_BUFFER_POOL.getPoolSize() + ); + } + @AfterClass public static void teardownClass() { @@ -137,7 +160,7 @@ public static void teardownClass() public static Collection constructorFeeder() { final List args = new ArrayList<>(); - for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(FACTORY)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(FACTORY, true)) { args.add(new Object[]{runner}); } return args; @@ -253,15 +276,13 @@ public void testInsufficientResourcesOnBroker() @Test(timeout = 60_000L) public void testTimeoutExceptionOnQueryable() { - expectedException.expect(QueryTimeoutException.class); - final GroupByQuery query = GroupByQuery .builder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions(new DefaultDimensionSpec("quality", "alias")) .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows")) - .setGranularity(QueryRunnerTestHelper.DAY_GRAN) + .setGranularity(Granularities.ALL) .overrideContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 1)) .build(); @@ -288,6 +309,10 @@ public boolean isSingleThreaded() }; QueryRunner mergeRunners = factory.mergeRunners(Execs.directExecutor(), ImmutableList.of(runner, mockRunner)); - GroupByQueryRunnerTestHelper.runQuery(factory, mergeRunners, query); + + Assert.assertThrows( + QueryTimeoutException.class, + () -> GroupByQueryRunnerTestHelper.runQuery(factory, mergeRunners, query) + ); } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index 475848edbdc7..53050b20ad58 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -56,6 +56,7 @@ import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.FinalizeResultsQueryRunner; +import org.apache.druid.query.Order; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryPlus; @@ -65,6 +66,7 @@ import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.Result; import org.apache.druid.query.TestBigDecimalSumAggregatorFactory; +import org.apache.druid.query.TestQueryRunner; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; @@ -128,6 +130,9 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.search.ContainsSearchQuerySpec; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.Cursors; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; @@ -141,6 +146,7 @@ import org.joda.time.Period; import org.junit.AfterClass; import org.junit.Assert; +import org.junit.Assume; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Rule; @@ -199,7 +205,7 @@ public int getNumThreads() private static TestGroupByBuffers BUFFER_POOLS = null; private final QueryRunner runner; - private final QueryRunner originalRunner; + private final TestQueryRunner originalRunner; private final GroupByQueryRunnerFactory factory; private final GroupByQueryConfig config; private final boolean vectorize; @@ -379,7 +385,7 @@ public static Collection constructorFeeder() final List constructors = new ArrayList<>(); for (GroupByQueryConfig config : testConfigs()) { final GroupByQueryRunnerFactory factory = makeQueryRunnerFactory(config, BUFFER_POOLS); - for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory, true)) { for (boolean vectorize : ImmutableList.of(false, true)) { final String testName = StringUtils.format("config=%s, runner=%s, vectorize=%s", config, runner, vectorize); @@ -414,7 +420,7 @@ public GroupByQueryRunnerTest( String testName, GroupByQueryConfig config, GroupByQueryRunnerFactory factory, - QueryRunner runner, + TestQueryRunner runner, boolean vectorize ) { @@ -428,6 +434,9 @@ public GroupByQueryRunnerTest( @Test public void testGroupBy() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -741,6 +750,9 @@ public void testGroupByOnMissingColumn() @Test public void testGroupByWithStringPostAggregator() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -989,6 +1001,9 @@ public void testGroupByWithStringPostAggregator() @Test public void testGroupByWithStringVirtualColumn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -1052,6 +1067,9 @@ public void testGroupByWithStringVirtualColumn() @Test public void testGroupByWithStringVirtualColumnVectorizable() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -1115,6 +1133,9 @@ public void testGroupByWithStringVirtualColumnVectorizable() @Test public void testGroupByWithDurationGranularity() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -1167,6 +1188,9 @@ public void testGroupByWithOutputNameCollisions() @Test public void testGroupByWithSortDimsFirst() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -1212,6 +1236,9 @@ public void testGroupByWithSortDimsFirst() @Test public void testGroupByNoAggregators() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -1248,6 +1275,9 @@ public void testGroupByNoAggregators() @Test public void testMultiValueDimension() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to multi-value dimensions. cannotVectorize(); @@ -1278,6 +1308,9 @@ public void testMultiValueDimension() @Test public void testMultiValueDimensionNotAllowed() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final String dimName = "placementish"; if (!vectorize) { @@ -2743,6 +2776,9 @@ public void testMultipleDimensionsOneOfWhichIsMultiValueDifferentOrder() @Test public void testGroupByMaxRowsLimitContextOverride() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -2799,6 +2835,9 @@ public void testGroupByMaxRowsLimitContextOverride() @Test public void testGroupByTimeoutContextOverride() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -2837,6 +2876,9 @@ public void testGroupByTimeoutContextOverride() @Test public void testGroupByMaxOnDiskStorageContextOverride() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -2858,6 +2900,9 @@ public void testGroupByMaxOnDiskStorageContextOverride() @Test public void testNotEnoughDiskSpaceThroughContextOverride() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -2883,6 +2928,9 @@ public void testNotEnoughDiskSpaceThroughContextOverride() @Test public void testSubqueryWithOuterMaxOnDiskStorageContextOverride() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -2918,6 +2966,9 @@ public void testSubqueryWithOuterMaxOnDiskStorageContextOverride() @Test public void testGroupByWithRebucketRename() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -2989,6 +3040,9 @@ public void testGroupByWithRebucketRename() @Test public void testGroupByWithSimpleRenameRetainMissingNonInjective() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -3060,6 +3114,9 @@ public void testGroupByWithSimpleRenameRetainMissingNonInjective() @Test public void testGroupByWithSimpleRenameRetainMissing() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map map = new HashMap<>(); map.put("automotive", "automotive0"); map.put("business", "business0"); @@ -3114,6 +3171,9 @@ public void testGroupByWithSimpleRenameRetainMissing() @Test public void testGroupByWithSimpleRenameAndMissingString() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map map = new HashMap<>(); map.put("automotive", "automotive0"); map.put("business", "business0"); @@ -3163,6 +3223,9 @@ public void testGroupByWithSimpleRenameAndMissingString() @Test public void testGroupByWithSimpleRename() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map map = new HashMap<>(); map.put("automotive", "automotive0"); map.put("business", "business0"); @@ -3313,6 +3376,9 @@ public void testGroupByWithCardinality() @Test public void testGroupByWithFirstLast() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -3400,6 +3466,9 @@ public void testGroupByWithFirstLast() @Test public void testGroupByWithNoResult() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.EMPTY_INTERVAL) @@ -3422,6 +3491,9 @@ public void testGroupByWithNoResult() @Test public void testGroupByWithNullProducingDimExtractionFn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -3483,6 +3555,9 @@ public String apply(String dimValue) */ public void testGroupByWithEmptyStringProducingDimExtractionFn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final ExtractionFn emptyStringExtractionFn = new RegexDimExtractionFn("(\\w{1})", false, null) { @Override @@ -3536,6 +3611,9 @@ public String apply(String dimValue) @Test public void testGroupByWithTimeZone() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles"); GroupByQuery query = makeQueryBuilder() @@ -3746,6 +3824,9 @@ public void testGroupByWithTimeZone() @Test public void testMergeResults() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -3835,6 +3916,9 @@ public void testMergeResultsWithLimitAndOffset() private void doTestMergeResultsWithValidLimit(final int limit, final int offset) { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -3950,6 +4034,9 @@ private void doTestMergeResultsWithValidLimit(final int limit, final int offset) @Test public void testMergeResultsAcrossMultipleDaysWithLimitAndOrderBy() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final int limit = 14; GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -3992,6 +4079,9 @@ public void testMergeResultsAcrossMultipleDaysWithLimitAndOrderBy() @Test public void testMergeResultsAcrossMultipleDaysWithLimitAndOrderByUsingMathExpressions() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final int limit = 14; GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -4056,6 +4146,9 @@ public void testMergeResultsWithNegativeLimit() @Test public void testMergeResultsWithOrderBy() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + LimitSpec[] orderBySpecs = new LimitSpec[]{ new DefaultLimitSpec(OrderByColumnSpec.ascending("idx"), null), new DefaultLimitSpec(OrderByColumnSpec.ascending("rows", "idx"), null), @@ -4150,6 +4243,9 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r @Test public void testGroupByOrderLimit() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -4248,6 +4344,9 @@ public void testGroupByOrderLimit() @Test public void testGroupByWithOrderLimit2() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -4289,6 +4388,9 @@ public void testGroupByWithOrderLimit2() @Test public void testGroupByWithOrderLimit3() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -4332,6 +4434,9 @@ public void testGroupByWithOrderLimit3() @Test public void testGroupByOrderLimitNumeric() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -4704,6 +4809,9 @@ public void testGroupByWithLimitOnFinalizedHyperUnique() @Test public void testGroupByWithAlphaNumericDimensionOrder() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -4762,6 +4870,9 @@ public void testGroupByWithAlphaNumericDimensionOrder() @Test public void testGroupByWithLookupAndLimitAndSortByDimsFirst() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -4822,6 +4933,9 @@ public void testGroupByWithLookupAndLimitAndSortByDimsFirst() // in time when Druid does support this, we can re-evaluate this test. public void testLimitPerGrouping() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setGranularity(QueryRunnerTestHelper.DAY_GRAN).setDimensions(new DefaultDimensionSpec( @@ -4860,6 +4974,9 @@ public void testLimitPerGrouping() @Test public void testPostAggMergedHavingSpec() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5085,6 +5202,9 @@ public void testGroupByWithOrderLimitHavingSpec() @Test public void testPostAggHavingSpec() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5140,6 +5260,9 @@ public void testPostAggHavingSpec() @Test public void testHavingSpec() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5173,6 +5296,9 @@ public void testHavingSpec() @Test public void testDimFilterHavingSpec() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final DimFilterHavingSpec havingSpec = new DimFilterHavingSpec( new AndDimFilter( ImmutableList.of( @@ -5278,6 +5404,9 @@ public void testDimFilterHavingSpec() @Test public void testDimFilterHavingSpecWithExtractionFns() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + String extractionJsFn = "function(str) { return 'super-' + str; }"; ExtractionFn extractionFn = new JavaScriptExtractionFn( extractionJsFn, @@ -5328,6 +5457,9 @@ public void testDimFilterHavingSpecWithExtractionFns() @Test public void testMergedHavingSpec() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5391,6 +5523,9 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r @Test public void testMergedPostAggHavingSpec() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5500,6 +5635,9 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r @Test public void testCustomAggregatorHavingSpec() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5540,6 +5678,9 @@ public void testCustomAggregatorHavingSpec() @Test public void testGroupByWithRegEx() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5565,6 +5706,9 @@ public void testGroupByWithRegEx() @Test public void testGroupByWithNonexistentDimension() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5625,6 +5769,9 @@ public void testGroupByWithNonexistentDimension() @Test public void testIdenticalSubquery() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5681,6 +5828,9 @@ public void testIdenticalSubquery() @Test public void testSubqueryWithMultipleIntervalsInOuterQuery() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5746,6 +5896,9 @@ public void testSubqueryWithExtractionFnInOuterQuery() { //https://github.com/apache/druid/issues/2556 + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5791,6 +5944,9 @@ public void testSubqueryWithExtractionFnInOuterQuery() @Test public void testDifferentGroupingSubquery() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5854,6 +6010,9 @@ public void testDifferentGroupingSubquery() @Test public void testDifferentGroupingSubqueryMultipleAggregatorsOnSameField() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5903,6 +6062,9 @@ public void testDifferentGroupingSubqueryMultipleAggregatorsOnSameField() @Test public void testDifferentGroupingSubqueryWithFilter() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5945,6 +6107,9 @@ public void testDifferentGroupingSubqueryWithFilter() @Test public void testDifferentIntervalSubquery() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5971,7 +6136,7 @@ public void testDifferentIntervalSubquery() @Test public void testDoubleMeanQuery() { - GroupByQuery query = new GroupByQuery.Builder() + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setGranularity(Granularities.ALL) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -6057,6 +6222,9 @@ public void testGroupByWithUnderUnderTimeAsDimensionNameWithHavingAndLimit() @Test public void testEmptySubquery() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.EMPTY_INTERVAL) @@ -6079,6 +6247,9 @@ public void testEmptySubquery() @Test public void testSubqueryWithPostAggregators() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -6353,6 +6524,9 @@ public void testSubqueryWithPostAggregators() @Test public void testSubqueryWithPostAggregatorsAndHaving() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -6627,6 +6801,9 @@ public boolean eval(ResultRow row) @Test public void testSubqueryWithMultiColumnAggregators() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to javascript functionality. cannotVectorize(); @@ -6810,6 +6987,9 @@ public boolean eval(ResultRow row) @Test public void testSubqueryWithOuterFilterAggregator() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -6837,6 +7017,9 @@ public void testSubqueryWithOuterFilterAggregator() @Test public void testSubqueryWithOuterTimeFilter() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -6882,6 +7065,9 @@ public void testSubqueryWithOuterTimeFilter() @Test public void testSubqueryWithContextTimeout() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -6907,6 +7093,9 @@ public void testSubqueryWithContextTimeout() @Test public void testSubqueryWithOuterVirtualColumns() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -6962,6 +7151,9 @@ public void testSubqueryWithOuterCardinalityAggregator() @Test public void testSubqueryWithOuterCountAggregator() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -6992,6 +7184,9 @@ public void testSubqueryWithOuterCountAggregator() @Test public void testSubqueryWithOuterDimJavascriptAggregators() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7043,6 +7238,9 @@ public void testSubqueryWithOuterDimJavascriptAggregators() @Test public void testSubqueryWithOuterJavascriptAggregators() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7094,6 +7292,9 @@ public void testSubqueryWithOuterJavascriptAggregators() @Test public void testSubqueryWithHyperUniques() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7237,6 +7438,9 @@ public void testSubqueryWithHyperUniques() @Test public void testSubqueryWithHyperUniquesPostAggregator() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7294,6 +7498,9 @@ public void testSubqueryWithHyperUniquesPostAggregator() @Test public void testSubqueryWithFirstLast() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -7332,6 +7539,9 @@ public void testSubqueryWithFirstLast() @Test public void testGroupByWithSubtotalsSpecOfDimensionsPrefixes() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7446,6 +7656,9 @@ public void testGroupByWithSubtotalsSpecOfDimensionsPrefixes() @Test public void testGroupByWithSubtotalsSpecGeneral() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7802,6 +8015,9 @@ public void testGroupByWithSubtotalsSpecGeneral() @Test public void testGroupByWithSubtotalsSpecWithRenamedDimensionAndFilter() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = GroupByQuery .builder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -7896,6 +8112,9 @@ public void testGroupByWithSubtotalsSpecWithRenamedDimensionAndFilter() @Test public void testGroupByWithSubtotalsSpecWithLongDimensionColumn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -8187,6 +8406,9 @@ public void testGroupByWithSubtotalsSpecWithLongDimensionColumn() @Test public void testGroupByWithSubtotalsSpecWithOrderLimit() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -8222,6 +8444,9 @@ public void testGroupByWithSubtotalsSpecWithOrderLimit() @Test public void testGroupByWithSubtotalsSpecWithOrderLimitAndOffset() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -8257,6 +8482,9 @@ public void testGroupByWithSubtotalsSpecWithOrderLimitAndOffset() @Test public void testGroupByWithSubtotalsSpecWithOrderLimitForcePushdown() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -8834,6 +9062,9 @@ public ExtractionType getExtractionType() @Test public void testBySegmentResults() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -8894,6 +9125,9 @@ public void testBySegmentResults() @Test public void testBySegmentResultsUnOptimizedDimextraction() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04").setDimensions(new ExtractionDimensionSpec( @@ -8961,6 +9195,9 @@ public void testBySegmentResultsUnOptimizedDimextraction() @Test public void testBySegmentResultsOptimizedDimextraction() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04").setDimensions(new ExtractionDimensionSpec( @@ -9030,6 +9267,9 @@ public void testBySegmentResultsOptimizedDimextraction() @Test public void testGroupByWithExtractionDimFilter() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map extractionMap = new HashMap<>(); extractionMap.put("automotive", "automotiveAndBusinessAndNewsAndMezzanine"); extractionMap.put("business", "automotiveAndBusinessAndNewsAndMezzanine"); @@ -9089,6 +9329,9 @@ public void testGroupByWithExtractionDimFilter() @Test public void testGroupByWithExtractionDimFilterCaseMappingValueIsNullOrEmpty() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map extractionMap = new HashMap<>(); extractionMap.put("automotive", "automotive0"); extractionMap.put("business", "business0"); @@ -9138,6 +9381,9 @@ public void testGroupByWithExtractionDimFilterCaseMappingValueIsNullOrEmpty() @Test public void testGroupByWithExtractionDimFilterWhenSearchValueNotInTheMap() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map extractionMap = new HashMap<>(); MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, false); @@ -9165,6 +9411,9 @@ public void testGroupByWithExtractionDimFilterWhenSearchValueNotInTheMap() @Test public void testGroupByWithExtractionDimFilterKeyisNull() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map extractionMap = new HashMap<>(); @@ -9209,6 +9458,9 @@ public void testGroupByWithExtractionDimFilterKeyisNull() @Test public void testGroupByWithAggregatorFilterAndExtractionFunction() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map extractionMap = new HashMap<>(); extractionMap.put("automotive", "automotive0"); extractionMap.put("business", "business0"); @@ -9392,6 +9644,9 @@ public void testGroupByWithAggregatorFilterAndExtractionFunction() @Test public void testGroupByWithExtractionDimFilterOptimazitionManyToOne() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map extractionMap = new HashMap<>(); extractionMap.put("mezzanine", "newsANDmezzanine"); extractionMap.put("news", "newsANDmezzanine"); @@ -9431,6 +9686,9 @@ public void testGroupByWithExtractionDimFilterOptimazitionManyToOne() @Test public void testGroupByWithInjectiveLookupDimFilterNullDimsOptimized() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map extractionMap = new HashMap<>(); extractionMap.put("", "EMPTY"); @@ -9465,6 +9723,9 @@ public void testGroupByWithInjectiveLookupDimFilterNullDimsOptimized() @Test public void testGroupByWithInjectiveLookupDimFilterNullDimsNotOptimized() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map extractionMap = new HashMap<>(); extractionMap.put("", "EMPTY"); @@ -9499,6 +9760,9 @@ public void testGroupByWithInjectiveLookupDimFilterNullDimsNotOptimized() @Test public void testBySegmentResultsWithAllFiltersWithExtractionFns() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + String extractionJsFn = "function(str) { return 'super-' + str; }"; String jsFn = "function(x) { return(x === 'super-mezzanine') }"; ExtractionFn extractionFn = new JavaScriptExtractionFn( @@ -9593,6 +9857,9 @@ public void testBySegmentResultsWithAllFiltersWithExtractionFns() @Test public void testGroupByWithAllFiltersOnNullDimsWithExtractionFns() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + Map extractionMap = new HashMap<>(); extractionMap.put("", "EMPTY"); extractionMap.put(null, "EMPTY"); @@ -9639,6 +9906,9 @@ public void testGroupByWithAllFiltersOnNullDimsWithExtractionFns() @Test public void testGroupByCardinalityAggWithExtractionFn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -9731,6 +10001,9 @@ public void testGroupByCardinalityAggWithExtractionFn() @Test public void testGroupByCardinalityAggOnFloat() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -9922,6 +10195,9 @@ public void testGroupByCardinalityAggOnHyperUnique() @Test public void testGroupByLongColumn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10033,6 +10309,9 @@ public void testGroupByLongColumnDescending() @Test public void testGroupByLongColumnWithExFn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -10077,6 +10356,9 @@ public void testGroupByLongColumnWithExFn() @Test public void testGroupByLongTimeColumn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10115,6 +10397,9 @@ public void testGroupByLongTimeColumn() @Test public void testGroupByLongTimeColumnWithExFn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -10159,6 +10444,9 @@ public void testGroupByLongTimeColumnWithExFn() @Test public void testGroupByFloatColumn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10295,6 +10583,9 @@ public void testGroupByDoubleColumnDescending() @Test public void testGroupByFloatColumnWithExFn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -10394,6 +10685,9 @@ public void testGroupByWithHavingSpecOnLongAndFloat() @Test public void testGroupByLongAndFloatOutputAsString() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10439,6 +10733,9 @@ public void testGroupByLongAndFloatOutputAsString() @Test public void testGroupByNumericStringsAsNumeric() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10609,6 +10906,9 @@ public void testGroupByDecorationOnNumerics() @Test public void testGroupByNestedWithInnerQueryNumerics() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10680,6 +10980,9 @@ public void testGroupByNestedWithInnerQueryNumerics() @Test public void testGroupByNestedWithInnerQueryOutputNullNumerics() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + cannotVectorize(); // Following extractionFn will generate null value for one kind of quality @@ -10844,6 +11147,9 @@ public void testGroupByNestedWithInnerQueryNumericsWithLongTime() @Test public void testGroupByStringOutputAsLong() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -10893,6 +11199,9 @@ public void testGroupByStringOutputAsLong() @Test public void testGroupByWithAggsOnNumericDimensions() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to javascript aggregators. cannotVectorize(); @@ -10961,6 +11270,9 @@ public void testGroupByWithAggsOnNumericDimensions() @Test public void testGroupByNestedOuterExtractionFnOnFloatInner() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -11004,6 +11316,9 @@ public void testGroupByNestedOuterExtractionFnOnFloatInner() @Test public void testGroupByNestedDoubleTimeExtractionFnWithLongOutputTypes() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -11677,6 +11992,9 @@ public void testGroupByLimitPushDownPostAggNotSupported() @Test public void testEmptySubqueryWithLimitPushDown() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.EMPTY_INTERVAL) @@ -11710,6 +12028,9 @@ public void testEmptySubqueryWithLimitPushDown() @Test public void testSubqueryWithMultipleIntervalsInOuterQueryWithLimitPushDown() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -11781,6 +12102,9 @@ public void testSubqueryWithMultipleIntervalsInOuterQueryWithLimitPushDown() @Test public void testVirtualColumnFilterOnInnerQuery() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -11878,6 +12202,9 @@ public void testRejectForceLimitPushDownWithHaving() @Test public void testTypeConversionWithMergingChainedExecutionRunner() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -11915,6 +12242,9 @@ public void testTypeConversionWithMergingChainedExecutionRunner() @Test public void testGroupByNoMatchingPrefilter() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -12171,8 +12501,12 @@ public void testGroupByOnVirtualColumnTimeFloor() @Test public void testGroupByWithExpressionAggregator() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // expression agg not yet vectorized cannotVectorize(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -12494,6 +12828,9 @@ public void testGroupByWithExpressionAggregatorWithComplexOnSubquery() @Test public void testGroupByWithExpressionAggregatorWithArrays() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // expression agg not yet vectorized cannotVectorize(); @@ -12781,6 +13118,9 @@ public void testGroupByWithExpressionAggregatorWithArrays() @Test public void testGroupByExpressionAggregatorArrayMultiValue() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + // expression agg not yet vectorized cannotVectorize(); @@ -13201,6 +13541,9 @@ public void testSummaryrowFilteredByHaving() @Test public void testSummaryrowForEmptySubqueryInput() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -13250,6 +13593,9 @@ public void testSummaryrowForEmptySubqueryInput() @Test public void testSummaryrowForEmptyInputByDay() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -13313,6 +13659,14 @@ private GroupByQuery.Builder makeQueryBuilder(final GroupByQuery query) return new GroupByQuery.Builder(query).overrideContext(makeContext()); } + private void assumeTimeOrdered() + { + try (final CursorHolder cursorHolder = + originalRunner.getSegment().asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + Assume.assumeTrue(Cursors.getTimeOrdering(cursorHolder.getOrdering()) == Order.ASCENDING); + } + } + private Map makeContext() { return ImmutableMap.builder() diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index d8a609800bb7..1309334a939a 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -100,7 +100,7 @@ public static Iterable constructorFeeder() final List constructors = new ArrayList<>(); - for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunnersToMerge(factory)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunnersToMerge(factory, false)) { final QueryRunner modifiedRunner = new QueryRunner() { @Override diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java index ce38427728c9..15982c2caab2 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java @@ -33,6 +33,7 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.vector.VectorCursor; @@ -62,6 +63,8 @@ public void testCreateOneGrouperAndCloseItWhenClose() throws IOException .setAggregatorSpecs(factory) .build(); final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex()); + final QueryableIndexTimeBoundaryInspector timeBoundaryInspector = + QueryableIndexTimeBoundaryInspector.create(TestIndex.getMMappedTestIndex()); final CursorHolder cursorHolder = storageAdapter.makeCursorHolder( GroupingEngine.makeCursorBuildSpec(query, null) ); @@ -81,7 +84,9 @@ public void testCreateOneGrouperAndCloseItWhenClose() throws IOException new GroupByQueryConfig(), GroupByQueryRunnerTest.DEFAULT_PROCESSING_CONFIG, storageAdapter, + timeBoundaryInspector, cursor, + cursorHolder.getTimeOrder(), interval, dimensions, byteBuffer, diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java index 503eedb5db88..8a5605bc673c 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java @@ -45,6 +45,7 @@ import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.query.spec.LegacySegmentSpec; import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursors; import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; @@ -438,6 +439,9 @@ public void testAnalysisImproperComplex() throws IOException EasyMock.expect(mockIndex.getColumnCapabilities("x")) .andReturn(ColumnCapabilitiesImpl.createDefault().setType(ColumnType.UNKNOWN_COMPLEX)) .atLeastOnce(); + EasyMock.expect(mockIndex.getOrdering()) + .andReturn(Cursors.ascendingTimeOrder()) + .atLeastOnce(); ColumnHolder holder = EasyMock.createMock(ColumnHolder.class); EasyMock.expect(mockIndex.getColumnHolder("x")).andReturn(holder).atLeastOnce(); diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java index b1db91a9cb85..33147ce01b0e 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java @@ -24,9 +24,9 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.Druids; import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.TestQueryRunner; import org.apache.druid.query.metadata.metadata.ColumnAnalysis; import org.apache.druid.query.metadata.metadata.ListColumnIncluderator; import org.apache.druid.query.metadata.metadata.SegmentAnalysis; @@ -57,11 +57,11 @@ public class SegmentMetadataUnionQueryTest extends InitializedNullHandlingTest QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - private final QueryRunner runner; + private final TestQueryRunner runner; private final boolean mmap; public SegmentMetadataUnionQueryTest( - QueryRunner runner, + TestQueryRunner runner, boolean mmap ) { @@ -72,7 +72,7 @@ public SegmentMetadataUnionQueryTest( @Parameterized.Parameters public static Iterable constructorFeeder() { - final ArrayList> runners = QueryRunnerTestHelper.mapQueryRunnersToMerge( + final ArrayList> runners = QueryRunnerTestHelper.mapQueryRunnersToMerge( FACTORY, ImmutableList.of( QueryRunnerTestHelper.makeQueryRunner( 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 8dd183906189..7693f05471ff 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 @@ -151,9 +151,7 @@ public Object getResult() public static Iterable constructorFeeder() { return Iterables.transform( - QueryRunnerTestHelper.makeQueryRunners( - FACTORY - ), + QueryRunnerTestHelper.makeQueryRunners(FACTORY, false), (runner) -> new Object[]{runner} ); } diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java index 306ed04f634c..486cfb950170 100644 --- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java @@ -94,7 +94,8 @@ public static Iterable constructorFeeder() SELECTOR, TOOL_CHEST, QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + ), + true ) ); } diff --git a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index 02ba0ac44f4f..6b1ee55568fd 100644 --- a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -37,6 +37,7 @@ import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.Result; import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.TestQueryRunner; import org.apache.druid.query.context.ConcurrentResponseContext; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.ordering.StringComparators; @@ -79,12 +80,13 @@ public static Iterable constructorFeeder() { return QueryRunnerTestHelper.transformToConstructionFeeder( QueryRunnerTestHelper.makeQueryRunners( - new TimeBoundaryQueryRunnerFactory(QueryRunnerTestHelper.NOOP_QUERYWATCHER) + new TimeBoundaryQueryRunnerFactory(QueryRunnerTestHelper.NOOP_QUERYWATCHER), + true ) ); } - private final QueryRunner runner; + private final TestQueryRunner> runner; private static final QueryRunnerFactory FACTORY = new TimeBoundaryQueryRunnerFactory( QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -92,7 +94,7 @@ public static Iterable constructorFeeder() private static Segment segment1; public TimeBoundaryQueryRunnerTest( - QueryRunner runner + TestQueryRunner> runner ) { this.runner = runner; diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java index 7118b2ca50e9..4cca90c083d5 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -65,7 +65,8 @@ public static Iterable constructorFeeder() new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + ), + false ), // descending? Arrays.asList(false, true) diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java index eedbdf8410bb..781d9d31023e 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -115,7 +115,8 @@ public static Iterable constructorFeeder() new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + ), + false ), // descending? Arrays.asList(false, true), diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java index 5c5cba6ece68..ae320ac42613 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.QueryRunnerTestHelper; @@ -41,7 +40,6 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.IndexedInts; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -251,19 +249,6 @@ public int getDimensionCardinality(String column) return cardinality; } - @Override - public DateTime getMinTime() - { - return DateTimes.of(start); - } - - - @Override - public DateTime getMaxTime() - { - return DateTimes.of(end); - } - // stubs below this line not important for tests @Override @@ -305,12 +290,6 @@ public int getNumRows() return 0; } - @Override - public DateTime getMaxIngestedEventTime() - { - return null; - } - @Override public Metadata getMetadata() { diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java index b0f015c627fc..c674e575d703 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java @@ -42,11 +42,13 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.BySegmentResultValue; import org.apache.druid.query.BySegmentResultValueClass; +import org.apache.druid.query.Order; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.Result; +import org.apache.druid.query.TestQueryRunner; import org.apache.druid.query.TestQueryRunners; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; @@ -89,6 +91,9 @@ import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.Cursors; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; @@ -96,6 +101,7 @@ import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.AfterClass; import org.junit.Assert; +import org.junit.Assume; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -129,7 +135,7 @@ public static void teardown() throws IOException @Parameterized.Parameters(name = "{7}") public static Iterable constructorFeeder() { - List>> retVal = queryRunners(); + List>> retVal = queryRunners(true); List parameters = new ArrayList<>(); for (int i = 0; i < 32; i++) { for (QueryRunner> firstParameter : retVal) { @@ -152,7 +158,13 @@ public static Iterable constructorFeeder() return parameters; } - public static List>> queryRunners() + /** + * Create test query runners. + * + * @param includeNonTimeOrdered whether to include runners with non-time-ordered segments. Some test suites are not + * compatible with non-time-ordered segments. + */ + public static List>> queryRunners(boolean includeNonTimeOrdered) { final CloseableStupidPool defaultPool = TestQueryRunners.createDefaultNonBlockingPool(); final CloseableStupidPool customPool = new CloseableStupidPool<>( @@ -167,7 +179,8 @@ public static List>> queryRunners() defaultPool, new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + ), + includeNonTimeOrdered ) ); retVal.addAll( @@ -176,7 +189,8 @@ public static List>> queryRunners() customPool, new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + ), + includeNonTimeOrdered ) ); @@ -191,7 +205,7 @@ public static List>> queryRunners() return retVal; } - private final QueryRunner> runner; + private final TestQueryRunner> runner; private final boolean duplicateSingleAggregatorQueries; private final List commonAggregators; @@ -201,7 +215,7 @@ public static List>> queryRunners() @SuppressWarnings("unused") public TopNQueryRunnerTest( - QueryRunner> runner, + TestQueryRunner> runner, boolean specializeGeneric1AggPooledTopN, boolean specializeGeneric2AggPooledTopN, boolean specializeHistorical1SimpleDoubleAggPooledTopN, @@ -819,6 +833,9 @@ public void testTopNOverHyperUniqueExpressionRounded() @Test public void testTopNOverFirstLastAggregator() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.MONTH_GRAN) @@ -928,6 +945,9 @@ public void testTopNOverFirstLastAggregator() @Test public void testTopNOverFirstLastFloatAggregatorUsingDoubleColumn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.MONTH_GRAN) @@ -1037,6 +1057,9 @@ public void testTopNOverFirstLastFloatAggregatorUsingDoubleColumn() @Test public void testTopNOverFirstLastFloatAggregatorUsingFloatColumn() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.MONTH_GRAN) @@ -6245,6 +6268,9 @@ public void testTopNAggregateTopnMetricFirst() @Test public void testTopNAggregateTopnMetricFirstWithGranularity() { + // Granularity != ALL requires time-ordering. + assumeTimeOrdered(); + TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.MONTH_GRAN) @@ -6352,6 +6378,14 @@ public void testTopNAggregateTopnMetricFirstWithGranularity() assertExpectedResults(expectedResults, query); } + private void assumeTimeOrdered() + { + try (final CursorHolder cursorHolder = + runner.getSegment().asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + Assume.assumeTrue(Cursors.getTimeOrdering(cursorHolder.getOrdering()) == Order.ASCENDING); + } + } + private static Map makeRowWithNulls( String dimName, @Nullable Object dimValue, diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java index 29f6a2c012b9..031d3209a84b 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java @@ -57,7 +57,7 @@ public static void teardown() throws IOException @Parameterized.Parameters(name = "{0}") public static Iterable constructorFeeder() { - return QueryRunnerTestHelper.transformToConstructionFeeder(TopNQueryRunnerTest.queryRunners()); + return QueryRunnerTestHelper.transformToConstructionFeeder(TopNQueryRunnerTest.queryRunners(true)); } private final QueryRunner runner; diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerLongestSharedDimOrderTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerLongestSharedDimOrderTest.java index 1fedf2605530..9d8afc2d6e85 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerLongestSharedDimOrderTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerLongestSharedDimOrderTest.java @@ -78,14 +78,14 @@ public void testGetLongestSharedDimOrderWithNullDimensionSpecAndValidOrdering() QueryableIndexIndexableAdapter index2 = makeIndexWithDimensionList(ImmutableList.of("b", "c")); List actual = IndexMerger.getLongestSharedDimOrder(ImmutableList.of(index1, index2), null); Assert.assertNotNull(actual); - Assert.assertEquals(ImmutableList.of("a", "b", "c"), actual); + Assert.assertEquals(ImmutableList.of("__time", "a", "b", "c"), actual); // Valid ordering as although second index has gap, it is still same ordering index1 = makeIndexWithDimensionList(ImmutableList.of("a", "b", "c")); index2 = makeIndexWithDimensionList(ImmutableList.of("a", "c")); actual = IndexMerger.getLongestSharedDimOrder(ImmutableList.of(index1, index2), null); Assert.assertNotNull(actual); - Assert.assertEquals(ImmutableList.of("a", "b", "c"), actual); + Assert.assertEquals(ImmutableList.of("__time", "a", "b", "c"), actual); } @Test @@ -131,7 +131,7 @@ public void testGetLongestSharedDimOrderWithValidSchemaDimensionSpecAndNoValidOr QueryableIndexIndexableAdapter index2 = makeIndexWithDimensionList(ImmutableList.of("b", "c")); List actual = IndexMerger.getLongestSharedDimOrder(ImmutableList.of(index1, index2), valid); Assert.assertNotNull(actual); - Assert.assertEquals(ImmutableList.of("a", "b", "c"), actual); + Assert.assertEquals(ImmutableList.of("__time", "a", "b", "c"), actual); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java index 7f1e227f1ecb..1f46534266f4 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -37,11 +37,13 @@ import org.apache.druid.data.input.impl.FloatDimensionSchema; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.query.DefaultBitmapResultFactory; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -152,7 +154,9 @@ protected IndexMergerTestBase( ) { this.indexSpec = IndexSpec.builder() - .withBitmapSerdeFactory(bitmapSerdeFactory != null ? bitmapSerdeFactory : new ConciseBitmapSerdeFactory()) + .withBitmapSerdeFactory(bitmapSerdeFactory != null + ? bitmapSerdeFactory + : new ConciseBitmapSerdeFactory()) .withDimensionCompression(dimCompressionStrategy) .withMetricCompression(compressionStrategy) .withLongEncoding(longEncodingStrategy) @@ -176,6 +180,7 @@ public void testPersist() throws Exception Assert.assertEquals(2, index.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(makeOrderBys("__time"), Lists.newArrayList(index.getOrdering())); Assert.assertEquals(3, index.getColumnNames().size()); assertDimCompression(index, indexSpec.getDimensionCompression()); @@ -191,6 +196,144 @@ public void testPersist() throws Exception ); } + @Test + public void testPersistPlainNonTimeOrdered() throws Exception + { + final long timestamp = System.currentTimeMillis(); + + IncrementalIndex toPersist = + new OnheapIncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withDimensionsSpec( + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("__time") + ) + ) + .setForceSegmentSortByTime(false) + .build() + ) + .withMetrics(new CountAggregatorFactory("count")) + .withQueryGranularity(Granularities.NONE) + .withRollup(false) + .build() + ) + .setMaxRowCount(1000000) + .build(); + + // Expect 6 rows (no rollup). + IncrementalIndexTest.populateIndex(timestamp, toPersist); + IncrementalIndexTest.populateIndex(timestamp, toPersist); + IncrementalIndexTest.populateIndex(timestamp + 1, toPersist); + + final File tempDir = temporaryFolder.newFolder(); + QueryableIndex index = closer.closeLater( + indexIO.loadIndex(indexMerger.persist(toPersist, tempDir, indexSpec, null)) + ); + + Assert.assertEquals(6, index.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(makeOrderBys("dim1", "dim2", "__time"), Lists.newArrayList(index.getOrdering())); + Assert.assertEquals(3, index.getColumnNames().size()); + + assertDimCompression(index, indexSpec.getDimensionCompression()); + + Assert.assertArrayEquals( + IncrementalIndexTest.getDefaultCombiningAggregatorFactories(), + index.getMetadata().getAggregators() + ); + + Assert.assertEquals( + Granularities.NONE, + index.getMetadata().getQueryGranularity() + ); + + Assert.assertEquals(6, index.getNumRows()); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of(timestamp, "1", "2", 1L), + ImmutableList.of(timestamp, "1", "2", 1L), + ImmutableList.of(timestamp + 1, "1", "2", 1L), + ImmutableList.of(timestamp, "3", "4", 1L), + ImmutableList.of(timestamp, "3", "4", 1L), + ImmutableList.of(timestamp + 1, "3", "4", 1L) + ), + FrameTestUtil.readRowsFromAdapter(new QueryableIndexStorageAdapter(index), null, false).toList() + ); + } + + @Test + public void testPersistRollupNonTimeOrdered() throws Exception + { + final long timestamp = System.currentTimeMillis(); + + IncrementalIndex toPersist = + new OnheapIncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withDimensionsSpec( + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("__time") + ) + ) + .setForceSegmentSortByTime(false) + .build() + ) + .withMetrics(new CountAggregatorFactory("count")) + .withQueryGranularity(Granularities.NONE) + .withRollup(true) + .build() + ) + .setMaxRowCount(1000000) + .build(); + + // Expect 4 rows: the first two calls to populateIndex should roll up; the third call will create distinct rows. + IncrementalIndexTest.populateIndex(timestamp, toPersist); + IncrementalIndexTest.populateIndex(timestamp, toPersist); + IncrementalIndexTest.populateIndex(timestamp + 1, toPersist); + + final File tempDir = temporaryFolder.newFolder(); + QueryableIndex index = closer.closeLater( + indexIO.loadIndex(indexMerger.persist(toPersist, tempDir, indexSpec, null)) + ); + + Assert.assertEquals(4, index.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(makeOrderBys("dim1", "dim2", "__time"), Lists.newArrayList(index.getOrdering())); + Assert.assertEquals(3, index.getColumnNames().size()); + + assertDimCompression(index, indexSpec.getDimensionCompression()); + + Assert.assertArrayEquals( + IncrementalIndexTest.getDefaultCombiningAggregatorFactories(), + index.getMetadata().getAggregators() + ); + + Assert.assertEquals( + Granularities.NONE, + index.getMetadata().getQueryGranularity() + ); + + Assert.assertEquals(4, index.getNumRows()); + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of(timestamp, "1", "2", 2L), + ImmutableList.of(timestamp + 1, "1", "2", 1L), + ImmutableList.of(timestamp, "3", "4", 2L), + ImmutableList.of(timestamp + 1, "3", "4", 1L) + ), + FrameTestUtil.readRowsFromAdapter(new QueryableIndexStorageAdapter(index), null, false).toList() + ); + } + @Test public void testPersistWithDifferentDims() throws Exception { @@ -217,6 +360,7 @@ public void testPersistWithDifferentDims() throws Exception Assert.assertEquals(2, index.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(makeOrderBys("__time"), Lists.newArrayList(index.getOrdering())); Assert.assertEquals(3, index.getColumnNames().size()); assertDimCompression(index, indexSpec.getDimensionCompression()); @@ -253,6 +397,7 @@ public void testPersistWithSegmentMetadata() throws Exception Assert.assertEquals(2, index.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(makeOrderBys("__time"), Lists.newArrayList(index.getOrdering())); Assert.assertEquals(3, index.getColumnNames().size()); assertDimCompression(index, indexSpec.getDimensionCompression()); @@ -263,7 +408,8 @@ public void testPersistWithSegmentMetadata() throws Exception IncrementalIndexTest.getDefaultCombiningAggregatorFactories(), null, Granularities.NONE, - Boolean.TRUE + Boolean.TRUE, + Cursors.ascendingTimeOrder() ), index.getMetadata() ); @@ -616,7 +762,11 @@ public void testNonLexicographicDimOrderMerge() throws Exception final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); final List rowList = RowIteratorHelper.toList(adapter.getRows()); - Assert.assertEquals(Arrays.asList("d3", "d1", "d2"), ImmutableList.copyOf(adapter.getDimensionNames())); + Assert.assertEquals( + Arrays.asList("__time", "d3", "d1", "d2"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); + Assert.assertEquals(Arrays.asList("d3", "d1", "d2"), ImmutableList.copyOf(adapter.getDimensionNames(false))); Assert.assertEquals(3, rowList.size()); Assert.assertEquals(Arrays.asList("30000", "100", "4000"), rowList.get(0).dimensionValues()); @@ -706,7 +856,11 @@ public void testMergeWithDimensionsList() throws Exception final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); final List rowList = RowIteratorHelper.toList(adapter.getRows()); - Assert.assertEquals(ImmutableList.of("dimA", "dimC"), ImmutableList.copyOf(adapter.getDimensionNames())); + Assert.assertEquals( + ImmutableList.of("__time", "dimA", "dimC"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); + Assert.assertEquals(ImmutableList.of("dimA", "dimC"), ImmutableList.copyOf(adapter.getDimensionNames(false))); Assert.assertEquals(4, rowList.size()); Assert.assertEquals(Arrays.asList(null, "1"), rowList.get(0).dimensionValues()); @@ -739,6 +893,121 @@ public void testMergeWithDimensionsList() throws Exception checkBitmapIndex(Collections.emptyList(), getBitmapIndex(adapter, "dimB", "")); } + @Test + public void testMergePlainNonTimeOrdered() throws Exception + { + IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() + .withDimensionsSpec( + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dimA", null, useBitmapIndexes), + new StringDimensionSchema("dimB", null, useBitmapIndexes), + new StringDimensionSchema("dimC", null, useBitmapIndexes) + ) + ) + .setForceSegmentSortByTime(false) + .build() + ) + .withMetrics(new CountAggregatorFactory("count")) + .withQueryGranularity(Granularities.NONE) + .withRollup(false) + .build(); + + IncrementalIndex toPersist1 = new OnheapIncrementalIndex.Builder() + .setIndexSchema(schema) + .setMaxRowCount(1000) + .build(); + IncrementalIndex toPersist2 = new OnheapIncrementalIndex.Builder() + .setIndexSchema(schema) + .setMaxRowCount(1000) + .build(); + IncrementalIndex toPersist3 = new OnheapIncrementalIndex.Builder() + .setIndexSchema(schema) + .setMaxRowCount(1000) + .build(); + + addDimValuesToIndex(toPersist1, "dimA", Arrays.asList("1", "2")); + addDimValuesToIndex(toPersist2, "dimA", Arrays.asList("1", "2")); + addDimValuesToIndex(toPersist3, "dimC", Arrays.asList("1", "2")); + + + final File tmpDir = temporaryFolder.newFolder(); + final File tmpDir2 = temporaryFolder.newFolder(); + final File tmpDir3 = temporaryFolder.newFolder(); + final File tmpDirMerged = temporaryFolder.newFolder(); + + QueryableIndex index1 = closer.closeLater( + indexIO.loadIndex(indexMerger.persist(toPersist1, tmpDir, indexSpec, null)) + ); + + QueryableIndex index2 = closer.closeLater( + indexIO.loadIndex(indexMerger.persist(toPersist2, tmpDir2, indexSpec, null)) + ); + + QueryableIndex index3 = closer.closeLater( + indexIO.loadIndex(indexMerger.persist(toPersist3, tmpDir3, indexSpec, null)) + ); + + final QueryableIndex merged = closer.closeLater( + indexIO.loadIndex( + indexMerger.mergeQueryableIndex( + Arrays.asList(index1, index2, index3), + true, + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + tmpDirMerged, + indexSpec, + null, + -1 + ) + ) + ); + + // Confirm sort order on the merged QueryableIndex. + Assert.assertEquals(Arrays.asList("dimA", "dimC"), Lists.newArrayList(merged.getAvailableDimensions())); + + // dimB is included even though it's not actually stored. + Assert.assertEquals(makeOrderBys("dimA", "dimB", "dimC", "__time"), Lists.newArrayList(merged.getOrdering())); + + final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); + final List rowList = RowIteratorHelper.toList(adapter.getRows()); + + Assert.assertEquals( + ImmutableList.of("__time", "dimA", "dimC"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); + Assert.assertEquals(ImmutableList.of("dimA", "dimC"), ImmutableList.copyOf(adapter.getDimensionNames(false))); + Assert.assertEquals(4, rowList.size()); + + Assert.assertEquals(Arrays.asList(null, "1"), rowList.get(0).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList.get(0).metricValues()); + + Assert.assertEquals(Arrays.asList(null, "2"), rowList.get(1).dimensionValues()); + Assert.assertEquals(Collections.singletonList(1L), rowList.get(1).metricValues()); + + Assert.assertEquals(Arrays.asList("1", null), rowList.get(2).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList.get(2).metricValues()); + + Assert.assertEquals(Arrays.asList("2", null), rowList.get(3).dimensionValues()); + Assert.assertEquals(Collections.singletonList(2L), rowList.get(3).metricValues()); + + Assert.assertEquals(useBitmapIndexes, adapter.getCapabilities("dimA").hasBitmapIndexes()); + Assert.assertEquals(useBitmapIndexes, adapter.getCapabilities("dimC").hasBitmapIndexes()); + + if (useBitmapIndexes) { + checkBitmapIndex(Arrays.asList(0, 1), getBitmapIndex(adapter, "dimA", null)); + checkBitmapIndex(Collections.singletonList(2), getBitmapIndex(adapter, "dimA", "1")); + checkBitmapIndex(Collections.singletonList(3), getBitmapIndex(adapter, "dimA", "2")); + + checkBitmapIndex(Collections.emptyList(), getBitmapIndex(adapter, "dimB", null)); + + checkBitmapIndex(Arrays.asList(2, 3), getBitmapIndex(adapter, "dimC", null)); + checkBitmapIndex(Collections.singletonList(0), getBitmapIndex(adapter, "dimC", "1")); + checkBitmapIndex(Collections.singletonList(1), getBitmapIndex(adapter, "dimC", "2")); + } + + checkBitmapIndex(Collections.emptyList(), getBitmapIndex(adapter, "dimB", "")); + } @Test public void testDisjointDimMerge() throws Exception @@ -780,7 +1049,11 @@ public void testDisjointDimMerge() throws Exception final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); final List rowList = RowIteratorHelper.toList(adapter.getRows()); - Assert.assertEquals(ImmutableList.of("dimA", "dimB"), ImmutableList.copyOf(adapter.getDimensionNames())); + Assert.assertEquals( + ImmutableList.of("__time", "dimA", "dimB"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); + Assert.assertEquals(ImmutableList.of("dimA", "dimB"), ImmutableList.copyOf(adapter.getDimensionNames(false))); Assert.assertEquals(5, rowList.size()); Assert.assertEquals(Arrays.asList(null, "1"), rowList.get(0).dimensionValues()); @@ -911,14 +1184,22 @@ public void testJointDimMerge() throws Exception final List rowList = RowIteratorHelper.toList(adapter.getRows()); if (NullHandling.replaceWithDefault()) { + Assert.assertEquals( + ImmutableList.of("__time", "d2", "d3", "d5", "d6", "d7", "d8", "d9"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); Assert.assertEquals( ImmutableList.of("d2", "d3", "d5", "d6", "d7", "d8", "d9"), - ImmutableList.copyOf(adapter.getDimensionNames()) + ImmutableList.copyOf(adapter.getDimensionNames(false)) ); } else { + Assert.assertEquals( + ImmutableList.of("__time", "d1", "d2", "d3", "d5", "d6", "d7", "d8", "d9"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); Assert.assertEquals( ImmutableList.of("d1", "d2", "d3", "d5", "d6", "d7", "d8", "d9"), - ImmutableList.copyOf(adapter.getDimensionNames()) + ImmutableList.copyOf(adapter.getDimensionNames(false)) ); } Assert.assertEquals(4, rowList.size()); @@ -1085,14 +1366,22 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception final List rowList = RowIteratorHelper.toList(adapter.getRows()); if (NullHandling.replaceWithDefault()) { + Assert.assertEquals( + ImmutableList.of("__time", "d3", "d6", "d8", "d9"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); Assert.assertEquals( ImmutableList.of("d3", "d6", "d8", "d9"), - ImmutableList.copyOf(adapter.getDimensionNames()) + ImmutableList.copyOf(adapter.getDimensionNames(false)) ); } else { + Assert.assertEquals( + ImmutableList.of("__time", "d1", "d2", "d3", "d5", "d6", "d7", "d8", "d9"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); Assert.assertEquals( ImmutableList.of("d1", "d2", "d3", "d5", "d6", "d7", "d8", "d9"), - ImmutableList.copyOf(adapter.getDimensionNames()) + ImmutableList.copyOf(adapter.getDimensionNames(false)) ); } @@ -1232,7 +1521,11 @@ public void testMergeWithSupersetOrdering() throws Exception final QueryableIndexIndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(merged2); final List rowList2 = RowIteratorHelper.toList(adapter2.getRows()); - Assert.assertEquals(ImmutableList.of("dimB", "dimA"), ImmutableList.copyOf(adapter.getDimensionNames())); + Assert.assertEquals( + ImmutableList.of("__time", "dimB", "dimA"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); + Assert.assertEquals(ImmutableList.of("dimB", "dimA"), ImmutableList.copyOf(adapter.getDimensionNames(false))); Assert.assertEquals(5, rowList.size()); Assert.assertEquals(Arrays.asList(null, "1"), rowList.get(0).dimensionValues()); @@ -1259,8 +1552,14 @@ public void testMergeWithSupersetOrdering() throws Exception checkBitmapIndex(Collections.singletonList(3), getBitmapIndex(adapter, "dimB", "2")); checkBitmapIndex(Collections.singletonList(4), getBitmapIndex(adapter, "dimB", "3")); - - Assert.assertEquals(ImmutableList.of("dimA", "dimB", "dimC"), ImmutableList.copyOf(adapter2.getDimensionNames())); + Assert.assertEquals( + ImmutableList.of("__time", "dimA", "dimB", "dimC"), + ImmutableList.copyOf(adapter2.getDimensionNames(true)) + ); + Assert.assertEquals( + ImmutableList.of("dimA", "dimB", "dimC"), + ImmutableList.copyOf(adapter2.getDimensionNames(false)) + ); Assert.assertEquals(12, rowList2.size()); Assert.assertEquals(Arrays.asList(null, null, "1"), rowList2.get(0).dimensionValues()); Assert.assertEquals(Collections.singletonList(1L), rowList2.get(0).metricValues()); @@ -1630,7 +1929,14 @@ public void testMergeNumericDims() throws Exception final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); final List rowList = RowIteratorHelper.toList(adapter.getRows()); - Assert.assertEquals(ImmutableList.of("dimA", "dimB", "dimC"), ImmutableList.copyOf(adapter.getDimensionNames())); + Assert.assertEquals( + ImmutableList.of("__time", "dimA", "dimB", "dimC"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); + Assert.assertEquals( + ImmutableList.of("dimA", "dimB", "dimC"), + ImmutableList.copyOf(adapter.getDimensionNames(false)) + ); Assert.assertEquals(4, rowList.size()); Assert.assertEquals( @@ -1859,6 +2165,7 @@ public void testMultiValueHandling() throws Exception Assert.assertEquals(2, index.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(makeOrderBys("__time", "dim1", "dim2"), Lists.newArrayList(index.getOrdering())); Assert.assertEquals(3, index.getColumnNames().size()); Assert.assertEquals(2, rowList.size()); @@ -1891,6 +2198,7 @@ public void testMultiValueHandling() throws Exception Assert.assertEquals(1, index.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(makeOrderBys("__time", "dim1", "dim2"), Lists.newArrayList(index.getOrdering())); Assert.assertEquals(3, index.getColumnNames().size()); adapter = new QueryableIndexIndexableAdapter(index); @@ -1922,6 +2230,7 @@ public void testMultiValueHandling() throws Exception Assert.assertEquals(2, index.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(makeOrderBys("__time", "dim1", "dim2"), Lists.newArrayList(index.getOrdering())); Assert.assertEquals(3, index.getColumnNames().size()); adapter = new QueryableIndexIndexableAdapter(index); @@ -1978,10 +2287,8 @@ public void testDimensionWithEmptyName() throws Exception ); Assert.assertEquals(3, index.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getLength()); - Assert.assertEquals( - Arrays.asList("dim1", "dim2"), - Lists.newArrayList(index.getAvailableDimensions()) - ); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(makeOrderBys("__time"), Lists.newArrayList(index.getOrdering())); Assert.assertEquals(3, index.getColumnNames().size()); assertDimCompression(index, indexSpec.getDimensionCompression()); @@ -2087,9 +2394,13 @@ public void testMultivalDim_mergeAcrossSegments_rollupWorks() throws Exception final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); final List rowList = RowIteratorHelper.toList(adapter.getRows()); + Assert.assertEquals( + ImmutableList.of("__time", "dimA", "dimMultiVal"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); Assert.assertEquals( ImmutableList.of("dimA", "dimMultiVal"), - ImmutableList.copyOf(adapter.getDimensionNames()) + ImmutableList.copyOf(adapter.getDimensionNames(false)) ); Assert.assertEquals(3, rowList.size()); @@ -2317,9 +2628,13 @@ public void testMultivalDim_persistAndMerge_dimensionValueOrderingRules() throws final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); final List rowList = RowIteratorHelper.toList(adapter.getRows()); + Assert.assertEquals( + ImmutableList.of("__time", "dimA", "dimMultiVal"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); Assert.assertEquals( ImmutableList.of("dimA", "dimMultiVal"), - ImmutableList.copyOf(adapter.getDimensionNames()) + ImmutableList.copyOf(adapter.getDimensionNames(false)) ); if (NullHandling.replaceWithDefault()) { @@ -2453,9 +2768,13 @@ private void validateTestMaxColumnsToMergeOutputSegment(QueryableIndex merged) final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); final List rowList = RowIteratorHelper.toList(adapter.getRows()); + Assert.assertEquals( + ImmutableList.of("__time", "d1", "d2", "d3", "d4", "d5"), + ImmutableList.copyOf(adapter.getDimensionNames(true)) + ); Assert.assertEquals( ImmutableList.of("d1", "d2", "d3", "d4", "d5"), - ImmutableList.copyOf(adapter.getDimensionNames()) + ImmutableList.copyOf(adapter.getDimensionNames(false)) ); Assert.assertEquals(4, rowList.size()); @@ -2725,4 +3044,9 @@ private List makeDimensionSchemas( ) .collect(Collectors.toList()); } + + private static List makeOrderBys(final String... columnNames) + { + return Arrays.stream(columnNames).map(OrderBy::ascending).collect(Collectors.toList()); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/MergingRowIteratorTest.java b/processing/src/test/java/org/apache/druid/segment/MergingRowIteratorTest.java index 9117fb05be58..47e673f83c16 100644 --- a/processing/src/test/java/org/apache/druid/segment/MergingRowIteratorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/MergingRowIteratorTest.java @@ -166,6 +166,7 @@ private TestRowIterator(Iterable timestamps) this.timestamps = timestamps.iterator(); this.rowPointer = new RowPointer( currentTimestamp, + 0, ColumnValueSelector.EMPTY_ARRAY, Collections.emptyList(), ColumnValueSelector.EMPTY_ARRAY, @@ -174,6 +175,7 @@ private TestRowIterator(Iterable timestamps) ); this.markedRowPointer = new TimeAndDimsPointer( markedTimestamp, + 0, ColumnValueSelector.EMPTY_ARRAY, Collections.emptyList(), ColumnValueSelector.EMPTY_ARRAY, diff --git a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java index c763e66b1ed4..b158480b22d6 100644 --- a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java +++ b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.OrderBy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; @@ -31,10 +32,13 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; /** + * */ public class MetadataTest { @@ -43,7 +47,7 @@ public void testSerde() throws Exception { ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); - AggregatorFactory[] aggregators = new AggregatorFactory[] { + AggregatorFactory[] aggregators = new AggregatorFactory[]{ new LongSumAggregatorFactory("out", "in") }; @@ -52,7 +56,8 @@ public void testSerde() throws Exception aggregators, null, Granularities.ALL, - Boolean.FALSE + Boolean.FALSE, + null ); Metadata other = jsonMapper.readValue( @@ -75,7 +80,7 @@ public void testMerge() Assert.assertNull(Metadata.merge(metadataToBeMerged, null)); //sanity merge check - AggregatorFactory[] aggs = new AggregatorFactory[] { + AggregatorFactory[] aggs = new AggregatorFactory[]{ new LongMaxAggregatorFactory("n", "f") }; final Metadata m1 = new Metadata( @@ -83,7 +88,8 @@ public void testMerge() aggs, new TimestampSpec("ds", "auto", null), Granularities.ALL, - Boolean.FALSE + Boolean.FALSE, + null ); final Metadata m2 = new Metadata( @@ -91,7 +97,8 @@ public void testMerge() aggs, new TimestampSpec("ds", "auto", null), Granularities.ALL, - Boolean.FALSE + Boolean.FALSE, + null ); final Metadata m3 = new Metadata( @@ -99,7 +106,8 @@ public void testMerge() aggs, new TimestampSpec("ds", "auto", null), Granularities.ALL, - Boolean.TRUE + Boolean.TRUE, + null ); final Metadata merged = new Metadata( @@ -109,7 +117,8 @@ public void testMerge() }, new TimestampSpec("ds", "auto", null), Granularities.ALL, - Boolean.FALSE + Boolean.FALSE, + Cursors.ascendingTimeOrder() ); Assert.assertEquals(merged, Metadata.merge(ImmutableList.of(m1, m2), null)); @@ -119,16 +128,18 @@ public void testMerge() metadataToBeMerged.add(m2); metadataToBeMerged.add(null); - final Metadata merged2 = new Metadata(Collections.singletonMap("k", "v"), null, null, null, null); + final Metadata merged2 = + new Metadata(Collections.singletonMap("k", "v"), null, null, null, null, Cursors.ascendingTimeOrder()); Assert.assertEquals(merged2, Metadata.merge(metadataToBeMerged, null)); //merge check with client explicitly providing merged aggregators - AggregatorFactory[] explicitAggs = new AggregatorFactory[] { + AggregatorFactory[] explicitAggs = new AggregatorFactory[]{ new DoubleMaxAggregatorFactory("x", "y") }; - final Metadata merged3 = new Metadata(Collections.singletonMap("k", "v"), explicitAggs, null, null, null); + final Metadata merged3 = + new Metadata(Collections.singletonMap("k", "v"), explicitAggs, null, null, null, Cursors.ascendingTimeOrder()); Assert.assertEquals( merged3, @@ -140,11 +151,76 @@ public void testMerge() explicitAggs, new TimestampSpec("ds", "auto", null), Granularities.ALL, - null + null, + Cursors.ascendingTimeOrder() ); Assert.assertEquals( merged4, Metadata.merge(ImmutableList.of(m3, m2), explicitAggs) ); } + + @Test + public void testMergeOrderings() + { + Assert.assertThrows( + IllegalArgumentException.class, + () -> Metadata.mergeOrderings(Collections.emptyList()) + ); + + Assert.assertEquals( + Cursors.ascendingTimeOrder(), + Metadata.mergeOrderings(Collections.singletonList(null)) + ); + + Assert.assertEquals( + Collections.emptyList(), + Metadata.mergeOrderings(Arrays.asList(null, makeOrderBy("foo", "bar"))) + ); + + Assert.assertEquals( + Collections.emptyList(), + Metadata.mergeOrderings(Arrays.asList(makeOrderBy("foo", "bar"), null)) + ); + + Assert.assertEquals( + Cursors.ascendingTimeOrder(), + Metadata.mergeOrderings(Arrays.asList(makeOrderBy("__time", "foo", "bar"), null)) + ); + + Assert.assertEquals( + Collections.emptyList(), + Metadata.mergeOrderings( + Arrays.asList( + makeOrderBy("foo", "bar"), + makeOrderBy("bar", "foo") + ) + ) + ); + + Assert.assertEquals( + Collections.singletonList(OrderBy.ascending("bar")), + Metadata.mergeOrderings( + Arrays.asList( + makeOrderBy("bar", "baz"), + makeOrderBy("bar", "foo") + ) + ) + ); + + Assert.assertEquals( + ImmutableList.of(OrderBy.ascending("bar"), OrderBy.ascending("foo")), + Metadata.mergeOrderings( + Arrays.asList( + makeOrderBy("bar", "foo"), + makeOrderBy("bar", "foo") + ) + ) + ); + } + + private static List makeOrderBy(final String... columnNames) + { + return Arrays.stream(columnNames).map(OrderBy::ascending).collect(Collectors.toList()); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index dca89de18b12..6c46050c6971 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.math.LongMath; +import junitparams.converters.Nullable; import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.guava.GuavaUtils; import org.apache.druid.java.util.common.DateTimes; @@ -322,20 +323,6 @@ public void test_getDimensionCardinality_timeColumn() Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, adapter.getDimensionCardinality("__time")); } - @Test - public void test_getMinTime() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - Assert.assertEquals(Intervals.ETERNITY.getStart(), adapter.getMinTime()); - } - - @Test - public void test_getMaxTime() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - Assert.assertEquals(Intervals.ETERNITY.getEnd().minus(1), adapter.getMaxTime()); - } - @Test public void test_getMinValue() { @@ -476,13 +463,6 @@ public void test_getNumRows() assertThrows(UnsupportedOperationException.class, () -> adapter.getMetadata()); } - @Test - public void test_getMaxIngestedEventTime() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - Assert.assertEquals(Intervals.ETERNITY.getEnd().minus(1), adapter.getMaxIngestedEventTime()); - } - @Test public void test_getMetadata() { @@ -665,7 +645,6 @@ public void test_makeCursor_hourGranularity() .setInterval(Intervals.of("1970/1971")) .build(); try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { - final Cursor cursor = cursorHolder.asCursor(); Assert.assertEquals( ImmutableList.of( ImmutableList.of(DateTimes.of("1970-01-01T00"), "0"), @@ -674,7 +653,7 @@ public void test_makeCursor_hourGranularity() ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"), ImmutableList.of(DateTimes.of("1970-01-01T03"), "3") ), - walkCursorGranularized(adapter, cursor, buildSpec, Granularities.HOUR, READ_TIME_AND_STRING_GRAN) + walkCursorGranularized(cursorHolder, null, buildSpec, Granularities.HOUR, READ_TIME_AND_STRING_GRAN) ); } @@ -691,15 +670,13 @@ public void test_makeCursor_hourGranularityWithInterval() .build(); try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { - final Cursor cursor = cursorHolder.asCursor(); - Assert.assertEquals( ImmutableList.of( ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), ImmutableList.of(DateTimes.of("1970-01-01T02"), "2") ), - walkCursorGranularized(adapter, cursor, buildSpec, Granularities.HOUR, READ_TIME_AND_STRING_GRAN) + walkCursorGranularized(cursorHolder, null, buildSpec, Granularities.HOUR, READ_TIME_AND_STRING_GRAN) ); } @@ -717,14 +694,13 @@ public void test_makeCursor_hourGranularityWithIntervalDescending() .build(); try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { - final Cursor cursor = cursorHolder.asCursor(); Assert.assertEquals( ImmutableList.of( ImmutableList.of(DateTimes.of("1970-01-01T02"), "2"), ImmutableList.of(DateTimes.of("1970-01-01T01"), "1"), ImmutableList.of(DateTimes.of("1970-01-01T01"), "1") ), - walkCursorGranularized(adapter, cursor, buildSpec, Granularities.HOUR, READ_TIME_AND_STRING_GRAN) + walkCursorGranularized(cursorHolder, null, buildSpec, Granularities.HOUR, READ_TIME_AND_STRING_GRAN) ); } @@ -890,19 +866,21 @@ private static List> walkCursor( } private static List> walkCursorGranularized( - final StorageAdapter adapter, - final Cursor cursor, + final CursorHolder cursorHolder, + @Nullable final TimeBoundaryInspector timeBoundaryInspector, final CursorBuildSpec buildSpec, final Granularity granularity, final List>> processors ) { + final Cursor cursor = cursorHolder.asCursor(); + CursorGranularizer granularizer = CursorGranularizer.create( - adapter, cursor, + timeBoundaryInspector, + Cursors.getTimeOrdering(cursorHolder.getOrdering()), granularity, - buildSpec.getInterval(), - Cursors.preferDescendingTimeOrdering(buildSpec) + buildSpec.getInterval() ); final List> suppliers = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/segment/TestIndex.java b/processing/src/test/java/org/apache/druid/segment/TestIndex.java index 4aceafa5f3a0..1657a77b2094 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestIndex.java +++ b/processing/src/test/java/org/apache/druid/segment/TestIndex.java @@ -111,6 +111,23 @@ public class TestIndex new StringDimensionSchema("null_column") ); + public static final List DIMENSION_SCHEMAS_NON_TIME_ORDERED = Arrays.asList( + new StringDimensionSchema("market"), + new StringDimensionSchema("quality"), + new LongDimensionSchema("__time"), + new LongDimensionSchema("qualityLong"), + new FloatDimensionSchema("qualityFloat"), + new DoubleDimensionSchema("qualityDouble"), + new StringDimensionSchema("qualityNumericString"), + new LongDimensionSchema("longNumericNull"), + new FloatDimensionSchema("floatNumericNull"), + new DoubleDimensionSchema("doubleNumericNull"), + new StringDimensionSchema("placement"), + new StringDimensionSchema("placementish"), + new StringDimensionSchema("partial_null_column"), + new StringDimensionSchema("null_column") + ); + public static final List DIMENSION_SCHEMAS_NO_BITMAP = Arrays.asList( new StringDimensionSchema("market", null, false), new StringDimensionSchema("quality", null, false), @@ -128,7 +145,8 @@ public class TestIndex ); public static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec(DIMENSION_SCHEMAS); - + public static final DimensionsSpec DIMENSIONS_SPEC_NON_TIME_ORDERED = + new DimensionsSpec(DIMENSION_SCHEMAS_NON_TIME_ORDERED); public static final DimensionsSpec DIMENSIONS_SPEC_NO_BITMAPS = new DimensionsSpec(DIMENSION_SCHEMAS_NO_BITMAP); public static final String[] DOUBLE_METRICS = new String[]{"index", "indexMin", "indexMaxPlusTen"}; @@ -162,15 +180,27 @@ public class TestIndex private static Supplier realtimeIndex = Suppliers.memoize( () -> makeRealtimeIndex("druid.sample.numeric.tsv") ); + private static Supplier nonTimeOrderedRealtimeIndex = Suppliers.memoize( + () -> makeRealtimeIndex("druid.sample.numeric.tsv", true, DIMENSIONS_SPEC_NON_TIME_ORDERED) + ); + private static Supplier nonTimeOrderedNoRollupRealtimeIndex = Suppliers.memoize( + () -> makeRealtimeIndex("druid.sample.numeric.tsv", false, DIMENSIONS_SPEC_NON_TIME_ORDERED) + ); private static Supplier noRollupRealtimeIndex = Suppliers.memoize( () -> makeRealtimeIndex("druid.sample.numeric.tsv", false) ); private static Supplier noBitmapRealtimeIndex = Suppliers.memoize( - () -> makeRealtimeIndex("druid.sample.numeric.tsv", false, false) + () -> makeRealtimeIndex("druid.sample.numeric.tsv", false, DIMENSIONS_SPEC_NO_BITMAPS) ); private static Supplier mmappedIndex = Suppliers.memoize( () -> persistRealtimeAndLoadMMapped(realtimeIndex.get()) ); + private static Supplier nonTimeOrderedMmappedIndex = Suppliers.memoize( + () -> persistRealtimeAndLoadMMapped(nonTimeOrderedRealtimeIndex.get()) + ); + private static Supplier nonTimeOrderedNoRollupMmappedIndex = Suppliers.memoize( + () -> persistRealtimeAndLoadMMapped(nonTimeOrderedNoRollupRealtimeIndex.get()) + ); private static Supplier noRollupMmappedIndex = Suppliers.memoize( () -> persistRealtimeAndLoadMMapped(noRollupRealtimeIndex.get()) ); @@ -264,6 +294,26 @@ public static QueryableIndex getNoBitmapMMappedTestIndex() return noBitmapMmappedIndex.get(); } + public static IncrementalIndex getNonTimeOrderedRealtimeTestIndex() + { + return nonTimeOrderedRealtimeIndex.get(); + } + + public static IncrementalIndex getNonTimeOrderedNoRollupRealtimeTestIndex() + { + return nonTimeOrderedNoRollupRealtimeIndex.get(); + } + + public static QueryableIndex getNonTimeOrderedMMappedTestIndex() + { + return nonTimeOrderedMmappedIndex.get(); + } + + public static QueryableIndex getNonTimeOrderedNoRollupMMappedTestIndex() + { + return nonTimeOrderedNoRollupMmappedIndex.get(); + } + public static QueryableIndex mergedRealtimeIndex() { return mergedRealtime.get(); @@ -281,13 +331,17 @@ public static IncrementalIndex makeRealtimeIndex(final String resourceFilename) public static IncrementalIndex makeRealtimeIndex(final String resourceFilename, boolean rollup) { - return makeRealtimeIndex(resourceFilename, rollup, true); + return makeRealtimeIndex(resourceFilename, rollup, DIMENSIONS_SPEC); } - public static IncrementalIndex makeRealtimeIndex(final String resourceFilename, boolean rollup, boolean bitmap) + public static IncrementalIndex makeRealtimeIndex( + final String resourceFilename, + boolean rollup, + DimensionsSpec dimensionsSpec + ) { CharSource stream = getResourceCharSource(resourceFilename); - return makeRealtimeIndex(stream, rollup, bitmap); + return makeRealtimeIndex(stream, rollup, dimensionsSpec); } public static CharSource getResourceCharSource(final String resourceFilename) @@ -302,15 +356,19 @@ public static CharSource getResourceCharSource(final String resourceFilename) public static IncrementalIndex makeRealtimeIndex(final CharSource source) { - return makeRealtimeIndex(source, true, true); + return makeRealtimeIndex(source, true, DIMENSIONS_SPEC); } - public static IncrementalIndex makeRealtimeIndex(final CharSource source, boolean rollup, boolean bitmap) + public static IncrementalIndex makeRealtimeIndex( + final CharSource source, + boolean rollup, + DimensionsSpec dimensionsSpec + ) { final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() .withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis()) .withTimestampSpec(new TimestampSpec("ds", "auto", null)) - .withDimensionsSpec(bitmap ? DIMENSIONS_SPEC : DIMENSIONS_SPEC_NO_BITMAPS) + .withDimensionsSpec(dimensionsSpec) .withVirtualColumns(VIRTUAL_COLUMNS) .withMetrics(METRIC_AGGS) .withRollup(rollup) diff --git a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java index 8950693f14fa..93266894c695 100644 --- a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java @@ -21,7 +21,6 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.Indexed; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -62,12 +61,6 @@ public int getNumRows() return 0; } - @Override - public DateTime getMaxIngestedEventTime() - { - return null; - } - @Override public Indexed getAvailableDimensions() { @@ -86,18 +79,6 @@ public int getDimensionCardinality(String column) return 0; } - @Override - public DateTime getMinTime() - { - return null; - } - - @Override - public DateTime getMaxTime() - { - return null; - } - @Nullable @Override public Comparable getMinValue(String column) @@ -125,7 +106,6 @@ public Metadata getMetadata() { return null; } - }; Assert.assertFalse(sa.isFromTombstone()); diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java index 8ea5791463fc..96a06e3523b1 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.ResourceInputSource; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequences; @@ -169,21 +168,9 @@ public void test_group_of_unnest_adapters_methods() { String colName = "multi-string1"; for (StorageAdapter adapter : ADAPTERS) { - Assert.assertEquals( - DateTimes.of("2000-01-01T23:00:00.000Z"), - adapter.getMaxTime() - ); - Assert.assertEquals( - DateTimes.of("2000-01-01T12:00:00.000Z"), - adapter.getMinTime() - ); adapter.getColumnCapabilities(colName); Assert.assertEquals(adapter.getNumRows(), 0); Assert.assertNotNull(adapter.getMetadata()); - Assert.assertEquals( - DateTimes.of("2000-01-01T23:59:59.999Z"), - adapter.getMaxIngestedEventTime() - ); Assert.assertEquals( adapter.getColumnCapabilities(colName).toColumnType(), INCREMENTAL_INDEX_STORAGE_ADAPTER.getColumnCapabilities(colName).toColumnType() diff --git a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java index 3143d2f7be74..9c0e3d8f158c 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java @@ -231,7 +231,8 @@ public void testCaseSensitivity() throws Exception IncrementalIndex index = indexCreator.createIndex((Object) DEFAULT_AGGREGATOR_FACTORIES); populateIndex(timestamp, index); - Assert.assertEquals(Arrays.asList("dim1", "dim2"), index.getDimensionNames()); + Assert.assertEquals(Arrays.asList("__time", "dim1", "dim2"), index.getDimensionNames(true)); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), index.getDimensionNames(false)); Assert.assertEquals(2, index.size()); final Iterator rows = index.iterator(); @@ -286,7 +287,8 @@ public void testFilteredAggregators() throws Exception ) ); - Assert.assertEquals(Arrays.asList("dim1", "dim2", "dim3"), index.getDimensionNames()); + Assert.assertEquals(Arrays.asList("__time", "dim1", "dim2", "dim3"), index.getDimensionNames(true)); + Assert.assertEquals(Arrays.asList("dim1", "dim2", "dim3"), index.getDimensionNames(false)); Assert.assertEquals( Arrays.asList( "count", @@ -649,7 +651,8 @@ public void testgetDimensions() .build() ); - Assert.assertEquals(Arrays.asList("dim0", "dim1"), incrementalIndex.getDimensionNames()); + Assert.assertEquals(Arrays.asList("__time", "dim0", "dim1"), incrementalIndex.getDimensionNames(true)); + Assert.assertEquals(Arrays.asList("dim0", "dim1"), incrementalIndex.getDimensionNames(false)); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java index 55191fd192b3..b2bf5dd044b9 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java @@ -19,9 +19,10 @@ package org.apache.druid.segment.incremental; -import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.CloserRule; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -31,8 +32,10 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Map; @@ -42,19 +45,14 @@ @RunWith(Parameterized.class) public class IncrementalIndexRowCompTest extends InitializedNullHandlingTest { - public final IncrementalIndexCreator indexCreator; + public String indexType; @Rule public final CloserRule closer = new CloserRule(false); - public IncrementalIndexRowCompTest(String indexType) throws JsonProcessingException + public IncrementalIndexRowCompTest(String indexType) { - indexCreator = closer.closeLater( - new IncrementalIndexCreator(indexType, (builder, args) -> builder - .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) - .setMaxRowCount(1_000) - .build()) - ); + this.indexType = indexType; } @Parameterized.Parameters(name = "{index}: {0}") @@ -64,10 +62,21 @@ public static Collection constructorFeeder() } @Test - public void testBasic() + public void testBasic() throws IOException { + IncrementalIndexCreator indexCreator = closer.closeLater( + new IncrementalIndexCreator(indexType, (builder, args) -> builder + .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) + .setMaxRowCount(1_000) + .build()) + ); IncrementalIndex index = indexCreator.createIndex(); + // Expected ordering: __time + Assert.assertEquals(0, index.timePosition); + Assert.assertEquals(ImmutableList.of("__time"), index.getDimensionNames(true)); + Assert.assertEquals(Collections.emptyList(), index.getDimensionNames(false)); + long time = System.currentTimeMillis(); IncrementalIndexRow ir1 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "B")).getIncrementalIndexRow(); IncrementalIndexRow ir2 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "A")).getIncrementalIndexRow(); @@ -100,6 +109,69 @@ public void testBasic() Assert.assertTrue(comparator.compare(ir4, ir5) > 0); } + @Test + public void testSortByDim() throws IOException + { + IncrementalIndexCreator indexCreator = closer.closeLater( + new IncrementalIndexCreator( + indexType, + (builder, args) -> builder + .setIndexSchema( + IncrementalIndexSchema + .builder() + .withDimensionsSpec( + DimensionsSpec.builder() + .setDimensions(DimensionsSpec.getDefaultSchemas(Collections.singletonList("joe"))) + .useSchemaDiscovery(true) + .setForceSegmentSortByTime(false) + .build() + ) + .withMetrics(new CountAggregatorFactory("cnt")) + .build() + ) + .setMaxRowCount(1_000) + .build() + ) + ); + IncrementalIndex index = indexCreator.createIndex(); + + // Expected ordering: [joe, __time] + Assert.assertEquals(1, index.timePosition); + Assert.assertEquals(ImmutableList.of("joe", "__time"), index.getDimensionNames(true)); + Assert.assertEquals(ImmutableList.of("joe"), index.getDimensionNames(false)); + + long time = System.currentTimeMillis(); + IncrementalIndexRow ir1 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "B")).getIncrementalIndexRow(); + IncrementalIndexRow ir2 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "A")).getIncrementalIndexRow(); + IncrementalIndexRow ir3 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A")).getIncrementalIndexRow(); + + IncrementalIndexRow ir4 = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", "B")).getIncrementalIndexRow(); + IncrementalIndexRow ir5 = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", Arrays.asList("A", "B"))).getIncrementalIndexRow(); + IncrementalIndexRow ir6 = index.toIncrementalIndexRow(toMapRow(time + 1)).getIncrementalIndexRow(); + + Comparator comparator = index.dimsComparator(); + + Assert.assertEquals(0, comparator.compare(ir1, ir1)); + Assert.assertEquals(0, comparator.compare(ir2, ir2)); + Assert.assertEquals(0, comparator.compare(ir3, ir3)); + + Assert.assertTrue(comparator.compare(ir1, ir2) > 0); + Assert.assertTrue(comparator.compare(ir2, ir1) < 0); + Assert.assertTrue(comparator.compare(ir2, ir3) > 0); + Assert.assertTrue(comparator.compare(ir3, ir2) < 0); + Assert.assertTrue(comparator.compare(ir1, ir3) > 0); + Assert.assertTrue(comparator.compare(ir3, ir1) < 0); + + Assert.assertTrue(comparator.compare(ir6, ir1) < 0); + Assert.assertTrue(comparator.compare(ir6, ir2) < 0); + Assert.assertTrue(comparator.compare(ir6, ir3) > 0); + + Assert.assertTrue(comparator.compare(ir4, ir6) > 0); + Assert.assertTrue(comparator.compare(ir5, ir6) > 0); + Assert.assertTrue(comparator.compare(ir5, ir4) < 0); + Assert.assertTrue(comparator.compare(ir4, ir5) > 0); + } + private MapBasedInputRow toMapRow(long time, Object... dimAndVal) { Map data = new HashMap<>(); diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index c82003e1e84c..bab827130e4f 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -20,13 +20,17 @@ package org.apache.druid.segment.incremental; import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.collections.ResourceHolder; 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.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -63,6 +67,7 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; @@ -73,6 +78,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; +import org.junit.Assume; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -94,23 +100,56 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT { public final IncrementalIndexCreator indexCreator; + /** + * If true, sort by [billy, __time]. If false, sort by [__time]. + */ + public final boolean sortByDim; + @Rule public final CloserRule closer = new CloserRule(false); - public IncrementalIndexStorageAdapterTest(String indexType) throws JsonProcessingException + public IncrementalIndexStorageAdapterTest(String indexType, boolean sortByDim) throws JsonProcessingException { BuiltInTypesModule.registerHandlersAndSerde(); - indexCreator = closer.closeLater(new IncrementalIndexCreator(indexType, (builder, args) -> builder - .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) - .setMaxRowCount(1_000) - .build() - )); + this.sortByDim = sortByDim; + this.indexCreator = closer.closeLater( + new IncrementalIndexCreator( + indexType, + (builder, args) -> { + final DimensionsSpec dimensionsSpec; + + if (sortByDim) { + dimensionsSpec = + DimensionsSpec.builder() + .setDimensions(Collections.singletonList(new StringDimensionSchema("billy"))) + .setForceSegmentSortByTime(false) + .setIncludeAllDimensions(true) + .build(); + } else { + dimensionsSpec = DimensionsSpec.EMPTY; + } + + return builder + .setIndexSchema( + IncrementalIndexSchema + .builder() + .withDimensionsSpec(dimensionsSpec) + .withMetrics(new CountAggregatorFactory("cnt")) + .build() + ) + .setMaxRowCount(1_000) + .build(); + } + ) + ); } - @Parameterized.Parameters(name = "{index}: {0}") + @Parameterized.Parameters(name = "{index}: {0}, sortByDim: {1}") public static Collection constructorFeeder() { - return IncrementalIndexCreator.getAppendableIndexTypes(); + return IncrementalIndexCreator.indexTypeCartesianProduct( + ImmutableList.of(true, false) // sortByDim + ); } @Test @@ -155,6 +194,7 @@ public void testSanity() throws Exception final Sequence rows = GroupByQueryEngine.process( query, adapter, + new IncrementalIndexTimeBoundaryInspector(index), cursorHolder, buildSpec, processingBuffer.get(), @@ -231,6 +271,7 @@ public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception final Sequence rows = GroupByQueryEngine.process( query, adapter, + new IncrementalIndexTimeBoundaryInspector(index), cursorHolder, buildSpec, processingBuffer.get(), @@ -257,6 +298,8 @@ public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception @Test public void testResetSanity() throws IOException { + // Test is only valid when sortByDim = false, due to usage of Granularities.NONE. + Assume.assumeFalse(sortByDim); IncrementalIndex index = indexCreator.createIndex(); DateTime t = DateTimes.nowUtc(); @@ -280,13 +323,13 @@ public void testResetSanity() throws IOException IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); for (boolean descending : Arrays.asList(false, true)) { - final CursorBuildSpec buildSpec = CursorBuildSpec.builder() - .setFilter(new SelectorFilter("sally", "bo")) - .setInterval(interval) - .setPreferredOrdering( - descending ? Cursors.descendingTimeOrder() : null - ) - .build(); + final CursorBuildSpec buildSpec = CursorBuildSpec + .builder() + .setFilter(new SelectorFilter("sally", "bo")) + .setInterval(interval) + .setPreferredOrdering(descending ? Cursors.descendingTimeOrder() : Cursors.ascendingTimeOrder()) + .build(); + try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); DimensionSelector dimSelector; @@ -348,6 +391,7 @@ public void testSingleValueTopN() throws IOException .aggregators(new LongSumAggregatorFactory("cnt", "cnt")) .build(), new IncrementalIndexStorageAdapter(index), + new IncrementalIndexTimeBoundaryInspector(index), null ) .toList(); @@ -400,6 +444,7 @@ public void testFilterByNull() throws Exception final Sequence rows = GroupByQueryEngine.process( query, adapter, + new IncrementalIndexTimeBoundaryInspector(index), cursorHolder, buildSpec, processingBuffer.get(), @@ -519,14 +564,19 @@ public void testCursoringAndSnapshot() throws Exception final IncrementalIndex index = indexCreator.createIndex(); final long timestamp = System.currentTimeMillis(); + final List rows = ImmutableList.of( + new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v00")), + new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v01")), + new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v1")), + new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v2")), + new MapBasedInputRow(timestamp, Collections.singletonList("billy2"), ImmutableMap.of("billy2", "v3")), + new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v3")), + new MapBasedInputRow(timestamp, Collections.singletonList("billy3"), ImmutableMap.of("billy3", "")) + ); + + // Add first two rows. for (int i = 0; i < 2; i++) { - index.add( - new MapBasedInputRow( - timestamp, - Collections.singletonList("billy"), - ImmutableMap.of("billy", "v0" + i) - ) - ); + index.add(rows.get(i)); } final StorageAdapter sa = new IncrementalIndexStorageAdapter(index); @@ -544,7 +594,7 @@ public void testCursoringAndSnapshot() throws Exception //index gets more rows at this point, while other thread is iterating over the cursor try { - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v1"))); + index.add(rows.get(2)); } catch (Exception ex) { throw new RuntimeException(ex); @@ -555,12 +605,8 @@ public void testCursoringAndSnapshot() throws Exception .makeDimensionSelector(new DefaultDimensionSpec("billy", "billy")); //index gets more rows at this point, while other thread is iterating over the cursor try { - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v2"))); - index.add(new MapBasedInputRow( - timestamp, - Collections.singletonList("billy2"), - ImmutableMap.of("billy2", "v3") - )); + index.add(rows.get(3)); + index.add(rows.get(4)); } catch (Exception ex) { throw new RuntimeException(ex); @@ -575,8 +621,8 @@ public void testCursoringAndSnapshot() throws Exception .makeDimensionSelector(new DefaultDimensionSpec("billy2", "billy2")); //index gets more rows at this point, while other thread is iterating over the cursor try { - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy"), ImmutableMap.of("billy", "v3"))); - index.add(new MapBasedInputRow(timestamp, Collections.singletonList("billy3"), ImmutableMap.of("billy3", ""))); + index.add(rows.get(5)); + index.add(rows.get(6)); } catch (Exception ex) { throw new RuntimeException(ex); diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java index e323b2599131..b26da1ceba79 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java @@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableSet; 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.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; @@ -127,24 +126,6 @@ public void test_getDimensionCardinality_factToCountryNonexistentJoinColumn() ); } - @Test - public void test_getMinTime_factToCountry() - { - Assert.assertEquals( - DateTimes.of("2015-09-12T00:46:58.771Z"), - makeFactToCountrySegment().getMinTime() - ); - } - - @Test - public void test_getMaxTime_factToCountry() - { - Assert.assertEquals( - DateTimes.of("2015-09-12T05:21:00.059Z"), - makeFactToCountrySegment().getMaxTime() - ); - } - @Test public void test_getMinValue_factToCountryFactColumn() { @@ -268,15 +249,6 @@ public void test_getColumnCapabilities_typeString_factToCountryJoinColumn() ); } - @Test - public void test_getMaxIngestedEventTime_factToCountry() - { - Assert.assertEquals( - DateTimes.of("2015-09-12T05:21:00.059Z"), - makeFactToCountrySegment().getMaxIngestedEventTime() - ); - } - @Test public void test_getNumRows_factToCountry() { diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java index 581c9a1c705f..9f9905e47d47 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java @@ -20,13 +20,16 @@ package org.apache.druid.segment.join; import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.segment.MaxIngestedEventTimeInspector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; @@ -39,6 +42,7 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; @@ -164,6 +168,13 @@ public StorageAdapter asStorageAdapter() return referencedSegment.asStorageAdapter(); } + @Nullable + @Override + public T as(@Nonnull Class clazz) + { + return referencedSegment.as(clazz); + } + @Override public void close() { @@ -308,4 +319,22 @@ public void testJoinableClausesClosedIfJoinableOneIsAlreadyClosed() Assert.assertEquals(1, indexedTableJoinableReferenceCloseCount); Assert.assertEquals(0, allReferencesCloseCount); } + + + @Test + public void testGetMinTime() + { + final TimeBoundaryInspector timeBoundaryInspector = hashJoinSegment.as(TimeBoundaryInspector.class); + Assert.assertNotNull("non-null inspector", timeBoundaryInspector); + Assert.assertEquals("minTime", DateTimes.of("2015-09-12T00:46:58.771Z"), timeBoundaryInspector.getMinTime()); + Assert.assertEquals("maxTime", DateTimes.of("2015-09-12T05:21:00.059Z"), timeBoundaryInspector.getMaxTime()); + Assert.assertFalse("exact", timeBoundaryInspector.isMinMaxExact()); + } + + @Test + public void testGetMaxIngestedEventTime() + { + final MaxIngestedEventTimeInspector inspector = referencedSegment.as(MaxIngestedEventTimeInspector.class); + Assert.assertNull(inspector); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java index 2d929f079bfd..73daadc7dd1f 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java @@ -326,7 +326,7 @@ private void checkColumnSelectorFactory(String columnName) closer.register(theColumn); final BaseObjectColumnValueSelector selector = theColumn.makeColumnValueSelector(offset); - ColumnSelectorFactory tableFactory = broadcastTable.makeColumnSelectorFactory(offset, false, closer); + ColumnSelectorFactory tableFactory = broadcastTable.makeColumnSelectorFactory(offset, closer); final BaseObjectColumnValueSelector tableSelector = tableFactory.makeColumnValueSelector(columnName); // compare with base segment selector to make sure tables selector can read correct values diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java index 09ddec72b6bc..b08249f2a4d5 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java @@ -181,7 +181,6 @@ public void makeJoinMatcherWithDimensionSelectorOnString() dummyColumnSelectorFactory, condition, false, - false, Closer.create() ); diff --git a/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeShardSpecTest.java b/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeShardSpecTest.java index fa49a48bcfa2..ade40e6ae38f 100644 --- a/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeShardSpecTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/partition/DimensionRangeShardSpecTest.java @@ -406,6 +406,29 @@ public void testPossibleInDomain_nonNullValues_acceptanceScenarios() // EffectiveDomain[:3].size > 1 -> ACCEPT ); assertTrue(shard.possibleInDomain(domain)); + + // {Earth} * {France} * {Paris} + // Tests that we accept the ending point of the range + populateDomain( + domain, + getRangeSet(Range.singleton("Earth")), + getRangeSet(Range.singleton("France")), + getRangeSet(Range.singleton("Paris")) + ); + assertTrue(shard.possibleInDomain(domain)); + + // {Earth} * {USA} * {New York} + // Tests that we accept the ending point of the range + // This is important because shard specs generated by MSQ with "boosting" do not include the __boost column, + // which means that a data point matching the end of a range may be in this segment or may be in the next + // segment (so we do need to check both). + populateDomain( + domain, + getRangeSet(Range.singleton("Earth")), + getRangeSet(Range.singleton("USA")), + getRangeSet(Range.singleton("New York")) + ); + assertTrue(shard.possibleInDomain(domain)); } @Test diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index 1e717f5d17b6..bda884018812 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -31,15 +31,18 @@ import com.google.common.collect.Multiset; import com.google.common.collect.TreeMultiset; import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.TransformSpec; @@ -212,16 +215,51 @@ private static Set computeAndValidateOutputFieldNames( fields.computeIfAbsent(ColumnHolder.TIME_COLUMN_NAME, k -> TreeMultiset.create()).add( StringUtils.format( - "primary timestamp (%s cannot appear as a dimension or metric)", + "primary timestamp (%s cannot appear elsewhere except as long-typed dimension)", ColumnHolder.TIME_COLUMN_NAME ) ); if (dimensionsSpec != null) { + boolean sawTimeDimension = false; + for (int i = 0; i < dimensionsSpec.getDimensions().size(); i++) { - final String field = dimensionsSpec.getDimensions().get(i).getName(); + final DimensionSchema dimSchema = dimensionsSpec.getDimensions().get(i); + final String field = dimSchema.getName(); if (Strings.isNullOrEmpty(field)) { - throw new IAE("Encountered dimension with null or empty name at position %d", i); + throw DruidException + .forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Encountered dimension with null or empty name at position[%d]", i); + } + + if (ColumnHolder.TIME_COLUMN_NAME.equals(field)) { + if (i > 0 && dimensionsSpec.isForceSegmentSortByTime()) { + throw DruidException + .forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Encountered dimension[%s] at position[%d]. This is only supported when the dimensionsSpec " + + "parameter[%s] is set to[false]. %s", + field, + i, + DimensionsSpec.PARAMETER_FORCE_TIME_SORT, + DimensionsSpec.WARNING_NON_TIME_SORT_ORDER + ); + } else if (!dimSchema.getColumnType().is(ValueType.LONG)) { + throw DruidException + .forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Encountered dimension[%s] with incorrect type[%s]. Type must be 'long'.", + field, + dimSchema.getColumnType() + ); + } else if (!sawTimeDimension) { + // Skip adding __time to "fields" (once) if it's listed as a dimension, so it doesn't show up as an error. + sawTimeDimension = true; + continue; + } } fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("dimensions list"); @@ -266,7 +304,9 @@ private static Set computeAndValidateOutputFieldNames( if (errors.isEmpty()) { return fields.keySet(); } else { - throw new IAE("Cannot specify a column more than once: %s", String.join("; ", errors)); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Cannot specify a column more than once: %s", String.join("; ", errors)); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 72cfe4de2615..557198f14244 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -28,7 +28,6 @@ import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.ForegroundCachePopulator; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.FunctionalIterable; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -57,7 +56,7 @@ import org.apache.druid.query.spec.SpecificSegmentQueryRunner; import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.sink.Sink; import org.apache.druid.segment.realtime.sink.SinkSegmentReference; @@ -245,15 +244,21 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final // 1) Only use caching if data is immutable // 2) Hydrants are not the same between replicas, make sure cache is local if (segmentReference.isImmutable() && cache.isLocal()) { - StorageAdapter storageAdapter = segmentReference.getSegment().asStorageAdapter(); - long segmentMinTime = storageAdapter.getMinTime().getMillis(); - long segmentMaxTime = storageAdapter.getMaxTime().getMillis(); - Interval actualDataInterval = Intervals.utc(segmentMinTime, segmentMaxTime + 1); + final SegmentReference segment = segmentReference.getSegment(); + final TimeBoundaryInspector timeBoundaryInspector = segment.as(TimeBoundaryInspector.class); + final Interval cacheKeyInterval; + + if (timeBoundaryInspector != null) { + cacheKeyInterval = timeBoundaryInspector.getMinMaxInterval(); + } else { + cacheKeyInterval = segment.getDataInterval(); + } + runner = new CachingQueryRunner<>( makeHydrantCacheIdentifier(sinkSegmentId, segmentReference.getHydrantNumber()), cacheKeyPrefix, descriptor, - actualDataInterval, + cacheKeyInterval, objectMapper, cache, toolChest, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java index 44d0db3cb7f6..8d4fc0dd7f8b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java @@ -349,7 +349,7 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) Map oldFormat = null; newCount = lastHydrant.getCount() + 1; - boolean customDimensions = !indexSchema.getDimensionsSpec().hasCustomDimensions(); + boolean variableDimensions = !indexSchema.getDimensionsSpec().hasFixedDimensions(); if (lastHydrant.hasSwapped()) { oldFormat = new HashMap<>(); @@ -357,7 +357,7 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) try { QueryableIndex oldIndex = segment.asQueryableIndex(); overwriteIndexDimensions(new QueryableIndexStorageAdapter(oldIndex)); - if (customDimensions) { + if (variableDimensions) { for (String dim : oldIndex.getAvailableDimensions()) { dimOrder.add(dim); oldFormat.put(dim, oldIndex.getColumnHolder(dim).getColumnFormat()); @@ -370,12 +370,12 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) } else { IncrementalIndex oldIndex = lastHydrant.getIndex(); overwriteIndexDimensions(new IncrementalIndexStorageAdapter(oldIndex)); - if (customDimensions) { + if (variableDimensions) { dimOrder.addAll(oldIndex.getDimensionOrder()); oldFormat = oldIndex.getColumnFormats(); } } - if (customDimensions) { + if (variableDimensions) { newIndex.loadDimensionIterable(dimOrder, oldFormat); } } diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index b479737c9940..f3264dad44cf 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -28,7 +28,6 @@ import org.apache.druid.client.cache.CachePopulator; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.FunctionalIterable; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -60,6 +59,7 @@ import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.ResourceIdPopulatingQueryRunner; import org.apache.druid.server.SegmentManager; @@ -307,14 +307,14 @@ private QueryRunner buildAndDecorateQueryRunner( queryMetrics -> queryMetrics.segment(segmentIdString) ); - long segmentMaxTime = storageAdapter.getMaxTime().getMillis(); - long segmentMinTime = storageAdapter.getMinTime().getMillis(); - Interval actualDataInterval = Intervals.utc(segmentMinTime, segmentMaxTime + 1); + final TimeBoundaryInspector timeBoundaryInspector = segment.as(TimeBoundaryInspector.class); + final Interval cacheKeyInterval = + timeBoundaryInspector != null ? timeBoundaryInspector.getMinMaxInterval() : segmentInterval; CachingQueryRunner cachingQueryRunner = new CachingQueryRunner<>( segmentIdString, cacheKeyPrefix, segmentDescriptor, - actualDataInterval, + cacheKeyInterval, objectMapper, cache, toolChest, diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 78294fca0c4b..32b2a3830fdf 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -30,8 +30,11 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -242,7 +245,7 @@ public void testOverlapMetricNameAndDim() jsonMapper ); - expectedException.expect(IllegalArgumentException.class); + expectedException.expect(DruidException.class); expectedException.expectMessage( "Cannot specify a column more than once: [metric1] seen in dimensions list, metricsSpec list" ); @@ -251,22 +254,110 @@ public void testOverlapMetricNameAndDim() } @Test - public void testOverlapTimeAndDim() + public void testOverlapTimeAndDimPositionZero() { - expectedException.expect(IllegalArgumentException.class); + DataSchema schema = new DataSchema( + IdUtilsTest.VALID_ID_CHARS, + new TimestampSpec("time", "auto", null), + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new LongDimensionSchema("__time"), + new StringDimensionSchema("dimA"), + new StringDimensionSchema("dimB") + ) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .build(), + null, + new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + null, + null, + jsonMapper + ); + + Assert.assertEquals( + ImmutableList.of("__time", "dimA", "dimB"), + schema.getDimensionsSpec().getDimensionNames() + ); + + Assert.assertTrue(schema.getDimensionsSpec().isForceSegmentSortByTime()); + } + + @Test + public void testOverlapTimeAndDimPositionZeroWrongType() + { + expectedException.expect(DruidException.class); + expectedException.expectMessage("Encountered dimension[__time] with incorrect type[STRING]. Type must be 'long'."); + + DataSchema schema = new DataSchema( + IdUtilsTest.VALID_ID_CHARS, + new TimestampSpec("time", "auto", null), + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("__time"), + new StringDimensionSchema("dimA"), + new StringDimensionSchema("dimB") + ) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .build(), + null, + new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + null, + null, + jsonMapper + ); + } + + @Test + public void testOverlapTimeAndDimPositionOne() + { + expectedException.expect(DruidException.class); expectedException.expectMessage( - "Cannot specify a column more than once: [__time] seen in dimensions list, " - + "primary timestamp (__time cannot appear as a dimension or metric)" + "Encountered dimension[__time] at position[1]. This is only supported when the dimensionsSpec " + + "parameter[forceSegmentSortByTime] is set to[false]. " + + DimensionsSpec.WARNING_NON_TIME_SORT_ORDER + ); + + DataSchema schema = new DataSchema( + IdUtilsTest.VALID_ID_CHARS, + new TimestampSpec("time", "auto", null), + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dimA"), + new LongDimensionSchema("__time"), + new StringDimensionSchema("dimB") + ) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .build(), + null, + new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + null, + null, + jsonMapper ); + } + @Test + public void testOverlapTimeAndDimPositionOne_withExplicitSortOrder() + { DataSchema schema = new DataSchema( IdUtilsTest.VALID_ID_CHARS, new TimestampSpec("time", "auto", null), DimensionsSpec.builder() .setDimensions( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("__time", "dimA", "dimB", "metric1")) + ImmutableList.of( + new StringDimensionSchema("dimA"), + new LongDimensionSchema("__time"), + new StringDimensionSchema("dimB") + ) ) .setDimensionExclusions(ImmutableList.of("dimC")) + .setForceSegmentSortByTime(false) .build(), null, new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), @@ -274,6 +365,13 @@ public void testOverlapTimeAndDim() null, jsonMapper ); + + Assert.assertEquals( + ImmutableList.of("dimA", "__time", "dimB"), + schema.getDimensionsSpec().getDimensionNames() + ); + + Assert.assertFalse(schema.getDimensionsSpec().isForceSegmentSortByTime()); } @Test @@ -313,11 +411,8 @@ public void testOverlapTimeAndDimLegacy() jsonMapper ); - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage( - "Cannot specify a column more than once: [__time] seen in dimensions list, primary timestamp " - + "(__time cannot appear as a dimension or metric)" - ); + expectedException.expect(DruidException.class); + expectedException.expectMessage("Encountered dimension[__time] with incorrect type[STRING]. Type must be 'long'."); schema.getParser(); } @@ -341,7 +436,7 @@ public void testDuplicateAggregators() ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - expectedException.expect(IllegalArgumentException.class); + expectedException.expect(DruidException.class); expectedException.expectMessage( "Cannot specify a column more than once: [metric1] seen in metricsSpec list (2 occurrences); " + "[metric3] seen in metricsSpec list (2 occurrences)" diff --git a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java index 14eb3d44220d..866381dd70cf 100644 --- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -27,9 +27,11 @@ import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.query.OrderBy; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.Metadata; @@ -46,7 +48,6 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NoneShardSpec; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; @@ -174,6 +175,12 @@ public Map getDimensionHandlers() throw new UnsupportedOperationException(); } + @Override + public List getOrdering() + { + return Cursors.ascendingTimeOrder(); + } + @Override public void close() { @@ -264,19 +271,6 @@ public int getDimensionCardinality(String column) return cardinality; } - @Override - public DateTime getMinTime() - { - return interval.getStart(); - } - - - @Override - public DateTime getMaxTime() - { - return interval.getEnd(); - } - @Override public Indexed getAvailableDimensions() { @@ -316,12 +310,6 @@ public int getNumRows() return 0; } - @Override - public DateTime getMaxIngestedEventTime() - { - return null; - } - @Override public Metadata getMetadata() { diff --git a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java index 0ca08523fb2a..f17d51291433 100644 --- a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java +++ b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java @@ -103,6 +103,7 @@ public void testExecuteQuery() QueryRunner runner = Mockito.mock(QueryRunner.class); QueryRunner mergeRunner = Mockito.mock(QueryRunner.class); Query query = Mockito.mock(Query.class); + QueryableIndex index = Mockito.mock(QueryableIndex.class); Sequence expected = Sequences.simple(Collections.singletonList(123)); Mockito.when(query.withOverriddenContext(ArgumentMatchers.any())).thenReturn(query); Mockito.when(injector.getInstance(QueryRunnerFactoryConglomerate.class)).thenReturn(conglomerate); @@ -111,7 +112,8 @@ public void testExecuteQuery() Mockito.when(factory.getToolchest().mergeResults(factory.mergeRunners(DirectQueryProcessingPool.INSTANCE, ImmutableList.of(runner)))).thenReturn(mergeRunner); Mockito.when(factory.getToolchest().mergeResults(factory.mergeRunners(DirectQueryProcessingPool.INSTANCE, ImmutableList.of(runner)), true)).thenReturn(mergeRunner); Mockito.when(mergeRunner.run(ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(expected); - Sequence actual = DumpSegment.executeQuery(injector, null, query); + Mockito.when(index.getOrdering()).thenReturn(Collections.emptyList()); + Sequence actual = DumpSegment.executeQuery(injector, index, query); Assert.assertSame(expected, actual); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index 21ad4c236633..79f9d79e8499 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Numbers; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContext; @@ -112,6 +113,12 @@ public class PlannerContext public static final String CTX_SQL_REVERSE_LOOKUP = "sqlReverseLookup"; public static final boolean DEFAULT_SQL_REVERSE_LOOKUP = true; + /** + * Context key for {@link PlannerContext#isUseGranularity()}. + */ + public static final String CTX_SQL_USE_GRANULARITY = "sqlUseGranularity"; + public static final boolean DEFAULT_SQL_USE_GRANULARITY = true; + // DataContext keys public static final String DATA_CTX_AUTHENTICATION_RESULT = "authenticationResult"; @@ -127,6 +134,7 @@ public class PlannerContext private final boolean useBoundsAndSelectors; private final boolean pullUpLookup; private final boolean reverseLookup; + private final boolean useGranularity; private final CopyOnWriteArrayList nativeQueryIds = new CopyOnWriteArrayList<>(); private final PlannerHook hook; // bindings for dynamic parameters to bind during planning @@ -156,6 +164,7 @@ private PlannerContext( final boolean useBoundsAndSelectors, final boolean pullUpLookup, final boolean reverseLookup, + final boolean useGranularity, final SqlEngine engine, final Map queryContext, final PlannerHook hook @@ -172,6 +181,7 @@ private PlannerContext( this.useBoundsAndSelectors = useBoundsAndSelectors; this.pullUpLookup = pullUpLookup; this.reverseLookup = reverseLookup; + this.useGranularity = useGranularity; this.hook = hook == null ? NoOpPlannerHook.INSTANCE : hook; String sqlQueryId = (String) this.queryContext.get(QueryContexts.CTX_SQL_QUERY_ID); @@ -196,6 +206,7 @@ public static PlannerContext create( final boolean useBoundsAndSelectors; final boolean pullUpLookup; final boolean reverseLookup; + final boolean useGranularity; final Object stringifyParam = queryContext.get(QueryContexts.CTX_SQL_STRINGIFY_ARRAYS); final Object tsParam = queryContext.get(CTX_SQL_CURRENT_TIMESTAMP); @@ -203,6 +214,7 @@ public static PlannerContext create( final Object useBoundsAndSelectorsParam = queryContext.get(CTX_SQL_USE_BOUNDS_AND_SELECTORS); final Object pullUpLookupParam = queryContext.get(CTX_SQL_PULL_UP_LOOKUP); final Object reverseLookupParam = queryContext.get(CTX_SQL_REVERSE_LOOKUP); + final Object useGranularityParam = queryContext.get(CTX_SQL_USE_GRANULARITY); if (tsParam != null) { utcNow = new DateTime(tsParam, DateTimeZone.UTC); @@ -240,6 +252,12 @@ public static PlannerContext create( reverseLookup = DEFAULT_SQL_REVERSE_LOOKUP; } + if (useGranularityParam != null) { + useGranularity = Numbers.parseBoolean(useGranularityParam); + } else { + useGranularity = DEFAULT_SQL_USE_GRANULARITY; + } + return new PlannerContext( plannerToolbox, sql, @@ -249,6 +267,7 @@ public static PlannerContext create( useBoundsAndSelectors, pullUpLookup, reverseLookup, + useGranularity, engine, queryContext, hook @@ -425,6 +444,16 @@ public boolean isReverseLookup() return reverseLookup; } + /** + * Whether we should use granularities other than {@link Granularities#ALL} when planning queries. This is provided + * mainly so it can be set to false when running SQL queries on non-time-ordered tables, which do not support + * any other granularities. + */ + public boolean isUseGranularity() + { + return useGranularity; + } + public List getParameters() { return parameters; 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 dc127fa7f861..51cf9e923edb 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 @@ -1144,6 +1144,11 @@ private TimeseriesQuery toTimeseriesQuery() // Timeseries only applies if the single dimension is granular __time. return null; } + + if (!plannerContext.isUseGranularity() && !Granularities.ALL.equals(queryGranularity)) { + return null; + } + theContext.put( TimeseriesQuery.CTX_TIMESTAMP_RESULT_FIELD, Iterables.getOnlyElement(grouping.getDimensions()).toDimensionSpec().getOutputName() @@ -1406,7 +1411,7 @@ private GroupByQuery toGroupByQuery() // the query generated by "explain plan for select ..." doesn't match to the native query ACTUALLY being executed, // the granularity and dimensions are slightly different. // now, part of the query plan logic is handled in GroupByStrategyV2. - if (!grouping.getDimensions().isEmpty()) { + if (!grouping.getDimensions().isEmpty() && plannerContext.isUseGranularity()) { for (DimensionExpression dimensionExpression : grouping.getDimensions()) { Granularity granularity = Expressions.toQueryGranularity( dimensionExpression.getDruidExpression(), 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 546776575f28..04113067cb1f 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 @@ -3989,10 +3989,8 @@ public void testSemiJoinWithOuterTimeExtractScan() @ParameterizedTest(name = "{0}") public void testTwoSemiJoinsSimultaneously(Map queryContext) { - // Fully removing the join allows this query to vectorize. - if (!isRewriteJoinToFilter(queryContext)) { - cannotVectorize(); - } + // Cannot vectorize timeBoundary with maxTime (the engine will request descending order, which cannot vectorize). + cannotVectorize(); Map updatedQueryContext = new HashMap<>(queryContext); updatedQueryContext.put(QueryContexts.TIME_BOUNDARY_PLANNING_KEY, true); 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 556b437e7581..58d29a738c09 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 @@ -9726,6 +9726,52 @@ public void testTimeseries() ); } + @Test + public void testTimeseriesDontUseGranularity() + { + // When sqlUseGranularity: false, this query plans as a groupBy rather than a timeseries. + final Map context = QueryContexts.override( + QUERY_CONTEXT_DEFAULT, + PlannerContext.CTX_SQL_USE_GRANULARITY, + false + ); + + testQuery( + "SELECT SUM(cnt), gran FROM (\n" + + " SELECT floor(__time TO month) AS gran,\n" + + " cnt FROM druid.foo\n" + + ") AS x\n" + + "GROUP BY gran\n" + + "ORDER BY gran", + context, + ImmutableList.of( + GroupByQuery + .builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + expressionVirtualColumn("v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", ColumnType.LONG)) + .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setLimitSpec( + queryFramework().engine().featureAvailable(EngineFeature.GROUPBY_IMPLICITLY_SORTS) + ? NoopLimitSpec.instance() + : new DefaultLimitSpec( + ImmutableList.of(new OrderByColumnSpec("d0", Direction.ASCENDING, StringComparators.NUMERIC)), + Integer.MAX_VALUE + ) + ) + .setContext(context) + .build() + ), + ImmutableList.of( + new Object[]{3L, timestamp("2000-01-01")}, + new Object[]{3L, timestamp("2001-01-01")} + ) + ); + } + @Test public void testFilteredTimeAggregators() { @@ -11180,6 +11226,73 @@ public void testGroupByTimeAndOtherDimension() ); } + @Test + public void testGroupByTimeAndOtherDimensionDontUseGranularity() + { + // When sqlUseGranularity: false, this query doesn't use a timestamp result context. + final Map context = QueryContexts.override( + QUERY_CONTEXT_DEFAULT, + PlannerContext.CTX_SQL_USE_GRANULARITY, + false + ); + + testQuery( + "SELECT dim2, gran, SUM(cnt)\n" + + "FROM (SELECT FLOOR(__time TO MONTH) AS gran, dim2, cnt FROM druid.foo) AS x\n" + + "GROUP BY dim2, gran\n" + + "ORDER BY dim2, gran", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + expressionVirtualColumn( + "v0", + "timestamp_floor(\"__time\",'P1M',null,'UTC')", + ColumnType.LONG + ) + ) + .setDimensions( + dimensions( + new DefaultDimensionSpec("dim2", "d0"), + new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setLimitSpec( + queryFramework().engine().featureAvailable(EngineFeature.GROUPBY_IMPLICITLY_SORTS) + ? NoopLimitSpec.instance() + : new DefaultLimitSpec( + ImmutableList.of( + new OrderByColumnSpec("d0", Direction.ASCENDING, StringComparators.LEXICOGRAPHIC), + new OrderByColumnSpec("d1", Direction.ASCENDING, StringComparators.NUMERIC) + ), + Integer.MAX_VALUE + ) + ) + .setContext(context) + .build() + ), + NullHandling.replaceWithDefault() ? + ImmutableList.of( + new Object[]{"", timestamp("2000-01-01"), 2L}, + new Object[]{"", timestamp("2001-01-01"), 1L}, + new Object[]{"a", timestamp("2000-01-01"), 1L}, + new Object[]{"a", timestamp("2001-01-01"), 1L}, + new Object[]{"abc", timestamp("2001-01-01"), 1L} + ) : + ImmutableList.of( + new Object[]{null, timestamp("2000-01-01"), 1L}, + new Object[]{null, timestamp("2001-01-01"), 1L}, + new Object[]{"", timestamp("2000-01-01"), 1L}, + new Object[]{"a", timestamp("2000-01-01"), 1L}, + new Object[]{"a", timestamp("2001-01-01"), 1L}, + new Object[]{"abc", timestamp("2001-01-01"), 1L} + ) + ); + } + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.IMPROVED_PLAN, separateDefaultModeTest = true) @Test public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteTimeBoundaryQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteTimeBoundaryQueryTest.java index a6be8f7cf99a..8fde5d166b69 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteTimeBoundaryQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteTimeBoundaryQueryTest.java @@ -197,6 +197,9 @@ public void testMinMaxTimeQuery() @Test public void testMaxTimeQueryWithJoin() { + // Cannot vectorize timeBoundary with maxTime (the engine will request descending order, which cannot vectorize). + cannotVectorize(); + HashMap context = new HashMap<>(QUERY_CONTEXT_DEFAULT); context.put(QueryContexts.TIME_BOUNDARY_PLANNING_KEY, true);