From 0863b3e3df2c37a74eb5baf767424fcc22500e66 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 6 Aug 2024 14:17:17 -0700 Subject: [PATCH 01/25] Segments primarily sorted by non-time columns. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Currently, segments are always sorted by __time, followed by the sort order provided by the user via dimensionsSpec or CLUSTERED BY. Sorting by __time enables efficient execution of queries involving time-ordering or granularity. Time-ordering is a simple matter of reading the rows in stored order, and granular cursors can be generated in streaming fashion. However, for various workloads, it's better for storage footprint and query performance to sort by arbitrary orders that do not start with __time. With this patch, users can sort segments by such orders. For spec-based ingestion, users add "useExplicitSegmentSortOrder: true" to dimensionsSpec. The "dimensions" list determines the sort order. To define a sort order that includes "__time", users explicitly include a dimension named "__time". For SQL-based ingestion, users set the context parameter "useExplicitSegmentSortOrder: true". The CLUSTERED BY clause is then used as the explicit segment sort order. In both cases, when the new "useExplicitSegmentSortOrder" parameter is false (the default), __time is implicitly prepended to the sort order, as it always was prior to this patch. The new parameter is experimental for two main reasons. First, such segments can cause errors when loaded by older servers, due to violating their expectations that timestamps are always monotonically increasing. Second, even on newer servers, not all queries can run on non-time-sorted segments. Scan queries involving time-ordering and any query involving granularity will not run. (To partially mitigate this, a currently-undocumented SQL feature "sqlUseGranularity" is provided. When set to false the SQL planner avoids using "granularity".) Changes on the write path: 1) DimensionsSpec can now optionally contain a __time dimension, which controls the placement of __time in the sort order. If not present, __time is considered to be first in the sort order, as it has always been. 2) IncrementalIndex and IndexMerger are updated to sort facts more flexibly; not always by time first. 3) Metadata (stored in metadata.drd) gains a "sortOrder" field. 4) MSQ can generate range-based shard specs even when not all columns are singly-valued strings. It merely stops accepting new clustering key fields when it encounters the first one that isn't a singly-valued string. This is useful because it enables range shard specs on "someDim" to be created for clauses like "CLUSTERED BY someDim, __time". Changes on the read path: 1) Add StorageAdapter#getSortOrder so query engines can tell how a segment is sorted. 2) Update QueryableIndexStorageAdapter, IncrementalIndexStorageAdapter, and VectorCursorGranularizer to throw errors when using granularities on non-time-ordered segments. 3) Update ScanQueryEngine to throw an error when using the time-ordering "order" parameter on non-time-ordered segments. 4) Update TimeBoundaryQueryRunnerFactory to perform a segment scan when running on a non-time-ordered segment. 5) Add "sqlUseGranularity" context parameter that causes the SQL planner to avoid using granularities other than ALL. Other changes: 1) Rename DimensionsSpec "hasCustomDimensions" to "hasFixedDimensions" and change the meaning subtly: it now returns true if the DimensionsSpec represents an unchanging list of dimensions, or false if there is some discovery happening. This is what call sites had expected anyway. --- docs/ingestion/ingestion-spec.md | 2 +- docs/ingestion/partitioning.md | 59 ++- docs/multi-stage-query/concepts.md | 10 +- docs/multi-stage-query/reference.md | 9 +- docs/querying/sql-query-context.md | 10 +- .../ApproximateHistogramGroupByQueryTest.java | 2 +- ...FixedBucketsHistogramGroupByQueryTest.java | 2 +- .../apache/druid/msq/exec/ControllerImpl.java | 163 ++++--- .../druid/msq/sql/MSQTaskQueryMaker.java | 11 +- .../druid/msq/sql/MSQTaskSqlEngine.java | 111 ++++- .../msq/util/MSQTaskQueryMakerUtils.java | 27 +- .../msq/util/MultiStageQueryContext.java | 9 + .../apache/druid/msq/exec/MSQReplaceTest.java | 422 +++++++++++++++++- .../msq/exec/TaskDataSegmentProviderTest.java | 6 + .../msq/util/MSQTaskQueryMakerUtilsTest.java | 17 +- .../druid/indexer/IndexGeneratorJob.java | 13 +- .../indexer/path/DatasourcePathSpec.java | 2 +- .../common/task/AbstractBatchIndexTask.java | 16 +- .../indexing/common/task/CompactionTask.java | 73 +-- .../druid/indexing/input/InputRowSchemas.java | 2 +- .../overlord/sampler/InputSourceSampler.java | 4 +- .../common/task/CompactionTaskRunTest.java | 154 +++++++ .../common/task/CompactionTaskTest.java | 1 + .../druid/data/input/impl/DimensionsSpec.java | 70 ++- .../frame/segment/FrameStorageAdapter.java | 10 + .../segment/columnar/FrameCursorFactory.java | 5 +- .../segment/columnar/FrameQueryableIndex.java | 6 + .../common/granularity/Granularities.java | 17 + .../apache/druid/query/FluentQueryRunner.java | 27 -- .../druid/query/scan/ScanQueryEngine.java | 7 + .../query/search/UseIndexesStrategy.java | 2 +- .../TimeBoundaryQueryRunnerFactory.java | 99 ++-- .../vector/VectorCursorGranularizer.java | 3 + .../druid/segment/FilteredStorageAdapter.java | 7 + .../org/apache/druid/segment/IndexIO.java | 8 +- .../org/apache/druid/segment/IndexMerger.java | 60 ++- .../apache/druid/segment/IndexMergerV9.java | 56 ++- .../druid/segment/IndexableAdapter.java | 8 +- .../org/apache/druid/segment/Metadata.java | 66 ++- .../apache/druid/segment/QueryableIndex.java | 13 + .../QueryableIndexColumnSelectorFactory.java | 5 +- .../QueryableIndexCursorSequenceBuilder.java | 107 +++-- .../QueryableIndexIndexableAdapter.java | 49 +- .../segment/QueryableIndexStorageAdapter.java | 26 +- .../druid/segment/RowBasedStorageAdapter.java | 12 +- .../RowCombiningTimeAndDimsIterator.java | 1 + .../org/apache/druid/segment/RowPointer.java | 4 +- .../druid/segment/SimpleQueryableIndex.java | 12 + .../apache/druid/segment/StorageAdapter.java | 44 +- .../druid/segment/TimeAndDimsPointer.java | 59 ++- .../druid/segment/UnnestStorageAdapter.java | 17 + .../segment/incremental/IncrementalIndex.java | 149 +++++-- .../incremental/IncrementalIndexAdapter.java | 4 +- ...IncrementalIndexColumnSelectorFactory.java | 2 +- .../incremental/IncrementalIndexRow.java | 2 +- .../IncrementalIndexRowIterator.java | 1 + .../IncrementalIndexStorageAdapter.java | 17 +- .../incremental/OnheapIncrementalIndex.java | 126 +++++- .../join/HashJoinSegmentStorageAdapter.java | 16 + .../table/BroadcastSegmentIndexedTable.java | 1 + .../loading/TombstoneSegmentizerFactory.java | 7 +- .../segment/virtual/ExpressionSelectors.java | 6 +- .../druid/query/QueryRunnerTestHelper.java | 175 +++++--- .../apache/druid/query/TestQueryRunner.java | 70 +++ .../groupby/GroupByQueryMergeBufferTest.java | 2 +- .../GroupByQueryRunnerFailureTest.java | 2 +- .../query/groupby/GroupByQueryRunnerTest.java | 350 ++++++++++++++- .../GroupByTimeseriesQueryRunnerTest.java | 2 +- .../SegmentMetadataUnionQueryTest.java | 8 +- .../druid/query/scan/ScanQueryRunnerTest.java | 4 +- .../query/search/SearchQueryRunnerTest.java | 3 +- .../TimeBoundaryQueryRunnerTest.java | 8 +- .../TimeSeriesUnionQueryRunnerTest.java | 3 +- .../timeseries/TimeseriesQueryRunnerTest.java | 3 +- .../topn/TopNMetricSpecOptimizationsTest.java | 7 + .../druid/query/topn/TopNQueryRunnerTest.java | 21 +- .../IndexMergerLongestSharedDimOrderTest.java | 6 +- .../druid/segment/IndexMergerTestBase.java | 358 ++++++++++++++- .../druid/segment/MergingRowIteratorTest.java | 2 + .../apache/druid/segment/MetadataTest.java | 20 +- .../org/apache/druid/segment/TestIndex.java | 74 ++- .../TombstoneSegmentStorageAdapterTest.java | 7 + .../segment/data/IncrementalIndexTest.java | 9 +- .../IncrementalIndexRowCompTest.java | 92 +++- .../IncrementalIndexStorageAdapterTest.java | 86 +++- .../DimensionRangeShardSpecTest.java | 23 + .../druid/segment/indexing/DataSchema.java | 48 +- .../druid/segment/realtime/sink/Sink.java | 8 +- .../segment/indexing/DataSchemaTest.java | 119 ++++- .../apache/druid/server/TestSegmentUtils.java | 12 + .../sql/calcite/planner/PlannerContext.java | 29 ++ .../druid/sql/calcite/rel/DruidQuery.java | 7 +- .../druid/sql/calcite/CalciteQueryTest.java | 113 +++++ 93 files changed, 3322 insertions(+), 605 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/query/TestQueryRunner.java diff --git a/docs/ingestion/ingestion-spec.md b/docs/ingestion/ingestion-spec.md index a1a0ff1ee5b7..0db787bc9dd2 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 | - +| `useExplicitSegmentSortOrder` | When set to true, segments created by the ingestion job are sorted by `{dimensions[0], dimensions[1], ...}`, rather than `{__time, dimensions[0], dimensions[1], ...}`. This enables creation of segments that are sorted by something other than `__time`. To include `__time` in the sort order when this parameter is set, you must include a dimension named `__time` with type `long` explicitly in the `dimensions` list.

This is an experimental feature; see [Sorting](../partitioning#sorting) for details. | `false` | #### Dimension objects diff --git a/docs/ingestion/partitioning.md b/docs/ingestion/partitioning.md index 6cf5b0a74d28..20f8020e18a5 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 recluster 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 `useExplicitSegmentSortOrder` to `true` 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 `useExplicitSegmentSortOrder` 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: + +- 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 cf06156c6587..cea08971abab 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -362,8 +362,12 @@ 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/concepts.md#secondary-partitioning) of segments +within a time chunk, and [sorting](../ingestion/concepts#sorting) of rows within a segment. For sorting purposes, Druid +implicitly prepends `__time` to the `CLUSTERED BY` column list, unless [`useExplicitSegmentSortOrder`](#context) +(an experimental feature; see [Sorting](../ingestion/partitioning#sorting) for details) is set to `true`. For more information about clustering, see [Clustering](concepts.md#clustering). @@ -398,6 +402,7 @@ The following table lists the context parameters for the MSQ task engine: | `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 | +| `useExplicitSegmentSortOrder` | INSERT or REPLACE

When set to true, `CLUSTERED BY` and `segmentSortOrder` are permitted to include `__time` in some position other than the first. Additionally, `CLUSTERED BY x` will sort segments by `x`, rather than `{__time, x}`. This enables creation of segments that are sorted by something other than `__time`.

This is an experimental feature; see [Sorting](../ingestion/partitioning#sorting) for details. | `false` | | `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-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..c0d0c9b6d8ab 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, 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/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/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 839839db4e42..131d93b2c644 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 useExplicitSegmentSortOrder = + MultiStageQueryContext.isUseExplicitSegmentSortOrder(querySpec.getQuery().context()); final Pair, List> dimensionsAndAggregators = makeDimensionsAndAggregatorsForIngestion( querySignature, queryClusterBy, destination.getSegmentSortOrder(), + useExplicitSegmentSortOrder, columnMappings, isRollupQuery, querySpec.getQuery() @@ -1956,7 +1960,10 @@ private static DataSchema makeDataSchemaForIngestion( return new DataSchema( destination.getDataSource(), new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null), - new DimensionsSpec(dimensionsAndAggregators.lhs), + DimensionsSpec.builder() + .setDimensions(dimensionsAndAggregators.lhs) + .setUseExplicitSegmentSortOrder(useExplicitSegmentSortOrder) + .build(), dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]), makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper), new TransformSpec(null, Collections.emptyList()) @@ -2037,6 +2044,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, @@ -2047,7 +2056,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<>(); @@ -2055,7 +2067,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++) { @@ -2064,25 +2076,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."); } /** @@ -2101,22 +2135,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); @@ -2125,7 +2152,8 @@ private static StringTuple makeStringTuple( private static Pair, List> makeDimensionsAndAggregatorsForIngestion( final RowSignature querySignature, final ClusterBy queryClusterBy, - final List segmentSortOrder, + final List contextSegmentSortOrder, + final boolean useExplicitSegmentSortOrder, final ColumnMappings columnMappings, final boolean isRollupQuery, final Query query @@ -2154,7 +2182,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 useExplicitSegmentSortOrder is not set. + if (columnMappings.hasOutputColumn(ColumnHolder.TIME_COLUMN_NAME) && !useExplicitSegmentSortOrder) { + 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. @@ -2189,7 +2222,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) { @@ -2204,10 +2237,30 @@ 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() + ); + } else { + // complex columns only + if (DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.containsKey(type.getComplexTypeName())) { + dimensions.add( + DimensionSchemaUtils.createDimensionSchema( + outputColumnName, + type, + MultiStageQueryContext.useAutoColumnSchemas(query.context()), + MultiStageQueryContext.getArrayIngestMode(query.context()) + ) + ); + } else if (!isRollupQuery) { + aggregators.add(new PassthroughAggregatorFactory(outputColumnName, type.getComplexTypeName())); + } else { populateDimensionsAndAggregators( dimensions, aggregators, @@ -2216,37 +2269,18 @@ private static Pair, List> makeDimensio type, query.context() ); - } else { - // complex columns only - if (DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.containsKey(type.getComplexTypeName())) { - dimensions.add( - DimensionSchemaUtils.createDimensionSchema( - outputColumnName, - type, - MultiStageQueryContext.useAutoColumnSchemas(query.context()), - MultiStageQueryContext.getArrayIngestMode(query.context()) - ) - ); - } else if (!isRollupQuery) { - aggregators.add(new PassthroughAggregatorFactory(outputColumnName, type.getComplexTypeName())); - } else { - populateDimensionsAndAggregators( - dimensions, - aggregators, - outputColumnAggregatorFactories, - outputColumnName, - type, - query.context() - ); - } } } } + 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); + } + return Pair.of(dimensions, aggregators); } - /** * If the output column is present in the outputColumnAggregatorFactories that means we already have the aggregator information for this column. * else treat this column as a dimension. @@ -2266,7 +2300,12 @@ private static void populateDimensionsAndAggregators( QueryContext context ) { - 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/sql/MSQTaskQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java index c6396c0b3060..c3b2a26e852e 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,20 +237,19 @@ 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 ); - 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..627765056a27 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_EXPLICIT_SORT_ORDER} + * 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.isUseExplicitSegmentSortOrder(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[true]. %s", + MultiStageQueryContext.CTX_SORT_ORDER, + ColumnHolder.TIME_COLUMN_NAME, + MultiStageQueryContext.CTX_EXPLICIT_SORT_ORDER, + 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[true]. %s", + ColumnHolder.TIME_COLUMN_NAME, + timePosition, + MultiStageQueryContext.CTX_EXPLICIT_SORT_ORDER, + 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..34e9cac42e11 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_EXPLICIT_SORT_ORDER = DimensionsSpec.PARAMETER_EXPLICIT_SORT_ORDER; + private static final boolean DEFAULT_EXPLICIT_SORT_ORDER = false; + 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 isUseExplicitSegmentSortOrder(final QueryContext queryContext) + { + return queryContext.getBoolean(CTX_EXPLICIT_SORT_ORDER, DEFAULT_EXPLICIT_SORT_ORDER); + } + 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..8af3089ec5da 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) @@ -194,6 +196,393 @@ 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 useExplicitSegmentSortOrder (false). 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( + EasyMock.eq("foo"), + EasyMock.eq(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, "", 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_EXPLICIT_SORT_ORDER, true); + + Mockito.doCallRealMethod() + .doReturn(ImmutableSet.of(existingDataSegment0, existingDataSegment1)) + .when(testTaskActionClient) + .submit(new RetrieveUsedSegmentsAction( + EasyMock.eq("foo"), + EasyMock.eq(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, "", 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") + ) + ) + .setUseExplicitSegmentSortOrder(true) + .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 useExplicitSegmentSortOrder = true. + // (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_EXPLICIT_SORT_ORDER, true); + + Mockito.doCallRealMethod() + .doReturn(ImmutableSet.of(existingDataSegment0, existingDataSegment1)) + .when(testTaskActionClient) + .submit(new RetrieveUsedSegmentsAction( + EasyMock.eq("foo"), + EasyMock.eq(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, "", 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") + ) + ) + .setUseExplicitSegmentSortOrder(true) + .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 useExplicitSegmentSortOrder = false (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[useExplicitSegmentSortOrder] is set to[true]." + )) + .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 + // useExplicitSegmentSortOrder = 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[useExplicitSegmentSortOrder] is set to[true]." + )) + .verifyPlanningErrors(); + } + + @MethodSource("data") + @ParameterizedTest(name = "{index}:with context {0}") + public void testReplaceOnFooWithAllClusteredByTimeThenDimExplicitSort(String contextName, Map context) + { + // Tests [CLUSTERED BY dim1] with useExplicitSegmentSortOrder = true. + 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_EXPLICIT_SORT_ORDER, true); + + Mockito.doCallRealMethod() + .doReturn(ImmutableSet.of(existingDataSegment0, existingDataSegment1)) + .when(testTaskActionClient) + .submit(new RetrieveUsedSegmentsAction( + EasyMock.eq("foo"), + EasyMock.eq(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, "", 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 + // useExplicitSegmentSortOrder = false. + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dim1"), + new FloatDimensionSchema("m1") + ) + ) + .build(), + GranularityType.ALL, + Intervals.ETERNITY + ) + ) + .verifyResults(); + } + @MethodSource("data") @ParameterizedTest(name = "{index}:with context {0}") public void testReplaceOnFooWithWhere(String contextName, Map context) @@ -281,7 +670,8 @@ public void testReplaceOnFoo1WithAllExtern(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 +1509,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 +1728,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 +2596,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..dba18730b982 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 @@ -334,6 +334,12 @@ public ColumnHolder getColumnHolder(String columnName) return null; } + @Override + public List getSortOrder() + { + 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..37a20cea9897 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,7 @@ package org.apache.druid.msq.util; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import org.junit.Assert; import org.junit.Test; @@ -28,25 +29,25 @@ 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")); // These are not OK. Assert.assertThrows( IllegalArgumentException.class, - () -> MSQTaskQueryMakerUtils.validateSegmentSortOrder(ImmutableList.of("c"), ImmutableList.of("a", "b")) + () -> MSQTaskQueryMakerUtils.validateContextSortOrderColumnsExist(ImmutableList.of("c"), ImmutableSet.of("a", "b")) ); Assert.assertThrows( IllegalArgumentException.class, - () -> MSQTaskQueryMakerUtils.validateSegmentSortOrder( + () -> MSQTaskQueryMakerUtils.validateContextSortOrderColumnsExist( ImmutableList.of("b", "__time"), - ImmutableList.of("__time", "a", "b") + ImmutableSet.of("__time", "a", "b") ) ); } 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..43ca9e2511ed 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,18 @@ 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 dimensionExclusions and spatialDimensions since auto compaction does not support them + final DimensionsSpec inputDimensionsSpec = ingestionSpec.getDataSchema().getDimensionsSpec(); + dimensionsSpec = + DimensionsSpec.builder() + .setDimensions(inputDimensionsSpec.getDimensions()) + .setUseExplicitSegmentSortOrder(inputDimensionsSpec.isUseExplicitSegmentSortOrder()) + .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..abfe71ac3e65 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 @@ -71,9 +71,9 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; 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.Metadata; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.AppendableIndexSpec; @@ -109,6 +109,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 +754,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 +832,32 @@ 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); + return DimensionsSpec.builder() + .setDimensions(dimensionSchemas) + .setUseExplicitSegmentSortOrder(includeTimeAsDimension) + .build(); } public AggregatorFactory[] getMetricsSpec() @@ -923,27 +937,26 @@ private void processDimensionsSpec(final QueryableIndex index) return; } - final Map dimensionHandlerMap = index.getDimensionHandlers(); - - for (String dimension : index.getAvailableDimensions()) { - final ColumnHolder columnHolder = Preconditions.checkNotNull( - index.getColumnHolder(dimension), - "Cannot find column for dimension[%s]", - 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) - ); + final Metadata metadata = index.getMetadata(); + final List sortOrder = + metadata != null && metadata.getSortOrder() != null + ? metadata.getSortOrder() + : Collections.emptyList(); + + 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 7ebde700bee2..6bc8526d842f 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; @@ -1880,6 +1882,158 @@ 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) + )) + .setUseExplicitSegmentSortOrder(true) + .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) + )) + .setUseExplicitSegmentSortOrder(true) + .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 WindowedStorageAdapter adapter = new WindowedStorageAdapter( + new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), + compactSegment.getInterval() + ); + Assert.assertEquals(ImmutableList.of("x", "__time", "ts", "dim", "y"), adapter.getAdapter().getSortOrder()); + final Sequence cursorSequence = adapter.getAdapter().makeCursors( + null, + compactSegment.getInterval(), + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ); + + cursorSequence.accumulate(rowsFromSegment, (accumulated, cursor) -> { + 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(); + } + + return accumulated; + }); + + 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..8172352269c4 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,32 @@ @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_EXPLICIT_SORT_ORDER = "useExplicitSegmentSortOrder"; + + /** + * Warning about non-time ordering to include in error messages when {@link #PARAMETER_EXPLICIT_SORT_ORDER} 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 + ); + private final List dimensions; private final Set dimensionExclusions; private final Map dimensionSchemaMap; private final boolean includeAllDimensions; + private final boolean useExplicitSegmentSortOrder; 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 +101,7 @@ public static Builder builder() public DimensionsSpec(List dimensions) { - this(dimensions, null, null, false, null); + this(dimensions, null, null, false, null, false); } @JsonCreator @@ -89,7 +110,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_EXPLICIT_SORT_ORDER) boolean useExplicitSegmentSortOrder ) { this.dimensions = dimensions == null @@ -120,6 +142,7 @@ private DimensionsSpec( this.includeAllDimensions = includeAllDimensions; this.useSchemaDiscovery = useSchemaDiscovery != null && useSchemaDiscovery; + this.useExplicitSegmentSortOrder = useExplicitSegmentSortOrder; } @JsonProperty @@ -146,6 +169,13 @@ public boolean useSchemaDiscovery() return useSchemaDiscovery; } + @JsonProperty(PARAMETER_EXPLICIT_SORT_ORDER) + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public boolean isUseExplicitSegmentSortOrder() + { + return useExplicitSegmentSortOrder; + } + @Deprecated @JsonIgnore public List getSpatialDimensions() @@ -191,9 +221,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 +238,8 @@ public DimensionsSpec withDimensions(List dims) ImmutableList.copyOf(dimensionExclusions), null, includeAllDimensions, - useSchemaDiscovery + useSchemaDiscovery, + useExplicitSegmentSortOrder ); } @@ -215,7 +250,8 @@ public DimensionsSpec withDimensionExclusions(Set dimExs) ImmutableList.copyOf(Sets.union(dimensionExclusions, dimExs)), null, includeAllDimensions, - useSchemaDiscovery + useSchemaDiscovery, + useExplicitSegmentSortOrder ); } @@ -227,7 +263,8 @@ public DimensionsSpec withSpatialDimensions(List spatial ImmutableList.copyOf(dimensionExclusions), spatials, includeAllDimensions, - useSchemaDiscovery + useSchemaDiscovery, + useExplicitSegmentSortOrder ); } @@ -267,9 +304,11 @@ public boolean equals(Object o) } DimensionsSpec that = (DimensionsSpec) o; return includeAllDimensions == that.includeAllDimensions + && useExplicitSegmentSortOrder == that.useExplicitSegmentSortOrder && useSchemaDiscovery == that.useSchemaDiscovery && Objects.equals(dimensions, that.dimensions) - && Objects.equals(dimensionExclusions, that.dimensionExclusions); + && Objects.equals(dimensionExclusions, that.dimensionExclusions) + && Objects.equals(dimensionSchemaMap, that.dimensionSchemaMap); } @Override @@ -278,7 +317,9 @@ public int hashCode() return Objects.hash( dimensions, dimensionExclusions, + dimensionSchemaMap, includeAllDimensions, + useExplicitSegmentSortOrder, useSchemaDiscovery ); } @@ -291,6 +332,7 @@ public String toString() ", dimensionExclusions=" + dimensionExclusions + ", includeAllDimensions=" + includeAllDimensions + ", useSchemaDiscovery=" + useSchemaDiscovery + + (useExplicitSegmentSortOrder ? ", useExplicitSegmentSortOrder=" + useExplicitSegmentSortOrder : "") + '}'; } @@ -301,6 +343,7 @@ public static final class Builder private List spatialDimensions; private boolean includeAllDimensions; private boolean useSchemaDiscovery; + private boolean useExplicitSegmentSortOrder; public Builder setDimensions(List dimensions) { @@ -339,6 +382,12 @@ public Builder useSchemaDiscovery(boolean useSchemaDiscovery) return this; } + public Builder setUseExplicitSegmentSortOrder(boolean useExplicitSegmentSortOrder) + { + this.useExplicitSegmentSortOrder = useExplicitSegmentSortOrder; + return this; + } + public DimensionsSpec build() { return new DimensionsSpec( @@ -346,7 +395,8 @@ public DimensionsSpec build() dimensionExclusions, spatialDimensions, includeAllDimensions, - useSchemaDiscovery + useSchemaDiscovery, + useExplicitSegmentSortOrder ); } } 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 9733c548d422..5632c6d3ad24 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 @@ -41,6 +41,7 @@ import javax.annotation.Nullable; import java.util.Collections; +import java.util.List; /** * A {@link StorageAdapter} implementation based on a single {@link Frame}. @@ -146,6 +147,15 @@ public Metadata getMetadata() return null; } + @Override + public List getSortOrder() + { + // 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. + return Collections.emptyList(); + } + @Override public boolean canVectorize(@Nullable Filter filter, VirtualColumns virtualColumns, boolean descending) { diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java index 3497ed39898a..737a16fc2cba 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java @@ -181,8 +181,11 @@ private static Cursor makeGranularityAllCursor( descending ? new SimpleDescendingOffset(numRows) : new SimpleAscendingOffset(numRows); final SimpleSettableOffset offset; + // Set timeOrdered = false, because 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. final QueryableIndexColumnSelectorFactory columnSelectorFactory = - new QueryableIndexColumnSelectorFactory(virtualColumns, descending, baseOffset, columnSelector); + new QueryableIndexColumnSelectorFactory(virtualColumns, descending, false, baseOffset, columnSelector); if (filterToUse == null) { offset = baseOffset; 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 f42a33ce6f78..6adb633be029 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 @@ -100,6 +100,12 @@ public Indexed getAvailableDimensions() return new ListIndexed<>(signature.getColumnNames()); } + @Override + public List getSortOrder() + { + return Collections.emptyList(); + } + @Override public BitmapFactory getBitmapFactoryForDimensions() { 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 303f835d889e..4c7ac7928d60 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 @@ -19,6 +19,9 @@ package org.apache.druid.java.util.common.granularity; +import org.apache.druid.error.DruidException; +import org.apache.druid.segment.StorageAdapter; + /** * This class was created b/c sometimes static initializers of a class that use a subclass can deadlock. * See: #2979, #3979 @@ -46,4 +49,18 @@ public static Granularity nullToAll(Granularity granularity) { return granularity == null ? Granularities.ALL : granularity; } + + /** + * Validate that the provided granularity can be used with the given storage adapter. Specifically, this method + * requires that if {@link StorageAdapter#isTimeOrdered()} is false, the granularity must be {@link #ALL}. + */ + public static void validateGranularity(final StorageAdapter storageAdapter, final Granularity granularity) + { + if (!ALL.equals(granularity) && !storageAdapter.isTimeOrdered()) { + throw DruidException + .forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build("Cannot use granularity[%s] on non-time-sorted data.", granularity); + } + } } 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/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 567e07fbdc73..d62c3bd1a147 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; @@ -114,6 +115,12 @@ public Sequence process( final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); + if (!query.getTimeOrder().equals(ScanQuery.Order.NONE) && !adapter.isTimeOrdered()) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build("Cannot use order[%s] on non-time-ordered segment.", query.getTimeOrder()); + } + // If the row count is not set, set it to 0, else do nothing. responseContext.addRowScanCount(0); final long limit = calculateRemainingScanRowsLimit(query, responseContext); 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 8fc431a756b5..85f5c954f94d 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 @@ -98,7 +98,7 @@ 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) && adapter.isTimeOrdered()) { final ImmutableBitmap timeFilteredBitmap = makeTimeFilteredBitmap( index, segment, 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 7a695987fb76..1ea669cdae9c 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,9 @@ 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.error.DruidException; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -50,6 +51,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Iterator; import java.util.List; @@ -92,29 +94,18 @@ public QueryToolChest, TimeBoundaryQuery> getToo private static class TimeBoundaryQueryRunner implements QueryRunner> { private final StorageAdapter adapter; - private final Function> skipToFirstMatching; 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); - } - }; } - private DateTime getTimeBoundary(StorageAdapter adapter, TimeBoundaryQuery legacyQuery, boolean descending) + @Nullable + private DateTime getTimeBoundaryFromTimeOrderedAdapter( + StorageAdapter adapter, + TimeBoundaryQuery legacyQuery, + boolean descending + ) { final Sequence> resultSequence = QueryRunnerHelper.makeCursorBasedQuery( adapter, @@ -123,17 +114,60 @@ private DateTime getTimeBoundary(StorageAdapter adapter, TimeBoundaryQuery legac VirtualColumns.EMPTY, descending, Granularities.ALL, - this.skipToFirstMatching, + cursor -> returnFirstTimestamp(cursor), null ); final List> resultList = resultSequence.limit(1).toList(); - if (resultList.size() > 0) { + if (!resultList.isEmpty()) { return resultList.get(0).getValue(); } return null; } + private Pair getTimeBoundaryFromNonTimeOrderedAdapter( + StorageAdapter adapter, + TimeBoundaryQuery legacyQuery + ) + { + final Sequence> resultSequence = adapter.makeCursors( + Filters.toFilter(legacyQuery.getFilter()), + CollectionUtils.getOnlyElement( + legacyQuery.getQuerySegmentSpec().getIntervals(), + intervals -> DruidException.defensive("Can only handle a single interval, got[%s]", intervals) + ), + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ).map( + cursor -> { + if (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( + !legacyQuery.isMaxTime() ? DateTimes.utc(minTime) : null, + !legacyQuery.isMinTime() ? DateTimes.utc(maxTime) : null + ); + } + ); + + final List> resultList = resultSequence.limit(1).toList(); + return !resultList.isEmpty() ? resultList.get(0) : Pair.of(null, null); + } + @Override public Sequence> run( final QueryPlus> queryPlus, @@ -170,16 +204,20 @@ public Iterator> make() if (!query.isMinTime()) { maxTime = adapter.getMaxTime(); } - } else { + } else if (adapter.isTimeOrdered()) { if (!query.isMaxTime()) { - minTime = getTimeBoundary(adapter, query, false); + minTime = getTimeBoundaryFromTimeOrderedAdapter(adapter, query, false); } if (!query.isMinTime()) { if (query.isMaxTime() || minTime != null) { - maxTime = getTimeBoundary(adapter, query, true); + maxTime = getTimeBoundaryFromTimeOrderedAdapter(adapter, query, true); } } + } else { + final Pair minMaxTime = getTimeBoundaryFromNonTimeOrderedAdapter(adapter, query); + minTime = minMaxTime.left(); + maxTime = minMaxTime.right(); } return query.buildResult( @@ -210,8 +248,8 @@ private static boolean canUseAdapterMinMaxTime(final TimeBoundaryQuery query, fi return false; } - if (!(query.getDataSource() instanceof TableDataSource)) { - // In general, minTime / maxTime are only guaranteed to match data for regular tables. + if (!(query.getDataSource() instanceof TableDataSource) || !adapter.isTimeOrdered()) { + // In general, minTime / maxTime are only guaranteed to match data for regular tables that are time-ordered. // // 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. @@ -232,4 +270,15 @@ private static boolean canUseAdapterMinMaxTime(final TimeBoundaryQuery query, fi // Passed all checks. return true; } + + private static Result returnFirstTimestamp(final 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<>(DateTimes.EPOCH /* Unused */, timestamp); + } } 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 163befcf2817..31598963b827 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,6 +20,7 @@ package org.apache.druid.query.vector; import com.google.common.collect.Iterables; +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.segment.column.ColumnHolder; @@ -76,6 +77,8 @@ public static VectorCursorGranularizer create( final Interval queryInterval ) { + Granularities.validateGranularity(storageAdapter, granularity); + final DateTime minTime = storageAdapter.getMinTime(); final DateTime maxTime = storageAdapter.getMaxTime(); 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 1168e34b28af..9e1c86416f6f 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java @@ -32,6 +32,7 @@ import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.List; public class FilteredStorageAdapter implements StorageAdapter { @@ -143,6 +144,12 @@ public Metadata getMetadata() return baseStorageAdapter.getMetadata(); } + @Override + public List getSortOrder() + { + return baseStorageAdapter.getSortOrder(); + } + @Override public boolean isFromTombstone() { 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..cefccec8d6e0 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,12 @@ private File makeIndexFiles( ); } + if (segmentMetadata != null + && segmentMetadata.getSortOrder() != null + && !segmentMetadata.getSortOrder().contains(ColumnHolder.TIME_COLUMN_NAME)) { + throw DruidException.defensive("sortOrder must include[%s]", ColumnHolder.TIME_COLUMN_NAME); + } + Closer closer = Closer.create(); try { final FileSmoosher v9Smoosher = new FileSmoosher(outDir); @@ -220,7 +233,7 @@ private File makeIndexFiles( progress.progress(); final TimeAndDimsIterator timeAndDimsIterator = makeMergedTimeAndDimsIterator( adapters, - mergedDimensions, + mergedDimensionsWithTime, mergedMetrics, rowMergerFn, handlers, @@ -827,7 +840,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 +1095,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 +1118,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 +1169,7 @@ private File merge( sortedMetricAggs, outDir, progress, - mergedDimensions, + mergedDimensionsWithTime, new DimensionsSpecInspector(storeEmptyColumns, dimensionsSpec), mergedMetrics, rowMergerFn, @@ -1183,7 +1195,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 +1206,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 +1222,7 @@ private TimeAndDimsIterator makeMergedTimeAndDimsIterator( } private TransformableRowIterator makeRowIteratorWithReorderedColumns( - List reorderedDimensions, + List reorderedDimensionsWithTime, List reorderedMetrics, Map originalHandlers, IndexableAdapter originalAdapter, @@ -1217,14 +1230,14 @@ private TransformableRowIterator makeRowIteratorWithReorderedColumns( ) { RowPointer reorderedRowPointer = reorderRowPointerColumns( - reorderedDimensions, + reorderedDimensionsWithTime, reorderedMetrics, originalHandlers, originalAdapter, originalIterator.getPointer() ); TimeAndDimsPointer reorderedMarkedRowPointer = reorderRowPointerColumns( - reorderedDimensions, + reorderedDimensionsWithTime, reorderedMetrics, originalHandlers, originalAdapter, @@ -1247,17 +1260,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 +1284,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 +1302,7 @@ private static T reorderRowPointerColumns( //noinspection unchecked return (T) new RowPointer( originalRowPointer.timestampSelector, + reorderedTimePosition, reorderedDimensionSelectors, reorderedHandlers, reorderedMetricSelectors, @@ -1292,6 +1313,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/Metadata.java b/processing/src/main/java/org/apache/druid/segment/Metadata.java index c3ac554a0faf..f0e26a1fac72 100644 --- a/processing/src/main/java/org/apache/druid/segment/Metadata.java +++ b/processing/src/main/java/org/apache/druid/segment/Metadata.java @@ -19,15 +19,18 @@ 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.granularity.Granularity; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.column.ColumnHolder; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -39,6 +42,11 @@ @PublicApi public class Metadata { + /** + * Return value for {@link #getSortOrder()} when a segment is sorted by {@link ColumnHolder#TIME_COLUMN_NAME} only. + */ + public static final List SORTED_BY_TIME_ONLY = Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME); + // container is used for arbitrary key-value pairs in segment metadata e.g. // kafka input reader uses it to store commit offset private final Map container; @@ -50,13 +58,16 @@ public class Metadata private final Granularity queryGranularity; @Nullable private final Boolean rollup; + @Nullable + private final List sortOrder; 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("sortOrder") @Nullable List sortOrder ) { this.container = container == null ? new ConcurrentHashMap<>() : container; @@ -64,6 +75,7 @@ public Metadata( this.timestampSpec = timestampSpec; this.queryGranularity = queryGranularity; this.rollup = rollup; + this.sortOrder = sortOrder; } @JsonProperty @@ -100,6 +112,14 @@ public Boolean isRollup() return rollup; } + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getSortOrder() + { + return sortOrder; + } + public Metadata putAll(@Nullable Map other) { if (other != null) { @@ -130,6 +150,7 @@ public static Metadata merge( List timestampSpecsToMerge = new ArrayList<>(); List gransToMerge = new ArrayList<>(); List rollupToMerge = new ArrayList<>(); + List> sortOrdersToMerge = new ArrayList<>(); for (Metadata metadata : toBeMerged) { if (metadata != null) { @@ -149,6 +170,8 @@ public static Metadata merge( if (rollupToMerge != null) { rollupToMerge.add(metadata.isRollup()); } + + sortOrdersToMerge.add(metadata.getSortOrder()); mergedContainer.putAll(metadata.container); } else { //if metadata and hence aggregators and queryGranularity for some segment being merged are unknown then @@ -176,6 +199,8 @@ public static Metadata merge( null : Granularity.mergeGranularities(gransToMerge); + final List mergedSortOrder = mergeSortOrders(sortOrdersToMerge); + Boolean rollup = null; if (rollupToMerge != null && !rollupToMerge.isEmpty()) { rollup = rollupToMerge.get(0); @@ -197,7 +222,8 @@ public static Metadata merge( mergedAggregators, mergedTimestampSpec, mergedGranularity, - rollup + rollup, + mergedSortOrder ); } @@ -215,7 +241,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(sortOrder, metadata.sortOrder); } @Override @@ -233,6 +260,39 @@ public String toString() ", timestampSpec=" + timestampSpec + ", queryGranularity=" + queryGranularity + ", rollup=" + rollup + + ", sortOrder=" + sortOrder + '}'; } + + @Nullable + private static List mergeSortOrders(List> sortOrdersToMerge) + { + if (sortOrdersToMerge.isEmpty()) { + return null; + } + + if (sortOrdersToMerge.stream().anyMatch(Objects::isNull)) { + return null; + } + + final List mergedSortOrder = new ArrayList<>(); + + while (true) { + String column = null; + + for (final List sortOrder : sortOrdersToMerge) { + if (mergedSortOrder.size() >= sortOrder.size()) { + return mergedSortOrder; + } + + if (column == null) { + column = sortOrder.get(mergedSortOrder.size()); + } else if (!column.equals(sortOrder.get(mergedSortOrder.size()))) { + return mergedSortOrder; + } + } + + mergedSortOrder.add(column); + } + } } 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..ddaeb7f46239 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java @@ -42,9 +42,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 +71,11 @@ default ColumnCapabilities getColumnCapabilities(String column) return columnHolder.getCapabilities(); } + /** + * Returns column names that this index is sorted by. + */ + List getSortOrder(); + /** * 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..d711b83da040 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java @@ -44,6 +44,7 @@ public class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactor { private final VirtualColumns virtualColumns; private final boolean descending; + private final boolean timeOrdered; protected final ReadableOffset offset; // Share Column objects, since they cache decompressed buffers internally, and we can avoid recomputation if the @@ -57,12 +58,14 @@ public class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactor public QueryableIndexColumnSelectorFactory( VirtualColumns virtualColumns, boolean descending, + boolean timeOrdered, ReadableOffset offset, ColumnCache columnCache ) { this.virtualColumns = virtualColumns; this.descending = descending; + this.timeOrdered = timeOrdered; this.offset = offset; this.columnCache = columnCache; this.dimensionSelectorCache = new HashMap<>(); @@ -107,7 +110,7 @@ private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensi return DimensionSelector.constant(null, extractionFn); } - if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME)) { + if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME) && timeOrdered) { return new SingleScanTimeDimensionSelector(makeColumnValueSelector(dimension), extractionFn, descending); } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java index 86069068eaf9..8a3fcb031b1b 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java @@ -37,13 +37,16 @@ import org.apache.druid.query.QueryMetrics; 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.Filters; import org.apache.druid.segment.historical.HistoricalCursor; import org.apache.druid.segment.vector.BitmapVectorOffset; import org.apache.druid.segment.vector.FilteredVectorOffset; @@ -58,6 +61,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Arrays; import java.util.concurrent.TimeUnit; public class QueryableIndexCursorSequenceBuilder @@ -68,6 +72,7 @@ public class QueryableIndexCursorSequenceBuilder private final VirtualColumns virtualColumns; @Nullable private final Filter filter; + private final boolean timeOrdered; @Nullable private final QueryMetrics metrics; private final long minDataTimestamp; @@ -79,6 +84,7 @@ public QueryableIndexCursorSequenceBuilder( Interval interval, VirtualColumns virtualColumns, @Nullable Filter filter, + boolean timeOrdered, @Nullable QueryMetrics metrics, long minDataTimestamp, long maxDataTimestamp, @@ -89,6 +95,7 @@ public QueryableIndexCursorSequenceBuilder( this.interval = interval; this.virtualColumns = virtualColumns; this.filter = filter; + this.timeOrdered = timeOrdered; this.metrics = metrics; this.minDataTimestamp = minDataTimestamp; this.maxDataTimestamp = maxDataTimestamp; @@ -111,7 +118,8 @@ public Sequence build(final Granularity gran) ); final int numRows = index.getNumRows(); - final FilterBundle filterBundle = makeFilterBundle(bitmapIndexSelector, numRows); + final FilterBundle filterBundle = + makeFilterBundle(computeFilterWithIntervalIfNeeded(), bitmapIndexSelector, numRows, metrics); // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter // can use an index @@ -142,39 +150,48 @@ public Cursor apply(final Interval inputInterval) gran.increment(inputInterval.getStartMillis()) ); - if (descending) { - for (; baseOffset.withinBounds(); baseOffset.increment()) { - if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) { - break; + if (timeOrdered) { + if (descending) { + for (; baseOffset.withinBounds(); baseOffset.increment()) { + if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) < timeEnd) { + break; + } } - } - } else { - for (; baseOffset.withinBounds(); baseOffset.increment()) { - if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) { - break; + } else { + for (; baseOffset.withinBounds(); baseOffset.increment()) { + if (timestamps.getLongSingleValueRow(baseOffset.getOffset()) >= timeStart) { + break; + } } } } - final Offset offset = descending ? - new DescendingTimestampCheckingOffset( - baseOffset, - timestamps, - timeStart, - minDataTimestamp >= timeStart - ) : - new AscendingTimestampCheckingOffset( - baseOffset, - timestamps, - timeEnd, - maxDataTimestamp < timeEnd - ); - + final Offset offset; + + if (timeOrdered && descending) { + offset = new DescendingTimestampCheckingOffset( + baseOffset, + timestamps, + timeStart, + minDataTimestamp >= timeStart + ); + } else if (timeOrdered) { + offset = new AscendingTimestampCheckingOffset( + baseOffset, + timestamps, + timeEnd, + maxDataTimestamp < timeEnd + ); + } 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, + timeOrdered, baseCursorOffset.getBaseReadableOffset(), columnCache ); @@ -218,14 +235,17 @@ public VectorCursor buildVectorized(final int vectorSize) ); final int numRows = index.getNumRows(); - final FilterBundle filterBundle = makeFilterBundle(bitmapIndexSelector, numRows); + final FilterBundle filterBundle = + makeFilterBundle(computeFilterWithIntervalIfNeeded(), bitmapIndexSelector, numRows, metrics); NumericColumn timestamps = null; + // startOffset, endOffset 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 (timeOrdered && interval.getStartMillis() > minDataTimestamp) { timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME); startOffset = timeSearch(timestamps, interval.getStartMillis(), 0, index.getNumRows()); @@ -233,7 +253,7 @@ public VectorCursor buildVectorized(final int vectorSize) startOffset = 0; } - if (interval.getEndMillis() <= maxDataTimestamp) { + if (timeOrdered && interval.getEndMillis() <= maxDataTimestamp) { if (timestamps == null) { timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME); } @@ -281,10 +301,39 @@ public VectorCursor buildVectorized(final int vectorSize) } } + /** + * Compute filter to use for cursor creation. For non-time-ordered segments, this includes the query interval + * as a filter. + */ @Nullable - private FilterBundle makeFilterBundle( + private Filter computeFilterWithIntervalIfNeeded() + { + if (!timeOrdered && minDataTimestamp < interval.getStartMillis() || maxDataTimestamp >= interval.getEndMillis()) { + return Filters.and( + Arrays.asList( + new RangeFilter( + ColumnHolder.TIME_COLUMN_NAME, + ColumnType.LONG, + minDataTimestamp < interval.getStartMillis() ? interval.getStartMillis() : null, + maxDataTimestamp >= interval.getEndMillis() ? interval.getEndMillis() : null, + false, + true, + null + ), + filter + ) + ); + } else { + return filter; + } + } + + @Nullable + private static FilterBundle makeFilterBundle( + @Nullable Filter filter, ColumnSelectorColumnIndexSelector bitmapIndexSelector, - int numRows + int numRows, + @Nullable QueryMetrics metrics ) { final BitmapFactory bitmapFactory = bitmapIndexSelector.getBitmapFactory(); 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..c172ea41216b 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java @@ -21,6 +21,7 @@ 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.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.BaseColumn; @@ -60,13 +61,45 @@ 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(); + + if (metadata != null && metadata.getSortOrder() != null) { + 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 String columnName : metadata.getSortOrder()) { + 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 = metadata.getSortOrder().size(); + } + } else { + this.timePositionForComparator = 0; + } } public QueryableIndex getQueryableIndex() @@ -87,16 +120,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)); } @@ -263,6 +303,7 @@ class RowIteratorImpl implements TransformableRowIterator final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( VirtualColumns.EMPTY, false, + timePositionForComparator == 0, offset, columnCache ); @@ -292,6 +333,7 @@ class RowIteratorImpl implements TransformableRowIterator rowPointer = new RowPointer( rowTimestampSelector, + timePositionForComparator, rowDimensionValueSelectors, dimensionHandlers, rowMetricSelectors, @@ -309,6 +351,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/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index 57c7da953605..8b9510062e7d 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -43,6 +43,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.LinkedHashSet; +import java.util.List; import java.util.Objects; import java.util.Set; @@ -231,6 +232,7 @@ public VectorCursor makeVectorCursor( actualInterval, virtualColumns, filter, + isTimeOrdered(), queryMetrics, getMinTime().getMillis(), getMaxTime().getMillis(), @@ -248,6 +250,8 @@ public Sequence makeCursors( @Nullable QueryMetrics queryMetrics ) { + Granularities.validateGranularity(this, gran); + if (queryMetrics != null) { queryMetrics.vectorized(false); } @@ -264,6 +268,7 @@ public Sequence makeCursors( actualInterval, virtualColumns, filter, + isTimeOrdered(), queryMetrics, getMinTime().getMillis(), getMaxTime().getMillis(), @@ -279,13 +284,26 @@ public Metadata getMetadata() return index.getMetadata(); } + @Override + public List getSortOrder() + { + return index.getSortOrder(); + } + 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)); + if (isTimeOrdered()) { + // StorageAdapter getMinTime, getMaxTime contract requires exact min/max time in the time-ordered case. + 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 { + // Exact min/max time not required. + this.minTime = getInterval().getStart(); + this.maxTime = getInterval().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 efd3b96829d6..27d116d641ee 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java @@ -58,7 +58,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"); @@ -167,6 +168,15 @@ public Metadata getMetadata() throw new UnsupportedOperationException("Cannot retrieve metadata"); } + @Override + public List getSortOrder() + { + // It's possibly incorrect in some cases for sort order to be SORTED_BY_TIME_ONLY here, but for historical reasons, + // we're keeping this in place for now. The handling of "interval" in "makeCursors", which has been in place for + // some time, suggests we think the data is always sorted by time. + return Metadata.SORTED_BY_TIME_ONLY; + } + @Override public Sequence makeCursors( @Nullable final Filter filter, 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/SimpleQueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java index 013a634fdc4b..838a42db199d 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java @@ -102,6 +102,18 @@ public Indexed getAvailableDimensions() return availableDimensions; } + @Override + public List getSortOrder() + { + final Metadata metadata = getMetadata(); + if (metadata != null && metadata.getSortOrder() != null) { + return metadata.getSortOrder(); + } else { + // When sort order isn't set in metadata.drd, assume the segment is sorted by __time. + return Metadata.SORTED_BY_TIME_ONLY; + } + } + @Override public BitmapFactory getBitmapFactoryForDimensions() { 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 2d3fc6a50bbb..25331fe94fec 100644 --- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java @@ -21,13 +21,17 @@ import com.google.common.collect.Iterables; import org.apache.druid.guice.annotations.PublicApi; +import org.apache.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory; 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.incremental.IncrementalIndexStorageAdapter; import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.List; import java.util.Optional; /** @@ -36,7 +40,15 @@ public interface StorageAdapter extends CursorFactory, ColumnInspector { 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(); /** @@ -69,25 +81,25 @@ default RowSignature getRowSignature() /** * 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 + * {@link 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. + * For {@link QueryableIndexStorageAdapter} and {@link IncrementalIndexStorageAdapter} specifically, which back + * regular tables (i.e. {@link org.apache.druid.query.TableDataSource}), this method contract is tighter: if the + * table {@link #isTimeOrdered()} then this method does return the actual minimum data timestamp. This fact is + * leveraged by {@link TimeBoundaryQueryRunnerFactory} to return results using metadata only. */ DateTime getMinTime(); /** * 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 + * {@link 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. + * For {@link QueryableIndexStorageAdapter} and {@link IncrementalIndexStorageAdapter} specifically, which back + * regular tables (i.e. {@link org.apache.druid.query.TableDataSource}), this method contract is tighter: if the + * table {@link #isTimeOrdered()} then this method does return the actual maximum data timestamp. This fact is + * leveraged by {@link TimeBoundaryQueryRunnerFactory} to return results using metadata only. */ DateTime getMaxTime(); @@ -129,6 +141,18 @@ default RowSignature getRowSignature() @Nullable Metadata getMetadata(); + /** + * Returns column names that this adapter's data is sorted by. Cursors returned by this adapter return rows in + * this ordering, using the natural comparator for the type of the column as returned by + * {@link #getColumnCapabilities(String)}. Includes {@link ColumnHolder#TIME_COLUMN_NAME} if appropriate. + */ + List getSortOrder(); + + default boolean isTimeOrdered() + { + return ColumnHolder.TIME_COLUMN_NAME.equals(Iterables.getFirst(getSortOrder(), null)); + } + /** * Returns true if this storage adapter can filter some rows out. The actual column cardinality can be lower than * what {@link #getDimensionCardinality} returns if this returns true. Dimension selectors for such storage adapter 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/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index 752c6f486786..71eac0aead7f 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -256,6 +256,23 @@ public Metadata getMetadata() return baseAdapter.getMetadata(); } + @Override + public List getSortOrder() + { + final List baseSortOrder = baseAdapter.getSortOrder(); + + // Sorted the same way as the base segment, unless the unnested column shadows one of the base columns. + int limit = 0; + for (; limit < baseSortOrder.size(); limit++) { + final String columnName = baseSortOrder.get(limit); + if (columnName.equals(outputColumnName) || columnName.equals(unnestColumn.getOutputName())) { + break; + } + } + + return limit == baseSortOrder.size() ? baseSortOrder : baseSortOrder.subList(0, limit); + } + @Override public boolean isFromTombstone() { 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 06fc7195d526..73306f0fa31c 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 @@ -25,6 +25,7 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.primitives.Ints; @@ -234,6 +235,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 +287,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.gran, - this.rollup - ); initAggs(metrics, inputRowHolder); @@ -319,8 +319,23 @@ 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 useExplicitSegmentSortOrder. + this.timePosition = dimensionsSpec.isUseExplicitSegmentSortOrder() ? dimensionDescsList.size() : 0; + } else { + this.timePosition = foundTimePosition; } //__time capabilities @@ -334,6 +349,17 @@ protected IncrementalIndex( if (!spatialDimensions.isEmpty()) { this.rowTransformers.add(new SpatialDimensionRowTransformer(spatialDimensions)); } + + // Set metadata last, so dimensionOrder is populated + final List dimensionOrder = getDimensionOrder(); + this.metadata = new Metadata( + null, + getCombiningAggregators(metrics), + incrementalIndexSchema.getTimestampSpec(), + this.gran, + this.rollup, + ColumnHolder.TIME_COLUMN_NAME.equals(Iterables.getFirst(dimensionOrder, null)) ? null : dimensionOrder + ); } public abstract FactsHolder getFacts(); @@ -582,7 +608,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 +815,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 +854,9 @@ public List getDimensions() } } + /** + * Returns the descriptor for a particular dimension. + */ @Nullable public DimensionDesc getDimension(String dimension) { @@ -859,11 +913,12 @@ public Integer getDimensionIndex(String dimension) return dimSpec == null ? null : dimSpec.getIndex(); } + /** + * Returns names of time and dimension columns, in 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 +942,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 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 +984,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 +1130,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 +1181,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 +1196,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..7469b3c0f709 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 @@ -75,7 +75,7 @@ private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensi final String dimension = dimensionSpec.getDimension(); final ExtractionFn extractionFn = dimensionSpec.getExtractionFn(); - if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME)) { + if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME) && adapter.isTimeOrdered()) { return new SingleScanTimeDimensionSelector(makeColumnValueSelector(dimension), extractionFn, descending); } 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..2a4aebf72c4a 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 @@ -91,6 +91,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 8d72133a205c..d4fc3e160f28 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 @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +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.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -48,7 +49,9 @@ import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.Collections; import java.util.Iterator; +import java.util.List; /** */ @@ -138,7 +141,7 @@ public Interval getInterval() @Override public Indexed getAvailableDimensions() { - return new ListIndexed<>(index.getDimensionNames()); + return new ListIndexed<>(index.getDimensionNames(false)); } @Override @@ -265,6 +268,8 @@ public Sequence makeCursors( @Nullable QueryMetrics queryMetrics ) { + Granularities.validateGranularity(this, gran); + if (index.isEmpty()) { return Sequences.empty(); } @@ -295,6 +300,16 @@ public Metadata getMetadata() return index.getMetadata(); } + @Override + public List getSortOrder() + { + if (index.timePosition == 0) { + return Metadata.SORTED_BY_TIME_ONLY; + } else { + return Collections.emptyList(); + } + } + private class IncrementalIndexCursor implements Cursor { private IncrementalIndexRowHolder currEntry; 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/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 9dade664f8ac..2e3d6953e1c4 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 @@ -208,6 +208,22 @@ public Metadata getMetadata() throw new UnsupportedOperationException("Cannot retrieve metadata from join segment"); } + @Override + public List getSortOrder() + { + final List baseSortOrder = baseAdapter.getSortOrder(); + + // Sorted the same way as the base segment, unless a joined-in column shadows one of the base columns. + int limit = 0; + for (; limit < baseSortOrder.size(); limit++) { + if (!isBaseColumn(baseSortOrder.get(limit))) { + break; + } + } + + return limit == baseSortOrder.size() ? baseSortOrder : baseSortOrder.subList(0, limit); + } + @Override public boolean hasBuiltInFilters() { 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 de306209f908..dc594fad20c8 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 @@ -241,6 +241,7 @@ public ColumnSelectorFactory makeColumnSelectorFactory(ReadableOffset offset, bo return new QueryableIndexColumnSelectorFactory( VirtualColumns.EMPTY, descending, + adapter.isTimeOrdered(), offset, new ColumnCache(queryableIndex, 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..52a22afd11bd 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 @@ -36,7 +36,6 @@ import org.joda.time.Interval; import javax.annotation.Nullable; - import java.io.File; import java.util.List; import java.util.Map; @@ -102,6 +101,12 @@ public Map getDimensionHandlers() throw new UnsupportedOperationException(); } + @Override + public List getSortOrder() + { + 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..351f3032d028 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 + // useExplicitSegmentSortOrder: true, 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/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..d1c45ec66128 --- /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)} 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/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..9ff6f499cc7a 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 @@ -137,7 +137,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/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index 475848edbdc7..d4d228f1b202 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 @@ -65,6 +65,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; @@ -141,6 +142,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 +201,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 +381,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 +416,7 @@ public GroupByQueryRunnerTest( String testName, GroupByQueryConfig config, GroupByQueryRunnerFactory factory, - QueryRunner runner, + TestQueryRunner runner, boolean vectorize ) { @@ -428,6 +430,9 @@ public GroupByQueryRunnerTest( @Test public void testGroupBy() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -741,6 +746,9 @@ public void testGroupByOnMissingColumn() @Test public void testGroupByWithStringPostAggregator() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -989,6 +997,9 @@ public void testGroupByWithStringPostAggregator() @Test public void testGroupByWithStringVirtualColumn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -1052,6 +1063,9 @@ public void testGroupByWithStringVirtualColumn() @Test public void testGroupByWithStringVirtualColumnVectorizable() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -1115,6 +1129,9 @@ public void testGroupByWithStringVirtualColumnVectorizable() @Test public void testGroupByWithDurationGranularity() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -1167,6 +1184,9 @@ public void testGroupByWithOutputNameCollisions() @Test public void testGroupByWithSortDimsFirst() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -1212,6 +1232,9 @@ public void testGroupByWithSortDimsFirst() @Test public void testGroupByNoAggregators() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -1248,6 +1271,9 @@ public void testGroupByNoAggregators() @Test public void testMultiValueDimension() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to multi-value dimensions. cannotVectorize(); @@ -1278,6 +1304,9 @@ public void testMultiValueDimension() @Test public void testMultiValueDimensionNotAllowed() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final String dimName = "placementish"; if (!vectorize) { @@ -2743,6 +2772,9 @@ public void testMultipleDimensionsOneOfWhichIsMultiValueDifferentOrder() @Test public void testGroupByMaxRowsLimitContextOverride() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -2799,6 +2831,9 @@ public void testGroupByMaxRowsLimitContextOverride() @Test public void testGroupByTimeoutContextOverride() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -2837,6 +2872,9 @@ public void testGroupByTimeoutContextOverride() @Test public void testGroupByMaxOnDiskStorageContextOverride() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -2858,6 +2896,9 @@ public void testGroupByMaxOnDiskStorageContextOverride() @Test public void testNotEnoughDiskSpaceThroughContextOverride() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -2883,6 +2924,9 @@ public void testNotEnoughDiskSpaceThroughContextOverride() @Test public void testSubqueryWithOuterMaxOnDiskStorageContextOverride() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -2918,6 +2962,9 @@ public void testSubqueryWithOuterMaxOnDiskStorageContextOverride() @Test public void testGroupByWithRebucketRename() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -2989,6 +3036,9 @@ public void testGroupByWithRebucketRename() @Test public void testGroupByWithSimpleRenameRetainMissingNonInjective() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -3060,6 +3110,9 @@ public void testGroupByWithSimpleRenameRetainMissingNonInjective() @Test public void testGroupByWithSimpleRenameRetainMissing() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map map = new HashMap<>(); map.put("automotive", "automotive0"); map.put("business", "business0"); @@ -3114,6 +3167,9 @@ public void testGroupByWithSimpleRenameRetainMissing() @Test public void testGroupByWithSimpleRenameAndMissingString() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map map = new HashMap<>(); map.put("automotive", "automotive0"); map.put("business", "business0"); @@ -3163,6 +3219,9 @@ public void testGroupByWithSimpleRenameAndMissingString() @Test public void testGroupByWithSimpleRename() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map map = new HashMap<>(); map.put("automotive", "automotive0"); map.put("business", "business0"); @@ -3313,6 +3372,9 @@ public void testGroupByWithCardinality() @Test public void testGroupByWithFirstLast() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -3400,6 +3462,9 @@ public void testGroupByWithFirstLast() @Test public void testGroupByWithNoResult() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.EMPTY_INTERVAL) @@ -3422,6 +3487,9 @@ public void testGroupByWithNoResult() @Test public void testGroupByWithNullProducingDimExtractionFn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -3483,6 +3551,9 @@ public String apply(String dimValue) */ public void testGroupByWithEmptyStringProducingDimExtractionFn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final ExtractionFn emptyStringExtractionFn = new RegexDimExtractionFn("(\\w{1})", false, null) { @Override @@ -3536,6 +3607,9 @@ public String apply(String dimValue) @Test public void testGroupByWithTimeZone() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles"); GroupByQuery query = makeQueryBuilder() @@ -3746,6 +3820,9 @@ public void testGroupByWithTimeZone() @Test public void testMergeResults() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -3835,6 +3912,9 @@ public void testMergeResultsWithLimitAndOffset() private void doTestMergeResultsWithValidLimit(final int limit, final int offset) { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -3950,6 +4030,9 @@ private void doTestMergeResultsWithValidLimit(final int limit, final int offset) @Test public void testMergeResultsAcrossMultipleDaysWithLimitAndOrderBy() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final int limit = 14; GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -3992,6 +4075,9 @@ public void testMergeResultsAcrossMultipleDaysWithLimitAndOrderBy() @Test public void testMergeResultsAcrossMultipleDaysWithLimitAndOrderByUsingMathExpressions() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final int limit = 14; GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -4056,6 +4142,9 @@ public void testMergeResultsWithNegativeLimit() @Test public void testMergeResultsWithOrderBy() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + LimitSpec[] orderBySpecs = new LimitSpec[]{ new DefaultLimitSpec(OrderByColumnSpec.ascending("idx"), null), new DefaultLimitSpec(OrderByColumnSpec.ascending("rows", "idx"), null), @@ -4150,6 +4239,9 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r @Test public void testGroupByOrderLimit() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -4248,6 +4340,9 @@ public void testGroupByOrderLimit() @Test public void testGroupByWithOrderLimit2() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -4289,6 +4384,9 @@ public void testGroupByWithOrderLimit2() @Test public void testGroupByWithOrderLimit3() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -4332,6 +4430,9 @@ public void testGroupByWithOrderLimit3() @Test public void testGroupByOrderLimitNumeric() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -4704,6 +4805,9 @@ public void testGroupByWithLimitOnFinalizedHyperUnique() @Test public void testGroupByWithAlphaNumericDimensionOrder() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -4762,6 +4866,9 @@ public void testGroupByWithAlphaNumericDimensionOrder() @Test public void testGroupByWithLookupAndLimitAndSortByDimsFirst() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -4822,6 +4929,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. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setGranularity(QueryRunnerTestHelper.DAY_GRAN).setDimensions(new DefaultDimensionSpec( @@ -4860,6 +4970,9 @@ public void testLimitPerGrouping() @Test public void testPostAggMergedHavingSpec() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5085,6 +5198,9 @@ public void testGroupByWithOrderLimitHavingSpec() @Test public void testPostAggHavingSpec() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5140,6 +5256,9 @@ public void testPostAggHavingSpec() @Test public void testHavingSpec() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5173,6 +5292,9 @@ public void testHavingSpec() @Test public void testDimFilterHavingSpec() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final DimFilterHavingSpec havingSpec = new DimFilterHavingSpec( new AndDimFilter( ImmutableList.of( @@ -5278,6 +5400,9 @@ public void testDimFilterHavingSpec() @Test public void testDimFilterHavingSpecWithExtractionFns() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + String extractionJsFn = "function(str) { return 'super-' + str; }"; ExtractionFn extractionFn = new JavaScriptExtractionFn( extractionJsFn, @@ -5328,6 +5453,9 @@ public void testDimFilterHavingSpecWithExtractionFns() @Test public void testMergedHavingSpec() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5391,6 +5519,9 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r @Test public void testMergedPostAggHavingSpec() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5500,6 +5631,9 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r @Test public void testCustomAggregatorHavingSpec() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5540,6 +5674,9 @@ public void testCustomAggregatorHavingSpec() @Test public void testGroupByWithRegEx() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5565,6 +5702,9 @@ public void testGroupByWithRegEx() @Test public void testGroupByWithNonexistentDimension() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -5625,6 +5765,9 @@ public void testGroupByWithNonexistentDimension() @Test public void testIdenticalSubquery() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5681,6 +5824,9 @@ public void testIdenticalSubquery() @Test public void testSubqueryWithMultipleIntervalsInOuterQuery() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5746,6 +5892,9 @@ public void testSubqueryWithExtractionFnInOuterQuery() { //https://github.com/apache/druid/issues/2556 + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5791,6 +5940,9 @@ public void testSubqueryWithExtractionFnInOuterQuery() @Test public void testDifferentGroupingSubquery() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5854,6 +6006,9 @@ public void testDifferentGroupingSubquery() @Test public void testDifferentGroupingSubqueryMultipleAggregatorsOnSameField() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5903,6 +6058,9 @@ public void testDifferentGroupingSubqueryMultipleAggregatorsOnSameField() @Test public void testDifferentGroupingSubqueryWithFilter() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5945,6 +6103,9 @@ public void testDifferentGroupingSubqueryWithFilter() @Test public void testDifferentIntervalSubquery() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -5971,7 +6132,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 +6218,9 @@ public void testGroupByWithUnderUnderTimeAsDimensionNameWithHavingAndLimit() @Test public void testEmptySubquery() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.EMPTY_INTERVAL) @@ -6079,6 +6243,9 @@ public void testEmptySubquery() @Test public void testSubqueryWithPostAggregators() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -6353,6 +6520,9 @@ public void testSubqueryWithPostAggregators() @Test public void testSubqueryWithPostAggregatorsAndHaving() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -6627,6 +6797,9 @@ public boolean eval(ResultRow row) @Test public void testSubqueryWithMultiColumnAggregators() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to javascript functionality. cannotVectorize(); @@ -6810,6 +6983,9 @@ public boolean eval(ResultRow row) @Test public void testSubqueryWithOuterFilterAggregator() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -6837,6 +7013,9 @@ public void testSubqueryWithOuterFilterAggregator() @Test public void testSubqueryWithOuterTimeFilter() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -6882,6 +7061,9 @@ public void testSubqueryWithOuterTimeFilter() @Test public void testSubqueryWithContextTimeout() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -6907,6 +7089,9 @@ public void testSubqueryWithContextTimeout() @Test public void testSubqueryWithOuterVirtualColumns() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -6962,6 +7147,9 @@ public void testSubqueryWithOuterCardinalityAggregator() @Test public void testSubqueryWithOuterCountAggregator() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -6992,6 +7180,9 @@ public void testSubqueryWithOuterCountAggregator() @Test public void testSubqueryWithOuterDimJavascriptAggregators() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7043,6 +7234,9 @@ public void testSubqueryWithOuterDimJavascriptAggregators() @Test public void testSubqueryWithOuterJavascriptAggregators() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + final GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7094,6 +7288,9 @@ public void testSubqueryWithOuterJavascriptAggregators() @Test public void testSubqueryWithHyperUniques() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7237,6 +7434,9 @@ public void testSubqueryWithHyperUniques() @Test public void testSubqueryWithHyperUniquesPostAggregator() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7294,6 +7494,9 @@ public void testSubqueryWithHyperUniquesPostAggregator() @Test public void testSubqueryWithFirstLast() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -7332,6 +7535,9 @@ public void testSubqueryWithFirstLast() @Test public void testGroupByWithSubtotalsSpecOfDimensionsPrefixes() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7446,6 +7652,9 @@ public void testGroupByWithSubtotalsSpecOfDimensionsPrefixes() @Test public void testGroupByWithSubtotalsSpecGeneral() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7802,6 +8011,9 @@ public void testGroupByWithSubtotalsSpecGeneral() @Test public void testGroupByWithSubtotalsSpecWithRenamedDimensionAndFilter() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = GroupByQuery .builder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -7896,6 +8108,9 @@ public void testGroupByWithSubtotalsSpecWithRenamedDimensionAndFilter() @Test public void testGroupByWithSubtotalsSpecWithLongDimensionColumn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -8187,6 +8402,9 @@ public void testGroupByWithSubtotalsSpecWithLongDimensionColumn() @Test public void testGroupByWithSubtotalsSpecWithOrderLimit() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -8222,6 +8440,9 @@ public void testGroupByWithSubtotalsSpecWithOrderLimit() @Test public void testGroupByWithSubtotalsSpecWithOrderLimitAndOffset() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -8257,6 +8478,9 @@ public void testGroupByWithSubtotalsSpecWithOrderLimitAndOffset() @Test public void testGroupByWithSubtotalsSpecWithOrderLimitForcePushdown() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -8834,6 +9058,9 @@ public ExtractionType getExtractionType() @Test public void testBySegmentResults() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -8894,6 +9121,9 @@ public void testBySegmentResults() @Test public void testBySegmentResultsUnOptimizedDimextraction() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04").setDimensions(new ExtractionDimensionSpec( @@ -8961,6 +9191,9 @@ public void testBySegmentResultsUnOptimizedDimextraction() @Test public void testBySegmentResultsOptimizedDimextraction() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04").setDimensions(new ExtractionDimensionSpec( @@ -9030,6 +9263,9 @@ public void testBySegmentResultsOptimizedDimextraction() @Test public void testGroupByWithExtractionDimFilter() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map extractionMap = new HashMap<>(); extractionMap.put("automotive", "automotiveAndBusinessAndNewsAndMezzanine"); extractionMap.put("business", "automotiveAndBusinessAndNewsAndMezzanine"); @@ -9089,6 +9325,9 @@ public void testGroupByWithExtractionDimFilter() @Test public void testGroupByWithExtractionDimFilterCaseMappingValueIsNullOrEmpty() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map extractionMap = new HashMap<>(); extractionMap.put("automotive", "automotive0"); extractionMap.put("business", "business0"); @@ -9138,6 +9377,9 @@ public void testGroupByWithExtractionDimFilterCaseMappingValueIsNullOrEmpty() @Test public void testGroupByWithExtractionDimFilterWhenSearchValueNotInTheMap() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map extractionMap = new HashMap<>(); MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, false); @@ -9165,6 +9407,9 @@ public void testGroupByWithExtractionDimFilterWhenSearchValueNotInTheMap() @Test public void testGroupByWithExtractionDimFilterKeyisNull() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map extractionMap = new HashMap<>(); @@ -9209,6 +9454,9 @@ public void testGroupByWithExtractionDimFilterKeyisNull() @Test public void testGroupByWithAggregatorFilterAndExtractionFunction() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map extractionMap = new HashMap<>(); extractionMap.put("automotive", "automotive0"); extractionMap.put("business", "business0"); @@ -9392,6 +9640,9 @@ public void testGroupByWithAggregatorFilterAndExtractionFunction() @Test public void testGroupByWithExtractionDimFilterOptimazitionManyToOne() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map extractionMap = new HashMap<>(); extractionMap.put("mezzanine", "newsANDmezzanine"); extractionMap.put("news", "newsANDmezzanine"); @@ -9431,6 +9682,9 @@ public void testGroupByWithExtractionDimFilterOptimazitionManyToOne() @Test public void testGroupByWithInjectiveLookupDimFilterNullDimsOptimized() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map extractionMap = new HashMap<>(); extractionMap.put("", "EMPTY"); @@ -9465,6 +9719,9 @@ public void testGroupByWithInjectiveLookupDimFilterNullDimsOptimized() @Test public void testGroupByWithInjectiveLookupDimFilterNullDimsNotOptimized() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map extractionMap = new HashMap<>(); extractionMap.put("", "EMPTY"); @@ -9499,6 +9756,9 @@ public void testGroupByWithInjectiveLookupDimFilterNullDimsNotOptimized() @Test public void testBySegmentResultsWithAllFiltersWithExtractionFns() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + String extractionJsFn = "function(str) { return 'super-' + str; }"; String jsFn = "function(x) { return(x === 'super-mezzanine') }"; ExtractionFn extractionFn = new JavaScriptExtractionFn( @@ -9593,6 +9853,9 @@ public void testBySegmentResultsWithAllFiltersWithExtractionFns() @Test public void testGroupByWithAllFiltersOnNullDimsWithExtractionFns() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + Map extractionMap = new HashMap<>(); extractionMap.put("", "EMPTY"); extractionMap.put(null, "EMPTY"); @@ -9639,6 +9902,9 @@ public void testGroupByWithAllFiltersOnNullDimsWithExtractionFns() @Test public void testGroupByCardinalityAggWithExtractionFn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -9731,6 +9997,9 @@ public void testGroupByCardinalityAggWithExtractionFn() @Test public void testGroupByCardinalityAggOnFloat() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -9922,6 +10191,9 @@ public void testGroupByCardinalityAggOnHyperUnique() @Test public void testGroupByLongColumn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10033,6 +10305,9 @@ public void testGroupByLongColumnDescending() @Test public void testGroupByLongColumnWithExFn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -10077,6 +10352,9 @@ public void testGroupByLongColumnWithExFn() @Test public void testGroupByLongTimeColumn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10115,6 +10393,9 @@ public void testGroupByLongTimeColumn() @Test public void testGroupByLongTimeColumnWithExFn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -10159,6 +10440,9 @@ public void testGroupByLongTimeColumnWithExFn() @Test public void testGroupByFloatColumn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10295,6 +10579,9 @@ public void testGroupByDoubleColumnDescending() @Test public void testGroupByFloatColumnWithExFn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -10394,6 +10681,9 @@ public void testGroupByWithHavingSpecOnLongAndFloat() @Test public void testGroupByLongAndFloatOutputAsString() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10439,6 +10729,9 @@ public void testGroupByLongAndFloatOutputAsString() @Test public void testGroupByNumericStringsAsNumeric() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10609,6 +10902,9 @@ public void testGroupByDecorationOnNumerics() @Test public void testGroupByNestedWithInnerQueryNumerics() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10680,6 +10976,9 @@ public void testGroupByNestedWithInnerQueryNumerics() @Test public void testGroupByNestedWithInnerQueryOutputNullNumerics() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + cannotVectorize(); // Following extractionFn will generate null value for one kind of quality @@ -10844,6 +11143,9 @@ public void testGroupByNestedWithInnerQueryNumericsWithLongTime() @Test public void testGroupByStringOutputAsLong() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -10893,6 +11195,9 @@ public void testGroupByStringOutputAsLong() @Test public void testGroupByWithAggsOnNumericDimensions() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to javascript aggregators. cannotVectorize(); @@ -10961,6 +11266,9 @@ public void testGroupByWithAggsOnNumericDimensions() @Test public void testGroupByNestedOuterExtractionFnOnFloatInner() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -11004,6 +11312,9 @@ public void testGroupByNestedOuterExtractionFnOnFloatInner() @Test public void testGroupByNestedDoubleTimeExtractionFnWithLongOutputTypes() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -11677,6 +11988,9 @@ public void testGroupByLimitPushDownPostAggNotSupported() @Test public void testEmptySubqueryWithLimitPushDown() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.EMPTY_INTERVAL) @@ -11710,6 +12024,9 @@ public void testEmptySubqueryWithLimitPushDown() @Test public void testSubqueryWithMultipleIntervalsInOuterQueryWithLimitPushDown() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -11781,6 +12098,9 @@ public void testSubqueryWithMultipleIntervalsInOuterQueryWithLimitPushDown() @Test public void testVirtualColumnFilterOnInnerQuery() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -11878,6 +12198,9 @@ public void testRejectForceLimitPushDownWithHaving() @Test public void testTypeConversionWithMergingChainedExecutionRunner() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // Cannot vectorize due to extraction dimension spec. cannotVectorize(); @@ -11915,6 +12238,9 @@ public void testTypeConversionWithMergingChainedExecutionRunner() @Test public void testGroupByNoMatchingPrefilter() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -12171,8 +12497,12 @@ public void testGroupByOnVirtualColumnTimeFloor() @Test public void testGroupByWithExpressionAggregator() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // expression agg not yet vectorized cannotVectorize(); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -12494,6 +12824,9 @@ public void testGroupByWithExpressionAggregatorWithComplexOnSubquery() @Test public void testGroupByWithExpressionAggregatorWithArrays() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // expression agg not yet vectorized cannotVectorize(); @@ -12781,6 +13114,9 @@ public void testGroupByWithExpressionAggregatorWithArrays() @Test public void testGroupByExpressionAggregatorArrayMultiValue() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + // expression agg not yet vectorized cannotVectorize(); @@ -13201,6 +13537,9 @@ public void testSummaryrowFilteredByHaving() @Test public void testSummaryrowForEmptySubqueryInput() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -13250,6 +13589,9 @@ public void testSummaryrowForEmptySubqueryInput() @Test public void testSummaryrowForEmptyInputByDay() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(originalRunner.getSegment().asStorageAdapter().isTimeOrdered()); + GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) 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/metadata/SegmentMetadataUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataUnionQueryTest.java index 9f37a47aae43..bddff9a6515e 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 f2c3a9ea18f5..cc6851603888 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 @@ -150,9 +150,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 f68104ff4815..5bc3bb28c798 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 @@ -50,6 +50,7 @@ import org.junit.Test; import javax.annotation.Nullable; +import java.util.Collections; import java.util.List; public class TopNMetricSpecOptimizationsTest @@ -320,6 +321,12 @@ public Metadata getMetadata() return null; } + @Override + public List getSortOrder() + { + return Collections.emptyList(); + } + @Override public Sequence makeCursors( @Nullable Filter filter, 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 79adf89fb86b..8db3e6306c4e 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 @@ -47,6 +47,7 @@ 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; @@ -96,6 +97,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; @@ -167,7 +169,8 @@ public static List>> queryRunners() defaultPool, new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + ), + true ) ); retVal.addAll( @@ -176,7 +179,8 @@ public static List>> queryRunners() customPool, new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + ), + true ) ); @@ -191,7 +195,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 +205,7 @@ public static List>> queryRunners() @SuppressWarnings("unused") public TopNQueryRunnerTest( - QueryRunner> runner, + TestQueryRunner> runner, boolean specializeGeneric1AggPooledTopN, boolean specializeGeneric2AggPooledTopN, boolean specializeHistorical1SimpleDoubleAggPooledTopN, @@ -819,6 +823,9 @@ public void testTopNOverHyperUniqueExpressionRounded() @Test public void testTopNOverFirstLastAggregator() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(runner.getSegment().asStorageAdapter().isTimeOrdered()); + TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.MONTH_GRAN) @@ -928,6 +935,9 @@ public void testTopNOverFirstLastAggregator() @Test public void testTopNOverFirstLastFloatAggregatorUsingDoubleColumn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(runner.getSegment().asStorageAdapter().isTimeOrdered()); + TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.MONTH_GRAN) @@ -1037,6 +1047,9 @@ public void testTopNOverFirstLastFloatAggregatorUsingDoubleColumn() @Test public void testTopNOverFirstLastFloatAggregatorUsingFloatColumn() { + // Granularity != ALL requires time-ordering. + Assume.assumeTrue(runner.getSegment().asStorageAdapter().isTimeOrdered()); + TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.MONTH_GRAN) 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..152bea782917 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -37,6 +37,7 @@ 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; @@ -152,7 +153,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 +179,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(Collections.singletonList("__time"), Lists.newArrayList(index.getSortOrder())); Assert.assertEquals(3, index.getColumnNames().size()); assertDimCompression(index, indexSpec.getDimensionCompression()); @@ -191,6 +195,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") + ) + ) + .setUseExplicitSegmentSortOrder(true) + .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(Arrays.asList("dim1", "dim2", "__time"), Lists.newArrayList(index.getSortOrder())); + 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") + ) + ) + .setUseExplicitSegmentSortOrder(true) + .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(Arrays.asList("dim1", "dim2", "__time"), Lists.newArrayList(index.getSortOrder())); + 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 +359,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(Collections.singletonList("__time"), Lists.newArrayList(index.getSortOrder())); Assert.assertEquals(3, index.getColumnNames().size()); assertDimCompression(index, indexSpec.getDimensionCompression()); @@ -253,6 +396,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(Collections.singletonList("__time"), Lists.newArrayList(index.getSortOrder())); Assert.assertEquals(3, index.getColumnNames().size()); assertDimCompression(index, indexSpec.getDimensionCompression()); @@ -263,7 +407,8 @@ public void testPersistWithSegmentMetadata() throws Exception IncrementalIndexTest.getDefaultCombiningAggregatorFactories(), null, Granularities.NONE, - Boolean.TRUE + Boolean.TRUE, + null ), index.getMetadata() ); @@ -616,7 +761,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 +855,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 +892,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"), + new StringDimensionSchema("dimB"), + new StringDimensionSchema("dimC") + ) + ) + .setUseExplicitSegmentSortOrder(true) + .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(ImmutableList.of("dimA", "dimB", "dimC", "__time"), Lists.newArrayList(merged.getSortOrder())); + + 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 +1048,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 +1183,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 +1365,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 +1520,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 +1551,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 +1928,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 +2164,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(Collections.singletonList("__time"), Lists.newArrayList(index.getSortOrder())); Assert.assertEquals(3, index.getColumnNames().size()); Assert.assertEquals(2, rowList.size()); @@ -1891,6 +2197,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(Collections.singletonList("__time"), Lists.newArrayList(index.getSortOrder())); Assert.assertEquals(3, index.getColumnNames().size()); adapter = new QueryableIndexIndexableAdapter(index); @@ -1922,6 +2229,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(Collections.singletonList("__time"), Lists.newArrayList(index.getSortOrder())); Assert.assertEquals(3, index.getColumnNames().size()); adapter = new QueryableIndexIndexableAdapter(index); @@ -1978,10 +2286,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(Collections.singletonList("__time"), Lists.newArrayList(index.getSortOrder())); Assert.assertEquals(3, index.getColumnNames().size()); assertDimCompression(index, indexSpec.getDimensionCompression()); @@ -2087,9 +2393,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 +2627,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 +2767,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()); 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..8625eeb43bb5 100644 --- a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java +++ b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java @@ -52,7 +52,8 @@ public void testSerde() throws Exception aggregators, null, Granularities.ALL, - Boolean.FALSE + Boolean.FALSE, + null ); Metadata other = jsonMapper.readValue( @@ -83,7 +84,8 @@ public void testMerge() aggs, new TimestampSpec("ds", "auto", null), Granularities.ALL, - Boolean.FALSE + Boolean.FALSE, + null ); final Metadata m2 = new Metadata( @@ -91,7 +93,8 @@ public void testMerge() aggs, new TimestampSpec("ds", "auto", null), Granularities.ALL, - Boolean.FALSE + Boolean.FALSE, + null ); final Metadata m3 = new Metadata( @@ -99,7 +102,8 @@ public void testMerge() aggs, new TimestampSpec("ds", "auto", null), Granularities.ALL, - Boolean.TRUE + Boolean.TRUE, + null ); final Metadata merged = new Metadata( @@ -109,7 +113,8 @@ public void testMerge() }, new TimestampSpec("ds", "auto", null), Granularities.ALL, - Boolean.FALSE + Boolean.FALSE, + null ); Assert.assertEquals(merged, Metadata.merge(ImmutableList.of(m1, m2), null)); @@ -119,7 +124,7 @@ 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, null); Assert.assertEquals(merged2, Metadata.merge(metadataToBeMerged, null)); @@ -128,7 +133,7 @@ public void testMerge() 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, null); Assert.assertEquals( merged3, @@ -140,6 +145,7 @@ public void testMerge() explicitAggs, new TimestampSpec("ds", "auto", null), Granularities.ALL, + null, null ); Assert.assertEquals( 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 c93abe864f16..b5b92a8fc638 100644 --- a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java @@ -31,6 +31,8 @@ import org.junit.Test; import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; public class TombstoneSegmentStorageAdapterTest @@ -128,6 +130,11 @@ public Metadata getMetadata() return null; } + @Override + public List getSortOrder() + { + return Collections.emptyList(); + } }; Assert.assertFalse(sa.isFromTombstone()); 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..18f9edbd3f6e 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) + .setUseExplicitSegmentSortOrder(true) + .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 09f78e066fb2..f3fc10050ce0 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; @@ -70,6 +74,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; @@ -92,23 +97,57 @@ 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"))) + .setUseExplicitSegmentSortOrder(true) + .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 @@ -255,6 +294,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(); @@ -536,14 +577,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); @@ -567,7 +613,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); @@ -578,8 +624,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); @@ -594,8 +640,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/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..a1be3ab37a0c 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.isUseExplicitSegmentSortOrder()) { + 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[true]. %s", + field, + i, + DimensionsSpec.PARAMETER_EXPLICIT_SORT_ORDER, + 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/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/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 78294fca0c4b..590fad659985 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.assertFalse(schema.getDimensionsSpec().isUseExplicitSegmentSortOrder()); + } + + @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[useExplicitSegmentSortOrder] is set to[true]. " + + 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")) + .setUseExplicitSegmentSortOrder(true) .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.assertTrue(schema.getDimensionsSpec().isUseExplicitSegmentSortOrder()); } @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 c58c686074d0..9cce3fe0d68b 100644 --- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -177,6 +177,12 @@ public Map getDimensionHandlers() throw new UnsupportedOperationException(); } + @Override + public List getSortOrder() + { + return Metadata.SORTED_BY_TIME_ONLY; + } + @Override public void close() { @@ -325,6 +331,12 @@ public DateTime getMaxIngestedEventTime() return null; } + @Override + public List getSortOrder() + { + return Metadata.SORTED_BY_TIME_ONLY; + } + @Override public Metadata getMetadata() { 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 bec5335f7c95..e4b8355de86d 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 @@ -1142,6 +1142,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() @@ -1404,7 +1409,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/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 84b437c20e08..319603bc631f 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 @@ -9725,6 +9725,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() { @@ -11179,6 +11225,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() From a07355b859db9cc9694620115290fad875609c3b Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 6 Aug 2024 16:40:41 -0700 Subject: [PATCH 02/25] Fixups from CI. --- docs/multi-stage-query/reference.md | 8 ++++---- .../histogram/ApproximateHistogramGroupByQueryTest.java | 2 +- .../java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java | 2 +- .../druid/segment/incremental/IncrementalIndex.java | 2 +- .../test/java/org/apache/druid/query/TestQueryRunner.java | 2 +- .../org/apache/druid/segment/IndexMergerTestBase.java | 6 +++--- 6 files changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index cea08971abab..37d30ab5430e 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -364,10 +364,10 @@ For more information about partitioning, see [Partitioning](concepts.md#partitio The `CLUSTERED BY ` clause is optional for [INSERT](#insert) and [REPLACE](#replace). It accepts a list of column names or expressions. -This column list is used for [secondary partitioning](../ingestion/concepts.md#secondary-partitioning) of segments -within a time chunk, and [sorting](../ingestion/concepts#sorting) of rows within a segment. For sorting purposes, Druid -implicitly prepends `__time` to the `CLUSTERED BY` column list, unless [`useExplicitSegmentSortOrder`](#context) -(an experimental feature; see [Sorting](../ingestion/partitioning#sorting) for details) is set to `true`. +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 [`useExplicitSegmentSortOrder`](#context) +(an experimental feature; see [Sorting](../ingestion/partitioning.md#sorting) for details) is set to `true`. For more information about clustering, see [Clustering](concepts.md#clustering). 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 c0d0c9b6d8ab..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, true)) { + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunnersToMerge(factory, false)) { final String testName = StringUtils.format( "config=%s, runner=%s", config.toString(), 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 627765056a27..a0eb35da3487 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 @@ -467,7 +467,7 @@ private static void validateSortOrderBeginsWithTimeIfRequired( } } else if (!rootCollation.getFieldCollations().isEmpty()) { int timePosition = -1; - for (int i = 0 ; i < rootCollation.getFieldCollations().size(); i++) { + 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)) { 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 73306f0fa31c..6aca2b63c1f1 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 @@ -953,7 +953,7 @@ public void loadDimensionIterable( { synchronized (dimensionDescs) { if (size() != 0) { - throw new ISE("Cannot load dimension order when existing index is not empty.", dimensionDescs.keySet()); + throw new ISE("Cannot load dimension order[%s] when existing index is not empty.", dimensionDescs.keySet()); } for (String dim : oldDimensionOrder) { // Skip __time; its position is solely based on configuration at index creation time. diff --git a/processing/src/test/java/org/apache/druid/query/TestQueryRunner.java b/processing/src/test/java/org/apache/druid/query/TestQueryRunner.java index d1c45ec66128..1a589d507010 100644 --- a/processing/src/test/java/org/apache/druid/query/TestQueryRunner.java +++ b/processing/src/test/java/org/apache/druid/query/TestQueryRunner.java @@ -25,7 +25,7 @@ /** * Wrapper around a {@link QueryRunner} that adds a name and allows retrieval of the underlying segment. Used by - * {@link QueryRunnerTestHelper#makeQueryRunners(QueryRunnerFactory)} and related methods. + * {@link QueryRunnerTestHelper#makeQueryRunners(QueryRunnerFactory, boolean)} and related methods. */ public class TestQueryRunner implements QueryRunner { 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 152bea782917..469f763bfef8 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -900,9 +900,9 @@ public void testMergePlainNonTimeOrdered() throws Exception DimensionsSpec.builder() .setDimensions( ImmutableList.of( - new StringDimensionSchema("dimA"), - new StringDimensionSchema("dimB"), - new StringDimensionSchema("dimC") + new StringDimensionSchema("dimA", null, useBitmapIndexes), + new StringDimensionSchema("dimB", null, useBitmapIndexes), + new StringDimensionSchema("dimC", null, useBitmapIndexes) ) ) .setUseExplicitSegmentSortOrder(true) From 985ecc9e8eba63b28cf23470d5e14e47f5f45a05 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 7 Aug 2024 15:20:23 -0700 Subject: [PATCH 03/25] Fixes. --- docs/ingestion/ingestion-spec.md | 2 +- docs/ingestion/partitioning.md | 8 ++++---- .../histogram/ApproximateHistogramTopNQueryTest.java | 3 ++- .../histogram/FixedBucketsHistogramTopNQueryTest.java | 2 +- .../query/aggregation/variance/VarianceTopNQueryTest.java | 2 +- .../org/apache/druid/query/topn/TopNQueryRunnerTest.java | 8 +++++++- .../org/apache/druid/query/topn/TopNUnionQueryTest.java | 2 +- 7 files changed, 17 insertions(+), 10 deletions(-) diff --git a/docs/ingestion/ingestion-spec.md b/docs/ingestion/ingestion-spec.md index 0db787bc9dd2..df03827d701e 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 | -| `useExplicitSegmentSortOrder` | When set to true, segments created by the ingestion job are sorted by `{dimensions[0], dimensions[1], ...}`, rather than `{__time, dimensions[0], dimensions[1], ...}`. This enables creation of segments that are sorted by something other than `__time`. To include `__time` in the sort order when this parameter is set, you must include a dimension named `__time` with type `long` explicitly in the `dimensions` list.

This is an experimental feature; see [Sorting](../partitioning#sorting) for details. | `false` | +| `useExplicitSegmentSortOrder` | When set to true, segments created by the ingestion job are sorted by `{dimensions[0], dimensions[1], ...}`, rather than `{__time, dimensions[0], dimensions[1], ...}`. This enables creation of segments that are sorted by something other than `__time`. To include `__time` in the sort order when this parameter is set, you must include a dimension named `__time` with type `long` explicitly in the `dimensions` list.

This is an experimental feature; see [Sorting](partitioning.md#sorting) for details. | `false` | #### Dimension objects diff --git a/docs/ingestion/partitioning.md b/docs/ingestion/partitioning.md index 20f8020e18a5..667c690e79c6 100644 --- a/docs/ingestion/partitioning.md +++ b/docs/ingestion/partitioning.md @@ -49,8 +49,8 @@ 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)| +|[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 @@ -81,8 +81,8 @@ The following table describes how to configure sorting. |Method|Configuration| |------|------------| |[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)| +|[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 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/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/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/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java index 8db3e6306c4e..133b00287187 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 @@ -154,7 +154,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<>( 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; From 4e69274bf1e98aa36f4e46f6af90a08f59b038e5 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 7 Aug 2024 15:46:47 -0700 Subject: [PATCH 04/25] Fix missing arg. --- .../org/apache/druid/query/topn/TopNQueryRunnerTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 133b00287187..ab5fa5bed277 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 @@ -131,7 +131,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) { @@ -176,7 +176,7 @@ public static List>> queryRunners(boolean in new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ), - true + includeNonTimeOrdered ) ); retVal.addAll( @@ -186,7 +186,7 @@ public static List>> queryRunners(boolean in new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ), - true + includeNonTimeOrdered ) ); From 57183b38dd65c97d6b68f9cc743006e429c9d88f Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 7 Aug 2024 16:48:38 -0700 Subject: [PATCH 05/25] Additional changes. --- docs/ingestion/partitioning.md | 2 +- .../common/task/AbstractBatchIndexTask.java | 3 +- .../org/apache/druid/segment/Metadata.java | 36 ++++++---- .../segment/incremental/IncrementalIndex.java | 4 +- .../IncrementalIndexStorageAdapter.java | 2 + .../apache/druid/segment/MetadataTest.java | 67 ++++++++++++++++++- 6 files changed, 93 insertions(+), 21 deletions(-) diff --git a/docs/ingestion/partitioning.md b/docs/ingestion/partitioning.md index 667c690e79c6..dac716bfee4a 100644 --- a/docs/ingestion/partitioning.md +++ b/docs/ingestion/partitioning.md @@ -65,7 +65,7 @@ The following table describes how to configure secondary partitioning. |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 recluster using [`REPLACE`](../multi-stage-query/concepts.md#replace) (with `CLUSTERED BY`) or [compaction](../data-management/compaction.md) after initial ingestion.| +|[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`| ## Sorting 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 43ca9e2511ed..8e8ba2282a48 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 @@ -624,7 +624,8 @@ public static Function, Set> addCompactionStateToS if (ingestionSpec.getDataSchema().getDimensionsSpec() == null) { dimensionsSpec = null; } else { - // We do not need to store dimensionExclusions and spatialDimensions since auto compaction does not support them + // 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() 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 f0e26a1fac72..7f7d250997f5 100644 --- a/processing/src/main/java/org/apache/druid/segment/Metadata.java +++ b/processing/src/main/java/org/apache/druid/segment/Metadata.java @@ -23,6 +23,7 @@ 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.aggregation.AggregatorFactory; import org.apache.druid.segment.column.ColumnHolder; @@ -264,30 +265,39 @@ public String toString() '}'; } - @Nullable - private static List mergeSortOrders(List> sortOrdersToMerge) + /** + * Merge {@link #getSortOrder()} from different metadatas. + * + * When an input sort order is null, we assume it is {@link Metadata#SORTED_BY_TIME_ONLY}, as this was the only + * sort order possible prior to the introduction of the "sortOrder" field. + */ + public static List mergeSortOrders(List> sortOrdersToMerge) { if (sortOrdersToMerge.isEmpty()) { - return null; - } - - if (sortOrdersToMerge.stream().anyMatch(Objects::isNull)) { - return null; + throw new IAE("sortOrdersToMerge is empty"); } final List mergedSortOrder = new ArrayList<>(); while (true) { + final int position = mergedSortOrder.size(); String column = null; - for (final List sortOrder : sortOrdersToMerge) { - if (mergedSortOrder.size() >= sortOrder.size()) { - return mergedSortOrder; + // Iterate through each sort order, check that the columns at "position" are all the same. If not, return + // the mergedSortOrder as-is. + for (List sortOrder : sortOrdersToMerge) { + if (sortOrder == null) { + // null sortOrder is treated as [__time]. + sortOrder = SORTED_BY_TIME_ONLY; } - if (column == null) { - column = sortOrder.get(mergedSortOrder.size()); - } else if (!column.equals(sortOrder.get(mergedSortOrder.size()))) { + if (position < sortOrder.size()) { + if (column == null) { + column = sortOrder.get(position); + } else if (!column.equals(sortOrder.get(position))) { + return mergedSortOrder; + } + } else { return mergedSortOrder; } } 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 6aca2b63c1f1..c1d8a43e3c10 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 @@ -25,7 +25,6 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.primitives.Ints; @@ -351,14 +350,13 @@ protected IncrementalIndex( } // Set metadata last, so dimensionOrder is populated - final List dimensionOrder = getDimensionOrder(); this.metadata = new Metadata( null, getCombiningAggregators(metrics), incrementalIndexSchema.getTimestampSpec(), this.gran, this.rollup, - ColumnHolder.TIME_COLUMN_NAME.equals(Iterables.getFirst(dimensionOrder, null)) ? null : dimensionOrder + getDimensionOrder() ); } 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 d4fc3e160f28..abd7b06154a7 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 @@ -306,6 +306,8 @@ public List getSortOrder() if (index.timePosition == 0) { return Metadata.SORTED_BY_TIME_ONLY; } else { + // 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. return Collections.emptyList(); } } 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 8625eeb43bb5..ae4a75077a43 100644 --- a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java +++ b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java @@ -31,10 +31,12 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; /** + * */ public class MetadataTest { @@ -43,7 +45,7 @@ public void testSerde() throws Exception { ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); - AggregatorFactory[] aggregators = new AggregatorFactory[] { + AggregatorFactory[] aggregators = new AggregatorFactory[]{ new LongSumAggregatorFactory("out", "in") }; @@ -76,7 +78,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( @@ -129,7 +131,7 @@ public void testMerge() 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") }; @@ -153,4 +155,63 @@ public void testMerge() Metadata.merge(ImmutableList.of(m3, m2), explicitAggs) ); } + + @Test + public void testMergeSortOrders() + { + Assert.assertThrows( + IllegalArgumentException.class, + () -> Metadata.mergeSortOrders(Collections.emptyList()) + ); + + Assert.assertEquals( + Metadata.SORTED_BY_TIME_ONLY, + Metadata.mergeSortOrders(Collections.singletonList(null)) + ); + + Assert.assertEquals( + Collections.emptyList(), + Metadata.mergeSortOrders(Arrays.asList(null, Arrays.asList("foo", "bar"))) + ); + + Assert.assertEquals( + Collections.emptyList(), + Metadata.mergeSortOrders(Arrays.asList(Arrays.asList("foo", "bar"), null)) + ); + + Assert.assertEquals( + Metadata.SORTED_BY_TIME_ONLY, + Metadata.mergeSortOrders(Arrays.asList(Arrays.asList("__time", "foo", "bar"), null)) + ); + + Assert.assertEquals( + Collections.emptyList(), + Metadata.mergeSortOrders( + Arrays.asList( + Arrays.asList("foo", "bar"), + Arrays.asList("bar", "foo") + ) + ) + ); + + Assert.assertEquals( + Collections.singletonList("bar"), + Metadata.mergeSortOrders( + Arrays.asList( + Arrays.asList("bar", "baz"), + Arrays.asList("bar", "foo") + ) + ) + ); + + Assert.assertEquals( + ImmutableList.of("bar", "foo"), + Metadata.mergeSortOrders( + Arrays.asList( + Arrays.asList("bar", "foo"), + Arrays.asList("bar", "foo") + ) + ) + ); + } } From 2a01e0f8855bb2ed21681afe9e76af5d6c84f599 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 7 Aug 2024 17:01:40 -0700 Subject: [PATCH 06/25] Fix logic. --- .../apache/druid/msq/exec/ControllerImpl.java | 16 +++++++++------- .../apache/druid/msq/exec/MSQReplaceTest.java | 14 +++++--------- .../msq/util/MSQTaskQueryMakerUtilsTest.java | 13 +++++-------- 3 files changed, 19 insertions(+), 24 deletions(-) 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 131d93b2c644..b9dc0ed9bbd2 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 @@ -1946,7 +1946,7 @@ private static DataSchema makeDataSchemaForIngestion( final boolean useExplicitSegmentSortOrder = MultiStageQueryContext.isUseExplicitSegmentSortOrder(querySpec.getQuery().context()); - final Pair, List> dimensionsAndAggregators = + final Pair> dimensionsAndAggregators = makeDimensionsAndAggregatorsForIngestion( querySignature, queryClusterBy, @@ -1960,10 +1960,7 @@ private static DataSchema makeDataSchemaForIngestion( return new DataSchema( destination.getDataSource(), new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null), - DimensionsSpec.builder() - .setDimensions(dimensionsAndAggregators.lhs) - .setUseExplicitSegmentSortOrder(useExplicitSegmentSortOrder) - .build(), + dimensionsAndAggregators.lhs, dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]), makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper), new TransformSpec(null, Collections.emptyList()) @@ -2149,7 +2146,7 @@ private static StringTuple makeStringTuple( return new StringTuple(array); } - private static Pair, List> makeDimensionsAndAggregatorsForIngestion( + private static Pair> makeDimensionsAndAggregatorsForIngestion( final RowSignature querySignature, final ClusterBy queryClusterBy, final List contextSegmentSortOrder, @@ -2273,12 +2270,17 @@ private static Pair, List> makeDimensio } } + 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.setUseExplicitSegmentSortOrder(false); + } else { + dimensionsSpecBuilder.setUseExplicitSegmentSortOrder(useExplicitSegmentSortOrder); } - return Pair.of(dimensions, aggregators); + return Pair.of(dimensionsSpecBuilder.setDimensions(dimensions).build(), aggregators); } /** 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 8af3089ec5da..dc7cafef0491 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 @@ -500,7 +500,7 @@ public void testReplaceOnFooWithAllClusteredByDimThenTimeError2(String contextNa @ParameterizedTest(name = "{index}:with context {0}") public void testReplaceOnFooWithAllClusteredByTimeThenDimExplicitSort(String contextName, Map context) { - // Tests [CLUSTERED BY dim1] with useExplicitSegmentSortOrder = true. + // Tests [CLUSTERED BY __time, dim1] with useExplicitSegmentSortOrder = true. RowSignature rowSignature = RowSignature.builder() .add("__time", ColumnType.LONG) .add("dim1", ColumnType.STRING) @@ -568,14 +568,10 @@ public void testReplaceOnFooWithAllClusteredByTimeThenDimExplicitSort(String con Collections.emptyList(), // For backwards-compatibility, compaction state is stored as if // useExplicitSegmentSortOrder = false. - DimensionsSpec.builder() - .setDimensions( - ImmutableList.of( - new StringDimensionSchema("dim1"), - new FloatDimensionSchema("m1") - ) - ) - .build(), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new FloatDimensionSchema("m1") + ), GranularityType.ALL, Intervals.ETERNITY ) 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 37a20cea9897..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 @@ -21,6 +21,7 @@ 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; @@ -36,18 +37,14 @@ public void testValidateContextSortOrderColumnsExist() 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.validateContextSortOrderColumnsExist(ImmutableList.of("c"), ImmutableSet.of("a", "b")) - ); - - Assert.assertThrows( - IllegalArgumentException.class, + DruidException.class, () -> MSQTaskQueryMakerUtils.validateContextSortOrderColumnsExist( - ImmutableList.of("b", "__time"), - ImmutableSet.of("__time", "a", "b") + ImmutableList.of("c"), + ImmutableSet.of("a", "b") ) ); } From ed0bbacb4109ec36210e0e42113b15a081bae4d0 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 8 Aug 2024 10:21:49 -0700 Subject: [PATCH 07/25] Fixes. --- .../apache/druid/msq/exec/MSQReplaceTest.java | 43 +++++-------------- .../variance/VarianceGroupByQueryTest.java | 40 +++++++++++------ .../druid/segment/IndexMergerTestBase.java | 8 ++-- 3 files changed, 43 insertions(+), 48 deletions(-) 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 dc7cafef0491..cbd0f542fd72 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 @@ -131,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 " @@ -225,10 +222,7 @@ public void testReplaceOnFooWithAllClusteredByDim(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' " 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/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java index 469f763bfef8..acc483de6b4a 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -408,7 +408,7 @@ public void testPersistWithSegmentMetadata() throws Exception null, Granularities.NONE, Boolean.TRUE, - null + Metadata.SORTED_BY_TIME_ONLY ), index.getMetadata() ); @@ -2164,7 +2164,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(Collections.singletonList("__time"), Lists.newArrayList(index.getSortOrder())); + Assert.assertEquals(ImmutableList.of("__time", "dim1", "dim2"), Lists.newArrayList(index.getSortOrder())); Assert.assertEquals(3, index.getColumnNames().size()); Assert.assertEquals(2, rowList.size()); @@ -2197,7 +2197,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(Collections.singletonList("__time"), Lists.newArrayList(index.getSortOrder())); + Assert.assertEquals(ImmutableList.of("__time", "dim1", "dim2"), Lists.newArrayList(index.getSortOrder())); Assert.assertEquals(3, index.getColumnNames().size()); adapter = new QueryableIndexIndexableAdapter(index); @@ -2229,7 +2229,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(Collections.singletonList("__time"), Lists.newArrayList(index.getSortOrder())); + Assert.assertEquals(ImmutableList.of("__time", "dim1", "dim2"), Lists.newArrayList(index.getSortOrder())); Assert.assertEquals(3, index.getColumnNames().size()); adapter = new QueryableIndexIndexableAdapter(index); From 4036b3f7690fc127087ee71e93610e7aab76046a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 8 Aug 2024 12:05:55 -0700 Subject: [PATCH 08/25] Fix test. --- .../java/org/apache/druid/segment/MetadataTest.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) 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 ae4a75077a43..9cccc7c36b02 100644 --- a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java +++ b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java @@ -116,7 +116,7 @@ public void testMerge() new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.FALSE, - null + Metadata.SORTED_BY_TIME_ONLY ); Assert.assertEquals(merged, Metadata.merge(ImmutableList.of(m1, m2), null)); @@ -126,7 +126,8 @@ public void testMerge() metadataToBeMerged.add(m2); metadataToBeMerged.add(null); - final Metadata merged2 = new Metadata(Collections.singletonMap("k", "v"), null, null, null, null, null); + final Metadata merged2 = + new Metadata(Collections.singletonMap("k", "v"), null, null, null, null, Metadata.SORTED_BY_TIME_ONLY); Assert.assertEquals(merged2, Metadata.merge(metadataToBeMerged, null)); @@ -135,7 +136,8 @@ public void testMerge() new DoubleMaxAggregatorFactory("x", "y") }; - final Metadata merged3 = new Metadata(Collections.singletonMap("k", "v"), explicitAggs, null, null, null, null); + final Metadata merged3 = + new Metadata(Collections.singletonMap("k", "v"), explicitAggs, null, null, null, Metadata.SORTED_BY_TIME_ONLY); Assert.assertEquals( merged3, @@ -148,7 +150,7 @@ public void testMerge() new TimestampSpec("ds", "auto", null), Granularities.ALL, null, - null + Metadata.SORTED_BY_TIME_ONLY ); Assert.assertEquals( merged4, From b9fdf2c18195db6db0533a8620df46ffbe86d598 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 13 Aug 2024 20:33:57 -0700 Subject: [PATCH 09/25] Adjust test. --- .../GroupByQueryRunnerFailureTest.java | 33 ++++++++++++++++--- 1 file changed, 29 insertions(+), 4 deletions(-) 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 9ff6f499cc7a..7b48363e079c 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() throws Exception + { + Assert.assertEquals( + "MERGE_BUFFER_POOL size, pre-test", + MERGE_BUFFER_POOL.maxSize(), + MERGE_BUFFER_POOL.getPoolSize() + ); + } + + @After + public void tearDown() throws Exception + { + Assert.assertEquals( + "MERGE_BUFFER_POOL size, post-test", + MERGE_BUFFER_POOL.maxSize(), + MERGE_BUFFER_POOL.getPoolSize() + ); + } + @AfterClass public static void teardownClass() { @@ -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) + ); } } From 6de8c254fbee4d95c704fea9e1bb195111d3b8bc Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 13 Aug 2024 21:23:03 -0700 Subject: [PATCH 10/25] Remove throws. --- .../druid/query/groupby/GroupByQueryRunnerFailureTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 7b48363e079c..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 @@ -130,7 +130,7 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( private QueryRunner runner; @Before - public void setUp() throws Exception + public void setUp() { Assert.assertEquals( "MERGE_BUFFER_POOL size, pre-test", @@ -140,7 +140,7 @@ public void setUp() throws Exception } @After - public void tearDown() throws Exception + public void tearDown() { Assert.assertEquals( "MERGE_BUFFER_POOL size, post-test", From 2ff791b092298d935759b835678da7db43dc33f7 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 19 Aug 2024 23:56:54 -0700 Subject: [PATCH 11/25] Fix styles. --- .../main/java/org/apache/druid/query/scan/ScanQueryEngine.java | 1 - .../query/timeboundary/TimeBoundaryQueryRunnerFactory.java | 2 +- .../segment/incremental/IncrementalIndexStorageAdapter.java | 1 - .../druid/segment/join/HashJoinSegmentStorageAdapter.java | 1 - 4 files changed, 1 insertion(+), 4 deletions(-) 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 8934b7cb0854..893480f54532 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -23,7 +23,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.apache.druid.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; 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 b4c3d9f56260..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 @@ -331,7 +331,7 @@ private static Pair getTimeBoundaryFullScanVectorized( 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++) { + for (int i = 0; i < cursor.getCurrentVectorSize(); i++) { final long timestamp = timeVector[i]; minTime = Math.min(minTime, timestamp); maxTime = Math.max(maxTime, timestamp); 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 ea0de97e9ee0..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,7 +32,6 @@ 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; 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 762bbccea542..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 @@ -263,7 +263,6 @@ public CursorHolder makeCursorHolder(CursorBuildSpec spec) baseFilter ); - if (joinFilterSplit.getBaseTableFilter().isPresent()) { cursorBuildSpecBuilder.setFilter(joinFilterSplit.getBaseTableFilter().get()); } From 7678a1680ad41c0637981e530ed58c4b2bad6e2f Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 19 Aug 2024 23:57:17 -0700 Subject: [PATCH 12/25] Fix javadocs. --- .../druid/query/groupby/epinephelinae/GroupByQueryEngine.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) 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 72d6d4c07ae0..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; @@ -78,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 From 4d21685ea5b487c521683c84af9067666f6826e3 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 20 Aug 2024 01:08:54 -0700 Subject: [PATCH 13/25] Cleanup. --- .../druid/query/scan/ScanQueryEngine.java | 17 ++++++++++++-- .../timeseries/TimeseriesQueryEngine.java | 3 --- .../org/apache/druid/segment/Cursors.java | 23 ------------------- 3 files changed, 15 insertions(+), 28 deletions(-) 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/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index 2d50dbcaff8a..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; @@ -102,9 +101,7 @@ 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; 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 986ce36010be..27c81c518447 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; @@ -53,27 +51,6 @@ public static boolean preferDescendingTimeOrdering(CursorBuildSpec buildSpec) return false; } - /** - * Require the first {@link OrderBy} of {@link CursorHolder#getOrdering()} is {@link ColumnHolder#TIME_COLUMN_NAME}. - * Throws {@link DruidException} if the order does not match expectations. - */ - public static void requireTimeOrdering(CursorHolder holder, Order expectedOrder) - { - if (holder.getTimeOrder() != expectedOrder) { - final String failureReason = StringUtils.format( - "Cannot order by[%s] with direction[%s] on cursor with order[%s].", - ColumnHolder.TIME_COLUMN_NAME, - expectedOrder, - holder.getOrdering() - ); - holder.close(); - - throw DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.UNSUPPORTED) - .build("%s", failureReason); - } - } - /** * Return the {@link Order} of the {@link ColumnHolder#TIME_COLUMN_NAME}, based on a * {@link CursorHolder#getOrdering()} or {@link Metadata#getOrdering()}. From 427402f531be3dd8d53770b7d7f371f3e7c12852 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 20 Aug 2024 03:53:39 -0700 Subject: [PATCH 14/25] Smoother handling of null ordering. --- .../indexing/common/task/CompactionTask.java | 14 ++--- .../org/apache/druid/segment/Cursors.java | 2 - .../org/apache/druid/segment/Metadata.java | 9 ++++ .../segment/QueryableIndexCursorHolder.java | 7 ++- .../QueryableIndexIndexableAdapter.java | 52 +++++++++---------- .../segment/incremental/IncrementalIndex.java | 3 ++ 6 files changed, 47 insertions(+), 40 deletions(-) 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 ba2eee255d4a..31bb9b5e6e76 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 @@ -76,7 +76,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.Metadata; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.AppendableIndexSpec; @@ -940,17 +939,14 @@ private void processDimensionsSpec(final QueryableIndex index) return; } - final Metadata metadata = index.getMetadata(); final List sortOrder = new ArrayList<>(); - if (metadata != null && metadata.getOrdering() != null) { - for (final OrderBy orderBy : metadata.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); + 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); } for (String dimension : Iterables.concat(sortOrder, index.getAvailableDimensions())) { 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 27c81c518447..36b37ce31469 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursors.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursors.java @@ -54,8 +54,6 @@ public static boolean preferDescendingTimeOrdering(CursorBuildSpec buildSpec) /** * Return the {@link Order} of the {@link ColumnHolder#TIME_COLUMN_NAME}, based on a * {@link CursorHolder#getOrdering()} or {@link Metadata#getOrdering()}. - * @param ordering - * @return */ public static Order getTimeOrdering(final List ordering) { 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 ea340aa9eed8..e21c673e2796 100644 --- a/processing/src/main/java/org/apache/druid/segment/Metadata.java +++ b/processing/src/main/java/org/apache/druid/segment/Metadata.java @@ -38,6 +38,7 @@ import java.util.concurrent.ConcurrentHashMap; /** + * */ @PublicApi public class Metadata @@ -107,6 +108,14 @@ 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) 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 7cf9e39eb370..305f9df9375f 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -90,14 +90,17 @@ public QueryableIndexCursorHolder( this.virtualColumns = cursorBuildSpec.getVirtualColumns(); this.aggregatorFactories = cursorBuildSpec.getAggregators(); this.filter = cursorBuildSpec.getFilter(); + + final List indexOrdering = index.getOrdering(); if (Cursors.preferDescendingTimeOrdering(cursorBuildSpec) - && Cursors.getTimeOrdering(index.getMetadata().getOrdering()) == Order.ASCENDING) { + && Cursors.getTimeOrdering(indexOrdering) == Order.ASCENDING) { this.ordering = Cursors.descendingTimeOrder(); this.descending = true; } else { - this.ordering = index.getMetadata().getOrdering(); + this.ordering = indexOrdering; this.descending = false; } + this.queryContext = cursorBuildSpec.getQueryContext(); this.vectorSize = cursorBuildSpec.getQueryContext().getVectorSize(); this.metrics = cursorBuildSpec.getQueryMetrics(); 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 c25ad8550a36..0060b276c4b2 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java @@ -75,38 +75,36 @@ public QueryableIndexIndexableAdapter(QueryableIndex input) } this.metadata = input.getMetadata(); - if (metadata != null && metadata.getOrdering() != null) { - 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 : metadata.getOrdering()) { - final String columnName = orderBy.getColumnName(); - - if (orderBy.getOrder() != Order.ASCENDING) { - throw DruidException.defensive("Order[%s] for column[%s] is not supported", orderBy.getOrder(), columnName); - } + final List inputOrdering = input.getOrdering(); - if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { - foundTimePosition = i; - break; - } else if (input.getDimensionHandlers().containsKey(columnName)) { - i++; - } + 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 (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 = metadata.getOrdering().size(); + if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { + foundTimePosition = i; + break; + } else if (input.getDimensionHandlers().containsKey(columnName)) { + i++; } + } + + if (foundTimePosition >= 0) { + this.timePositionForComparator = foundTimePosition; } else { - this.timePositionForComparator = 0; + // 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(); } } 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 3c41658dcde8..29bf146080dd 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 @@ -229,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; From af5d120c0116758fbad2d6f3af42b3712c3ce981 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 20 Aug 2024 12:02:14 -0700 Subject: [PATCH 15/25] Fix tests. --- .../org/apache/druid/sql/calcite/CalciteJoinQueryTest.java | 6 ++---- .../druid/sql/calcite/CalciteTimeBoundaryQueryTest.java | 3 +++ 2 files changed, 5 insertions(+), 4 deletions(-) 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/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); From c1c35df5f84f9a87dd42d8d7f6242e0aee49f4c4 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 21 Aug 2024 12:41:28 -0700 Subject: [PATCH 16/25] Missed a spot on the merge. --- .../segment/QueryableIndexCursorHolder.java | 219 ++++++++++++------ .../druid/segment/QueryableIndexSegment.java | 18 +- .../QueryableIndexTimeBoundaryInspector.java | 39 ++-- .../druid/query/topn/TopNQueryRunnerTest.java | 3 + 4 files changed, 182 insertions(+), 97 deletions(-) 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 305f9df9375f..d1a4ed517e3f 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -37,13 +37,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.Filters; import org.apache.druid.segment.historical.HistoricalCursor; import org.apache.druid.segment.vector.BitmapVectorOffset; import org.apache.druid.segment.vector.FilteredVectorOffset; @@ -58,6 +61,7 @@ import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; +import java.util.Arrays; 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; @@ -95,16 +98,23 @@ public QueryableIndexCursorHolder( if (Cursors.preferDescendingTimeOrdering(cursorBuildSpec) && Cursors.getTimeOrdering(indexOrdering) == Order.ASCENDING) { this.ordering = Cursors.descendingTimeOrder(); - this.descending = true; } else { this.ordering = indexOrdering; - this.descending = false; } 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 @@ -133,7 +143,7 @@ public boolean canVectorize() } // vector cursors can't iterate backwards yet - return !descending; + return Cursors.getTimeOrdering(ordering) != Order.DESCENDING; } @Override @@ -151,45 +161,55 @@ 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( @@ -205,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); @@ -228,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. @@ -240,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(); @@ -631,24 +653,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(), @@ -657,10 +676,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); @@ -672,49 +703,87 @@ 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); + /** + * TODO(gianm) javadoc + */ + @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; + } + + /** + * TODO(gianm) javadoc + */ + @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())) { + return Filters.and( + Arrays.asList( + new RangeFilter( + ColumnHolder.TIME_COLUMN_NAME, + ColumnType.LONG, + minDataTimestamp < interval.getStartMillis() ? interval.getStartMillis() : null, + maxDataTimestamp >= interval.getEndMillis() ? interval.getEndMillis() : null, + false, + true, + null + ), + filter + ) ); - 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); - } - return filterBundle; + } else { + return filter; } } } 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 7aa9cdf7287e..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,12 +89,17 @@ public T as(@Nonnull Class clazz) { final Function fn = AS_MAP.get(clazz); if (fn != null) { - return (T) fn.apply(this); - } else if (TimeBoundaryInspector.class.equals(clazz)) { - return (T) QueryableIndexTimeBoundaryInspector.create(index); - } else { - return Segment.super.as(clazz); + 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); } @SemanticCreator diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexTimeBoundaryInspector.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexTimeBoundaryInspector.java index a0c3024c2615..28292fc24391 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexTimeBoundaryInspector.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexTimeBoundaryInspector.java @@ -23,36 +23,35 @@ 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; -import javax.annotation.Nullable; - /** * {@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) + private QueryableIndexTimeBoundaryInspector(final QueryableIndex index, final boolean timeOrdered) { this.index = index; + this.timeOrdered = timeOrdered; } - @Nullable public static QueryableIndexTimeBoundaryInspector create(final QueryableIndex index) { - if (Cursors.getTimeOrdering(index.getOrdering()) == Order.ASCENDING) { - return new QueryableIndexTimeBoundaryInspector(index); - } else { - return null; - } + return new QueryableIndexTimeBoundaryInspector( + index, + Cursors.getTimeOrdering(index.getOrdering()) == Order.ASCENDING + ); } - @Nullable + @MonotonicNonNull private volatile DateTime minTime; - @Nullable + @MonotonicNonNull private volatile DateTime maxTime; @Override @@ -80,16 +79,22 @@ public DateTime getMaxTime() @Override public boolean isMinMaxExact() { - return true; + return timeOrdered; } 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)); + 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/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java index 9141333a35df..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 @@ -6268,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) From b3d20ba0932896827f44d4cdbb5be8e1aa839213 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 21 Aug 2024 14:39:25 -0700 Subject: [PATCH 17/25] Fixups. --- .../druid/segment/QueryableIndexCursorHolder.java | 10 ++++++++-- .../java/org/apache/druid/cli/DumpSegmentTest.java | 4 +++- 2 files changed, 11 insertions(+), 3 deletions(-) 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 d1a4ed517e3f..2670130955ad 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -346,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 @@ -706,7 +707,10 @@ public void close() throws IOException } /** - * TODO(gianm) javadoc + * 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( @@ -755,7 +759,9 @@ private static FilterBundle makeFilterBundle( } /** - * TODO(gianm) javadoc + * 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( 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); } From c118b8b6475c3f487a7db76c8762868e68a034bc Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 21 Aug 2024 14:41:34 -0700 Subject: [PATCH 18/25] Avoid needless Filters.and. --- .../segment/QueryableIndexCursorHolder.java | 31 ++++++++++--------- 1 file changed, 16 insertions(+), 15 deletions(-) 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 2670130955ad..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; @@ -46,7 +47,7 @@ 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.Filters; +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; @@ -61,7 +62,6 @@ import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; -import java.util.Arrays; import java.util.List; import java.util.concurrent.TimeUnit; @@ -774,20 +774,21 @@ private static Filter computeFilterWithIntervalIfNeeded( { if (timeOrder == Order.NONE && (minDataTimestamp < interval.getStartMillis() || maxDataTimestamp >= interval.getEndMillis())) { - return Filters.and( - Arrays.asList( - new RangeFilter( - ColumnHolder.TIME_COLUMN_NAME, - ColumnType.LONG, - minDataTimestamp < interval.getStartMillis() ? interval.getStartMillis() : null, - maxDataTimestamp >= interval.getEndMillis() ? interval.getEndMillis() : null, - false, - true, - null - ), - filter - ) + 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 (filter == null) { + return timeFilter; + } else { + return new AndFilter(ImmutableList.of(filter, timeFilter)); + } } else { return filter; } From 7b3271d4a85e22b662120db0d6497f5d3d12940b Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 21 Aug 2024 14:55:02 -0700 Subject: [PATCH 19/25] Add timeBoundaryInspector to test. --- .../apache/druid/query/CursorGranularizerTest.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) 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 2663bf0eab6a..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 @@ -157,7 +163,7 @@ public void testGranularizeFullScan() final Cursor cursor = cursorHolder.asCursor(); CursorGranularizer granularizer = CursorGranularizer.create( cursor, - null, + timeBoundaryInspector, Order.ASCENDING, Granularities.HOUR, adapter.getInterval() @@ -210,7 +216,7 @@ public void testGranularizeFullScanDescending() final Cursor cursor = cursorHolder.asCursor(); CursorGranularizer granularizer = CursorGranularizer.create( cursor, - null, + timeBoundaryInspector, Order.DESCENDING, Granularities.HOUR, adapter.getInterval() From 8468f3178b7ba91b4f6d8ccfafa4acd2838a7a34 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 22 Aug 2024 10:46:43 -0700 Subject: [PATCH 20/25] Fix tests. --- .../HashJoinSegmentStorageAdapterTest.java | 28 ------------------ .../segment/join/HashJoinSegmentTest.java | 29 +++++++++++++++++++ 2 files changed, 29 insertions(+), 28 deletions(-) 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); + } } From 1afda11691f96ae13cd796e69976a8bf9905335b Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 22 Aug 2024 10:57:50 -0700 Subject: [PATCH 21/25] Fix FrameStorageAdapterTest. --- .../segment/FrameStorageAdapterTest.java | 74 ++++++++++--------- 1 file changed, 41 insertions(+), 33 deletions(-) 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..2bc3531eeb6f 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; @@ -267,13 +269,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 +366,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); } /** From 63af1463b7e34b643543ebb4fe6ee022987381e2 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 22 Aug 2024 11:01:55 -0700 Subject: [PATCH 22/25] Fix various tests. --- .../frame/segment/FrameStorageAdapterTest.java | 12 ------------ .../druid/query/metadata/SegmentAnalyzerTest.java | 4 ++++ .../java/org/apache/druid/segment/MetadataTest.java | 4 ++-- .../druid/segment/UnnestStorageAdapterTest.java | 13 ------------- 4 files changed, 6 insertions(+), 27 deletions(-) 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 2bc3531eeb6f..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 @@ -207,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() { 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/segment/MetadataTest.java b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java index 7ab05bd163d6..b158480b22d6 100644 --- a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java +++ b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java @@ -199,7 +199,7 @@ public void testMergeOrderings() ); Assert.assertEquals( - Collections.singletonList("bar"), + Collections.singletonList(OrderBy.ascending("bar")), Metadata.mergeOrderings( Arrays.asList( makeOrderBy("bar", "baz"), @@ -209,7 +209,7 @@ public void testMergeOrderings() ); Assert.assertEquals( - ImmutableList.of("bar", "foo"), + ImmutableList.of(OrderBy.ascending("bar"), OrderBy.ascending("foo")), Metadata.mergeOrderings( Arrays.asList( makeOrderBy("bar", "foo"), 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() From 5451d0a7b314d6ebce4c77016a8054e693bab109 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 22 Aug 2024 14:04:11 -0700 Subject: [PATCH 23/25] Use forceSegmentSortByTime instead of useExplicitSegmentSortOrder. --- docs/ingestion/ingestion-spec.md | 2 +- docs/ingestion/partitioning.md | 8 +-- docs/multi-stage-query/reference.md | 9 +-- extensions-core/multi-stage-query/pom.xml | 10 ++-- .../apache/druid/msq/exec/ControllerImpl.java | 17 +++--- .../druid/msq/sql/MSQTaskSqlEngine.java | 12 ++-- .../msq/util/MultiStageQueryContext.java | 8 +-- .../apache/druid/msq/exec/MSQReplaceTest.java | 29 ++++----- .../common/task/AbstractBatchIndexTask.java | 10 ++-- .../indexing/common/task/CompactionTask.java | 4 +- .../common/task/CompactionTaskRunTest.java | 4 +- .../druid/data/input/impl/DimensionsSpec.java | 59 ++++++++++++------- .../segment/incremental/IncrementalIndex.java | 5 +- .../segment/virtual/ExpressionSelectors.java | 4 +- .../druid/segment/IndexMergerTestBase.java | 6 +- .../IncrementalIndexRowCompTest.java | 2 +- .../IncrementalIndexStorageAdapterTest.java | 2 +- .../druid/segment/indexing/DataSchema.java | 6 +- .../segment/indexing/DataSchemaTest.java | 8 +-- 19 files changed, 114 insertions(+), 91 deletions(-) diff --git a/docs/ingestion/ingestion-spec.md b/docs/ingestion/ingestion-spec.md index df03827d701e..866fd9f7b8d4 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 | -| `useExplicitSegmentSortOrder` | When set to true, segments created by the ingestion job are sorted by `{dimensions[0], dimensions[1], ...}`, rather than `{__time, dimensions[0], dimensions[1], ...}`. This enables creation of segments that are sorted by something other than `__time`. To include `__time` in the sort order when this parameter is set, you must include a dimension named `__time` with type `long` explicitly in the `dimensions` list.

This is an experimental feature; see [Sorting](partitioning.md#sorting) for details. | `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. | `false` | #### Dimension objects diff --git a/docs/ingestion/partitioning.md b/docs/ingestion/partitioning.md index dac716bfee4a..d1501f5fe0ff 100644 --- a/docs/ingestion/partitioning.md +++ b/docs/ingestion/partitioning.md @@ -86,13 +86,13 @@ The following table describes how to configure sorting. :::info Druid implicitly sorts rows within a segment by `__time` first before any `dimensions` or `CLUSTERED BY` fields, unless -you set `useExplicitSegmentSortOrder` to `true` in your +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 `useExplicitSegmentSortOrder` 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: +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. diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index a2ef31d0055c..c4345fedf81b 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -366,8 +366,9 @@ 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 [`useExplicitSegmentSortOrder`](#context) -(an experimental feature; see [Sorting](../ingestion/partitioning.md#sorting) for details) is set to `true`. +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). @@ -401,8 +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 | -| `useExplicitSegmentSortOrder` | INSERT or REPLACE

When set to true, `CLUSTERED BY` and `segmentSortOrder` are permitted to include `__time` in some position other than the first. Additionally, `CLUSTERED BY x` will sort segments by `x`, rather than `{__time, x}`. This enables creation of segments that are sorted by something other than `__time`.

This is an experimental feature; see [Sorting](../ingestion/partitioning#sorting) for details. | `false` | +| `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/extensions-core/multi-stage-query/pom.xml b/extensions-core/multi-stage-query/pom.xml index 22e85f0b0fcb..a661e68154c9 100644 --- a/extensions-core/multi-stage-query/pom.xml +++ b/extensions-core/multi-stage-query/pom.xml @@ -218,11 +218,11 @@ quidem test - - org.apache.calcite.avatica - avatica-core - test - + + + + + org.apache.calcite calcite-testkit 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 869f547cae6d..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 @@ -1943,15 +1943,15 @@ private static DataSchema makeDataSchemaForIngestion( { final DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); final boolean isRollupQuery = isRollupQuery(querySpec.getQuery()); - final boolean useExplicitSegmentSortOrder = - MultiStageQueryContext.isUseExplicitSegmentSortOrder(querySpec.getQuery().context()); + final boolean forceSegmentSortByTime = + MultiStageQueryContext.isForceSegmentSortByTime(querySpec.getQuery().context()); final Pair> dimensionsAndAggregators = makeDimensionsAndAggregatorsForIngestion( querySignature, queryClusterBy, destination.getSegmentSortOrder(), - useExplicitSegmentSortOrder, + forceSegmentSortByTime, columnMappings, isRollupQuery, querySpec.getQuery(), @@ -2172,7 +2172,7 @@ private static Pair> makeDimensionsAndAg final RowSignature querySignature, final ClusterBy queryClusterBy, final List contextSegmentSortOrder, - final boolean useExplicitSegmentSortOrder, + final boolean forceSegmentSortByTime, final ColumnMappings columnMappings, final boolean isRollupQuery, final Query query, @@ -2204,8 +2204,8 @@ private static Pair> makeDimensionsAndAg // Start with segmentSortOrder. final Set outputColumnsInOrder = new LinkedHashSet<>(contextSegmentSortOrder); - // Then __time, if it's an output column and useExplicitSegmentSortOrder is not set. - if (columnMappings.hasOutputColumn(ColumnHolder.TIME_COLUMN_NAME) && !useExplicitSegmentSortOrder) { + // 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); } @@ -2295,9 +2295,10 @@ private static Pair> makeDimensionsAndAg 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.setUseExplicitSegmentSortOrder(false); + dimensionsSpecBuilder.setForceSegmentSortByTime(null); } else { - dimensionsSpecBuilder.setUseExplicitSegmentSortOrder(useExplicitSegmentSortOrder); + // Store explicit forceSegmentSortByTime only if false, for compatibility with legacy dimensionSpecs. + dimensionsSpecBuilder.setForceSegmentSortByTime(forceSegmentSortByTime ? null : false); } return Pair.of(dimensionsSpecBuilder.setDimensions(dimensions).build(), aggregators); 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 a0eb35da3487..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 @@ -428,7 +428,7 @@ private static void validateTypeChanges( /** * 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_EXPLICIT_SORT_ORDER} + * begins with {@link ColumnHolder#TIME_COLUMN_NAME}, unless {@link MultiStageQueryContext#CTX_FORCE_TIME_SORT} * is set. * * @param fieldMappings field mappings from {@link #validateInsert(RelRoot, Table, PlannerContext)} @@ -446,7 +446,7 @@ private static void validateSortOrderBeginsWithTimeIfRequired( final QueryContext context = plannerContext.queryContext(); - if (MultiStageQueryContext.isUseExplicitSegmentSortOrder(context)) { + if (!MultiStageQueryContext.isForceSegmentSortByTime(context)) { // Any sort order is allowed. Skip check. return; } @@ -458,10 +458,10 @@ private static void validateSortOrderBeginsWithTimeIfRequired( if (!timeIsFirst) { throw InvalidSqlInput.exception( - "Context parameter[%s] must start with[%s] unless context parameter[%s] is set to[true]. %s", + "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_EXPLICIT_SORT_ORDER, + MultiStageQueryContext.CTX_FORCE_TIME_SORT, DimensionsSpec.WARNING_NON_TIME_SORT_ORDER ); } @@ -479,10 +479,10 @@ private static void validateSortOrderBeginsWithTimeIfRequired( if (timePosition > 0) { throw InvalidSqlInput.exception( "Sort order (CLUSTERED BY) cannot include[%s] in position[%d] unless context parameter[%s] " - + "is set to[true]. %s", + + "is set to[false]. %s", ColumnHolder.TIME_COLUMN_NAME, timePosition, - MultiStageQueryContext.CTX_EXPLICIT_SORT_ORDER, + MultiStageQueryContext.CTX_FORCE_TIME_SORT, DimensionsSpec.WARNING_NON_TIME_SORT_ORDER ); } 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 34e9cac42e11..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 @@ -168,8 +168,8 @@ 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_EXPLICIT_SORT_ORDER = DimensionsSpec.PARAMETER_EXPLICIT_SORT_ORDER; - private static final boolean DEFAULT_EXPLICIT_SORT_ORDER = false; + 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"; @@ -365,9 +365,9 @@ public static ArrayIngestMode getArrayIngestMode(final QueryContext queryContext return queryContext.getEnum(CTX_ARRAY_INGEST_MODE, ArrayIngestMode.class, DEFAULT_ARRAY_INGEST_MODE); } - public static boolean isUseExplicitSegmentSortOrder(final QueryContext queryContext) + public static boolean isForceSegmentSortByTime(final QueryContext queryContext) { - return queryContext.getBoolean(CTX_EXPLICIT_SORT_ORDER, DEFAULT_EXPLICIT_SORT_ORDER); + return queryContext.getBoolean(CTX_FORCE_TIME_SORT, DEFAULT_FORCE_TIME_SORT); } public static Set getColumnsExcludedFromTypeVerification(final QueryContext queryContext) 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 cbd0f542fd72..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 @@ -197,7 +197,7 @@ public void testReplaceOnFooWithAll(String contextName, Map cont @ParameterizedTest(name = "{index}:with context {0}") public void testReplaceOnFooWithAllClusteredByDim(String contextName, Map context) { - // Tests [CLUSTERED BY dim1] with the default useExplicitSegmentSortOrder (false). In this case, + // 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) @@ -299,7 +299,7 @@ public void testReplaceOnFooWithAllClusteredByDimExplicitSort(String contextName .build(); Map queryContext = new HashMap<>(context); - queryContext.put(DimensionsSpec.PARAMETER_EXPLICIT_SORT_ORDER, true); + queryContext.put(DimensionsSpec.PARAMETER_FORCE_TIME_SORT, false); Mockito.doCallRealMethod() .doReturn(ImmutableSet.of(existingDataSegment0, existingDataSegment1)) @@ -348,7 +348,7 @@ public void testReplaceOnFooWithAllClusteredByDimExplicitSort(String contextName new FloatDimensionSchema("m1") ) ) - .setUseExplicitSegmentSortOrder(true) + .setForceSegmentSortByTime(false) .build(), GranularityType.ALL, Intervals.ETERNITY @@ -361,8 +361,9 @@ public void testReplaceOnFooWithAllClusteredByDimExplicitSort(String contextName @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 useExplicitSegmentSortOrder = true. - // (Same expectations as the prior test, testReplaceOnFooWithAllClusteredByDimExplicitSort.) + // 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) @@ -384,7 +385,7 @@ public void testReplaceOnFooWithAllClusteredByDimThenTimeExplicitSort(String con .build(); Map queryContext = new HashMap<>(context); - queryContext.put(DimensionsSpec.PARAMETER_EXPLICIT_SORT_ORDER, true); + queryContext.put(DimensionsSpec.PARAMETER_FORCE_TIME_SORT, false); Mockito.doCallRealMethod() .doReturn(ImmutableSet.of(existingDataSegment0, existingDataSegment1)) @@ -433,7 +434,7 @@ public void testReplaceOnFooWithAllClusteredByDimThenTimeExplicitSort(String con new FloatDimensionSchema("m1") ) ) - .setUseExplicitSegmentSortOrder(true) + .setForceSegmentSortByTime(false) .build(), GranularityType.ALL, Intervals.ETERNITY @@ -446,7 +447,7 @@ public void testReplaceOnFooWithAllClusteredByDimThenTimeExplicitSort(String con @ParameterizedTest(name = "{index}:with context {0}") public void testReplaceOnFooWithAllClusteredByDimThenTimeError(String contextName, Map context) { - // Tests that [CLUSTERED BY dim1, __time] is an error when useExplicitSegmentSortOrder = false (the default). + // 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 " @@ -456,7 +457,7 @@ public void testReplaceOnFooWithAllClusteredByDimThenTimeError(String contextNam .setQueryContext(context) .setExpectedValidationErrorMatcher(invalidSqlContains( "Sort order (CLUSTERED BY) cannot include[__time] in position[1] unless context " - + "parameter[useExplicitSegmentSortOrder] is set to[true]." + + "parameter[forceSegmentSortByTime] is set to[false]." )) .verifyPlanningErrors(); } @@ -466,7 +467,7 @@ public void testReplaceOnFooWithAllClusteredByDimThenTimeError(String contextNam public void testReplaceOnFooWithAllClusteredByDimThenTimeError2(String contextName, Map context) { // Tests that setting segmentSortOrder = [dim1, __time] is an error when - // useExplicitSegmentSortOrder = false (the default). + // forceSegmentSortByTime = false (the default). Map queryContext = new HashMap<>(context); queryContext.put(MultiStageQueryContext.CTX_SORT_ORDER, "dim1, __time"); @@ -479,7 +480,7 @@ public void testReplaceOnFooWithAllClusteredByDimThenTimeError2(String contextNa .setQueryContext(queryContext) .setExpectedValidationErrorMatcher(invalidSqlContains( "Context parameter[segmentSortOrder] must start with[__time] unless context " - + "parameter[useExplicitSegmentSortOrder] is set to[true]." + + "parameter[forceSegmentSortByTime] is set to[false]." )) .verifyPlanningErrors(); } @@ -488,7 +489,7 @@ public void testReplaceOnFooWithAllClusteredByDimThenTimeError2(String contextNa @ParameterizedTest(name = "{index}:with context {0}") public void testReplaceOnFooWithAllClusteredByTimeThenDimExplicitSort(String contextName, Map context) { - // Tests [CLUSTERED BY __time, dim1] with useExplicitSegmentSortOrder = true. + // Tests [CLUSTERED BY __time, dim1] with forceSegmentSortByTime = false. RowSignature rowSignature = RowSignature.builder() .add("__time", ColumnType.LONG) .add("dim1", ColumnType.STRING) @@ -510,7 +511,7 @@ public void testReplaceOnFooWithAllClusteredByTimeThenDimExplicitSort(String con .build(); Map queryContext = new HashMap<>(context); - queryContext.put(DimensionsSpec.PARAMETER_EXPLICIT_SORT_ORDER, true); + queryContext.put(DimensionsSpec.PARAMETER_FORCE_TIME_SORT, false); Mockito.doCallRealMethod() .doReturn(ImmutableSet.of(existingDataSegment0, existingDataSegment1)) @@ -552,7 +553,7 @@ public void testReplaceOnFooWithAllClusteredByTimeThenDimExplicitSort(String con context, Collections.emptyList(), // For backwards-compatibility, compaction state is stored as if - // useExplicitSegmentSortOrder = false. + // forceSegmentSortByTime = true. ImmutableList.of( new StringDimensionSchema("dim1"), new FloatDimensionSchema("m1") 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 8e8ba2282a48..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 @@ -627,11 +627,11 @@ public static Function, Set> addCompactionStateToS // 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()) - .setUseExplicitSegmentSortOrder(inputDimensionsSpec.isUseExplicitSegmentSortOrder()) - .build(); + 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()) 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 31bb9b5e6e76..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 @@ -856,9 +856,11 @@ public DimensionsSpec getDimensionsSpec() .filter(Objects::nonNull) .collect(Collectors.toList()); + // Store forceSegmentSortByTime only if false, for compatibility with legacy compaction states. + final Boolean forceSegmentSortByTime = includeTimeAsDimension ? false : null; return DimensionsSpec.builder() .setDimensions(dimensionSchemas) - .setUseExplicitSegmentSortOrder(includeTimeAsDimension) + .setForceSegmentSortByTime(forceSegmentSortByTime) .build(); } 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 f22cf314d3fb..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 @@ -1891,7 +1891,7 @@ public void testRunWithAutoCastDimensionsSortByDimension() throws Exception new AutoTypeColumnSchema("dim", null), new AutoTypeColumnSchema("y", ColumnType.LONG) )) - .setUseExplicitSegmentSortOrder(true) + .setForceSegmentSortByTime(false) .build(), "|", Arrays.asList("ts", "dim", "x", "y", "val"), @@ -1942,7 +1942,7 @@ public void testRunWithAutoCastDimensionsSortByDimension() throws Exception new AutoTypeColumnSchema("dim", null), new AutoTypeColumnSchema("y", ColumnType.LONG) )) - .setUseExplicitSegmentSortOrder(true) + .setForceSegmentSortByTime(false) .build(), expectedLongSumMetric ), 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 8172352269c4..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 @@ -51,10 +51,10 @@ 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_EXPLICIT_SORT_ORDER = "useExplicitSegmentSortOrder"; + public static final String PARAMETER_FORCE_TIME_SORT = "forceSegmentSortByTime"; /** - * Warning about non-time ordering to include in error messages when {@link #PARAMETER_EXPLICIT_SORT_ORDER} is + * 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( @@ -64,11 +64,13 @@ public class DimensionsSpec 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 useExplicitSegmentSortOrder; + private final Boolean forceSegmentSortByTime; private final boolean useSchemaDiscovery; @@ -101,7 +103,7 @@ public static Builder builder() public DimensionsSpec(List dimensions) { - this(dimensions, null, null, false, null, false); + this(dimensions, null, null, false, null, null); } @JsonCreator @@ -111,7 +113,7 @@ private DimensionsSpec( @Deprecated @JsonProperty("spatialDimensions") List spatialDimensions, @JsonProperty("includeAllDimensions") boolean includeAllDimensions, @JsonProperty("useSchemaDiscovery") Boolean useSchemaDiscovery, - @JsonProperty(PARAMETER_EXPLICIT_SORT_ORDER) boolean useExplicitSegmentSortOrder + @JsonProperty(PARAMETER_FORCE_TIME_SORT) Boolean forceSegmentSortByTime ) { this.dimensions = dimensions == null @@ -142,7 +144,7 @@ private DimensionsSpec( this.includeAllDimensions = includeAllDimensions; this.useSchemaDiscovery = useSchemaDiscovery != null && useSchemaDiscovery; - this.useExplicitSegmentSortOrder = useExplicitSegmentSortOrder; + this.forceSegmentSortByTime = forceSegmentSortByTime; } @JsonProperty @@ -169,11 +171,24 @@ public boolean useSchemaDiscovery() return useSchemaDiscovery; } - @JsonProperty(PARAMETER_EXPLICIT_SORT_ORDER) - @JsonInclude(JsonInclude.Include.NON_DEFAULT) - public boolean isUseExplicitSegmentSortOrder() + @JsonProperty(PARAMETER_FORCE_TIME_SORT) + @JsonInclude(JsonInclude.Include.NON_NULL) + public Boolean isForceSegmentSortByTimeConfigured() { - return useExplicitSegmentSortOrder; + 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 @@ -239,7 +254,7 @@ public DimensionsSpec withDimensions(List dims) null, includeAllDimensions, useSchemaDiscovery, - useExplicitSegmentSortOrder + forceSegmentSortByTime ); } @@ -251,7 +266,7 @@ public DimensionsSpec withDimensionExclusions(Set dimExs) null, includeAllDimensions, useSchemaDiscovery, - useExplicitSegmentSortOrder + forceSegmentSortByTime ); } @@ -264,7 +279,7 @@ public DimensionsSpec withSpatialDimensions(List spatial spatials, includeAllDimensions, useSchemaDiscovery, - useExplicitSegmentSortOrder + forceSegmentSortByTime ); } @@ -304,11 +319,11 @@ public boolean equals(Object o) } DimensionsSpec that = (DimensionsSpec) o; return includeAllDimensions == that.includeAllDimensions - && useExplicitSegmentSortOrder == that.useExplicitSegmentSortOrder && useSchemaDiscovery == that.useSchemaDiscovery && Objects.equals(dimensions, that.dimensions) && Objects.equals(dimensionExclusions, that.dimensionExclusions) - && Objects.equals(dimensionSchemaMap, that.dimensionSchemaMap); + && Objects.equals(dimensionSchemaMap, that.dimensionSchemaMap) + && Objects.equals(forceSegmentSortByTime, that.forceSegmentSortByTime); } @Override @@ -319,7 +334,7 @@ public int hashCode() dimensionExclusions, dimensionSchemaMap, includeAllDimensions, - useExplicitSegmentSortOrder, + forceSegmentSortByTime, useSchemaDiscovery ); } @@ -332,7 +347,9 @@ public String toString() ", dimensionExclusions=" + dimensionExclusions + ", includeAllDimensions=" + includeAllDimensions + ", useSchemaDiscovery=" + useSchemaDiscovery + - (useExplicitSegmentSortOrder ? ", useExplicitSegmentSortOrder=" + useExplicitSegmentSortOrder : "") + + (forceSegmentSortByTime != null + ? ", forceSegmentSortByTime=" + forceSegmentSortByTime + : "") + '}'; } @@ -343,7 +360,7 @@ public static final class Builder private List spatialDimensions; private boolean includeAllDimensions; private boolean useSchemaDiscovery; - private boolean useExplicitSegmentSortOrder; + private Boolean forceSegmentSortByTime; public Builder setDimensions(List dimensions) { @@ -382,9 +399,9 @@ public Builder useSchemaDiscovery(boolean useSchemaDiscovery) return this; } - public Builder setUseExplicitSegmentSortOrder(boolean useExplicitSegmentSortOrder) + public Builder setForceSegmentSortByTime(Boolean forceSegmentSortByTime) { - this.useExplicitSegmentSortOrder = useExplicitSegmentSortOrder; + this.forceSegmentSortByTime = forceSegmentSortByTime; return this; } @@ -396,7 +413,7 @@ public DimensionsSpec build() spatialDimensions, includeAllDimensions, useSchemaDiscovery, - useExplicitSegmentSortOrder + forceSegmentSortByTime ); } } 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 29bf146080dd..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 @@ -336,8 +336,9 @@ protected IncrementalIndex( } if (foundTimePosition == -1) { - // __time not found: that means it either goes at the end, or the beginning, based on useExplicitSegmentSortOrder. - this.timePosition = dimensionsSpec.isUseExplicitSegmentSortOrder() ? dimensionDescsList.size() : 0; + // __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; } 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 351f3032d028..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 @@ -206,8 +206,8 @@ public static ColumnValueSelector makeExprEvalSelector( 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 - // useExplicitSegmentSortOrder: true, 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. + // 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), 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 d00953329b21..1f46534266f4 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -214,7 +214,7 @@ public void testPersistPlainNonTimeOrdered() throws Exception new LongDimensionSchema("__time") ) ) - .setUseExplicitSegmentSortOrder(true) + .setForceSegmentSortByTime(false) .build() ) .withMetrics(new CountAggregatorFactory("count")) @@ -284,7 +284,7 @@ public void testPersistRollupNonTimeOrdered() throws Exception new LongDimensionSchema("__time") ) ) - .setUseExplicitSegmentSortOrder(true) + .setForceSegmentSortByTime(false) .build() ) .withMetrics(new CountAggregatorFactory("count")) @@ -906,7 +906,7 @@ public void testMergePlainNonTimeOrdered() throws Exception new StringDimensionSchema("dimC", null, useBitmapIndexes) ) ) - .setUseExplicitSegmentSortOrder(true) + .setForceSegmentSortByTime(false) .build() ) .withMetrics(new CountAggregatorFactory("count")) 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 18f9edbd3f6e..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 @@ -123,7 +123,7 @@ public void testSortByDim() throws IOException DimensionsSpec.builder() .setDimensions(DimensionsSpec.getDefaultSchemas(Collections.singletonList("joe"))) .useSchemaDiscovery(true) - .setUseExplicitSegmentSortOrder(true) + .setForceSegmentSortByTime(false) .build() ) .withMetrics(new CountAggregatorFactory("cnt")) 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 61b5677cbcea..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 @@ -122,7 +122,7 @@ public IncrementalIndexStorageAdapterTest(String indexType, boolean sortByDim) t dimensionsSpec = DimensionsSpec.builder() .setDimensions(Collections.singletonList(new StringDimensionSchema("billy"))) - .setUseExplicitSegmentSortOrder(true) + .setForceSegmentSortByTime(false) .setIncludeAllDimensions(true) .build(); } else { 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 a1be3ab37a0c..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 @@ -234,16 +234,16 @@ private static Set computeAndValidateOutputFieldNames( } if (ColumnHolder.TIME_COLUMN_NAME.equals(field)) { - if (i > 0 && !dimensionsSpec.isUseExplicitSegmentSortOrder()) { + 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[true]. %s", + + "parameter[%s] is set to[false]. %s", field, i, - DimensionsSpec.PARAMETER_EXPLICIT_SORT_ORDER, + DimensionsSpec.PARAMETER_FORCE_TIME_SORT, DimensionsSpec.WARNING_NON_TIME_SORT_ORDER ); } else if (!dimSchema.getColumnType().is(ValueType.LONG)) { 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 590fad659985..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 @@ -281,7 +281,7 @@ public void testOverlapTimeAndDimPositionZero() schema.getDimensionsSpec().getDimensionNames() ); - Assert.assertFalse(schema.getDimensionsSpec().isUseExplicitSegmentSortOrder()); + Assert.assertTrue(schema.getDimensionsSpec().isForceSegmentSortByTime()); } @Test @@ -317,7 +317,7 @@ public void testOverlapTimeAndDimPositionOne() expectedException.expect(DruidException.class); expectedException.expectMessage( "Encountered dimension[__time] at position[1]. This is only supported when the dimensionsSpec " - + "parameter[useExplicitSegmentSortOrder] is set to[true]. " + + "parameter[forceSegmentSortByTime] is set to[false]. " + DimensionsSpec.WARNING_NON_TIME_SORT_ORDER ); @@ -357,7 +357,7 @@ public void testOverlapTimeAndDimPositionOne_withExplicitSortOrder() ) ) .setDimensionExclusions(ImmutableList.of("dimC")) - .setUseExplicitSegmentSortOrder(true) + .setForceSegmentSortByTime(false) .build(), null, new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), @@ -371,7 +371,7 @@ public void testOverlapTimeAndDimPositionOne_withExplicitSortOrder() schema.getDimensionsSpec().getDimensionNames() ); - Assert.assertTrue(schema.getDimensionsSpec().isUseExplicitSegmentSortOrder()); + Assert.assertFalse(schema.getDimensionsSpec().isForceSegmentSortByTime()); } @Test From a9d6e0c8921ceb913108af0a72ec256637d48d10 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 22 Aug 2024 14:26:19 -0700 Subject: [PATCH 24/25] Pom fix. --- extensions-core/multi-stage-query/pom.xml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/extensions-core/multi-stage-query/pom.xml b/extensions-core/multi-stage-query/pom.xml index a661e68154c9..22e85f0b0fcb 100644 --- a/extensions-core/multi-stage-query/pom.xml +++ b/extensions-core/multi-stage-query/pom.xml @@ -218,11 +218,11 @@ quidem test - - - - - + + org.apache.calcite.avatica + avatica-core + test + org.apache.calcite calcite-testkit From 8a3788c98afd84aebfb2bd528958674f75a16b64 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 22 Aug 2024 15:14:08 -0700 Subject: [PATCH 25/25] Fix doc. --- docs/ingestion/ingestion-spec.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ingestion/ingestion-spec.md b/docs/ingestion/ingestion-spec.md index 866fd9f7b8d4..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. | `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