diff --git a/docs/configuration/index.md b/docs/configuration/index.md
index c5988993e424..9b6fba9ee632 100644
--- a/docs/configuration/index.md
+++ b/docs/configuration/index.md
@@ -1862,7 +1862,7 @@ This strategy can be enabled by setting `druid.query.scheduler.laning.strategy=h
##### Guardrails for materialization of subqueries
Druid stores the subquery rows in temporary tables that live in the Java heap. It is a good practice to avoid large subqueries in Druid.
-Therefore there are guardrails that are built in Druid to prevent the queries from generating subquery results which can exhaust the heap
+Therefore, there are guardrails that are built in Druid to prevent the queries from generating subquery results which can exhaust the heap
space. They can be set on a cluster level or modified per query level as desired.
Note the following guardrails that can be set by the cluster admin to limit the subquery results:
@@ -1871,6 +1871,15 @@ Note the following guardrails that can be set by the cluster admin to limit the
Note that limiting the subquery by bytes is a newer feature therefore it is experimental as it materializes the results differently.
+`maxSubqueryBytes` can be configured to the following values:
+1. 'disabled' - It is the default setting out of the box. It disables the subquery's from the byte based limit, and effectively disables this feature.
+2. 'auto' - Druid automatically decides the optimal byte based limit based upon the heap space available and the max number of concurrent queries.
+3. A positive long value - User can manually specify the number of bytes that the results of the subqueries of a single query can occupy on the heap.
+
+Due to the conversion between the Java objects and the Frame's format, setting `maxSubqueryBytes` can become slow if the subquery starts generating
+rows in the order of magnitude of around 10 million and above. In those scenarios, disable the `maxSubqueryBytes` settings for such queries, assess the
+number of rows that the subqueries generate and override the `maxSubqueryRows` to appropriate value.
+
If you choose to modify or set any of the above limits, you must also think about the heap size of all Brokers, Historicals, and task Peons that process data for the subqueries to accommodate the subquery results.
There is no formula to calculate the correct value. Trial and error is the best approach.
@@ -1895,7 +1904,7 @@ Druid uses Jetty to serve HTTP requests. Each query being processed consumes a s
|`druid.server.http.defaultQueryTimeout`|Query timeout in millis, beyond which unfinished queries will be cancelled|300000|
|`druid.server.http.maxScatterGatherBytes`|Maximum number of bytes gathered from data processes such as Historicals and realtime processes to execute a query. Queries that exceed this limit will fail. This is an advance configuration that allows to protect in case Broker is under heavy load and not utilizing the data gathered in memory fast enough and leading to OOMs. This limit can be further reduced at query time using `maxScatterGatherBytes` in the context. Note that having large limit is not necessarily bad if broker is never under heavy concurrent load in which case data gathered is processed quickly and freeing up the memory used. Human-readable format is supported, see [here](human-readable-byte.md). |Long.MAX_VALUE|
|`druid.server.http.maxSubqueryRows`|Maximum number of rows from all subqueries per query. Druid stores the subquery rows in temporary tables that live in the Java heap. `druid.server.http.maxSubqueryRows` is a guardrail to prevent the system from exhausting available heap. When a subquery exceeds the row limit, Druid throws a resource limit exceeded exception: "Subquery generated results beyond maximum." It is a good practice to avoid large subqueries in Druid. However, if you choose to raise the subquery row limit, you must also increase the heap size of all Brokers, Historicals, and task Peons that process data for the subqueries to accommodate the subquery results. There is no formula to calculate the correct value. Trial and error is the best approach.|100000|
-|`druid.server.http.maxSubqueryBytes`|Maximum number of bytes from all subqueries per query. Since the results are stored on the Java heap, `druid.server.http.maxSubqueryBytes` is a guardrail like `druid.server.http.maxSubqueryRows` to prevent the heap space from exhausting. When a subquery exceeds the byte limit, Druid throws a resource limit exceeded exception. A negative value for the guardrail indicates that Druid won't guardrail by memory. Check the docs for `druid.server.http.maxSubqueryRows` to see how to set the optimal value for a cluster. This is an experimental feature for now as this materializes the results in a different format.|-1|
+|`druid.server.http.maxSubqueryBytes`|Maximum number of bytes from all subqueries per query. Since the results are stored on the Java heap, `druid.server.http.maxSubqueryBytes` is a guardrail like `druid.server.http.maxSubqueryRows` to prevent the heap space from exhausting. When a subquery exceeds the byte limit, Druid throws a resource limit exceeded exception. A negative value for the guardrail indicates that Druid won't guardrail by memory. This can be set to 'disabled' which disables the results from being limited via the byte limit, 'auto' which sets this value automatically taking free heap space into account, or a positive long value depicting the number of bytes per query's subqueries' results can occupy. This is an experimental feature for now as this materializes the results in a different format.|'disabled'|
|`druid.server.http.gracefulShutdownTimeout`|The maximum amount of time Jetty waits after receiving shutdown signal. After this timeout the threads will be forcefully shutdown. This allows any queries that are executing to complete(Only values greater than zero are valid).|`PT30S`|
|`druid.server.http.unannouncePropagationDelay`|How long to wait for ZooKeeper unannouncements to propagate before shutting down Jetty. This is a minimum and `druid.server.http.gracefulShutdownTimeout` does not start counting down until after this period elapses.|`PT0S` (do not wait)|
|`druid.server.http.maxQueryTimeout`|Maximum allowed value (in milliseconds) for `timeout` parameter. See [query-context](../querying/query-context.md) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE|
diff --git a/docs/development/extensions-core/lookups-cached-global.md b/docs/development/extensions-core/lookups-cached-global.md
index ebeca5a741e0..dc8827a5b368 100644
--- a/docs/development/extensions-core/lookups-cached-global.md
+++ b/docs/development/extensions-core/lookups-cached-global.md
@@ -352,6 +352,7 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol
|`filter`|The filter to use when selecting lookups, this is used to create a where clause on lookup population|No|No Filter|
|`tsColumn`| The column in `table` which contains when the key was updated|No|Not used|
|`pollPeriod`|How often to poll the DB|No|0 (only once)|
+|`jitterSeconds`| How much jitter to add (in seconds) up to maximum as a delay (actual value will be used as random from 0 to `jitterSeconds`), used to distribute db load more evenly|No|0|
|`maxHeapPercentage`|The maximum percentage of heap size that the lookup should consume. If the lookup grows beyond this size, warning messages will be logged in the respective service logs.|No|10% of JVM heap size|
```json
@@ -367,6 +368,7 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol
"valueColumn":"the_new_dim_value",
"tsColumn":"timestamp_column",
"pollPeriod":600000,
+ "jitterSeconds": 120,
"maxHeapPercentage": 10
}
```
diff --git a/docs/multi-stage-query/known-issues.md b/docs/multi-stage-query/known-issues.md
index bccb9779a835..570a7f58fa4e 100644
--- a/docs/multi-stage-query/known-issues.md
+++ b/docs/multi-stage-query/known-issues.md
@@ -39,9 +39,7 @@ an [UnknownError](./reference.md#error_UnknownError) with a message including "N
## `SELECT` Statement
-- `SELECT` from a Druid datasource does not include unpublished real-time data.
-
-- `GROUPING SETS` and `UNION ALL` are not implemented. Queries using these features return a
+- `GROUPING SETS` are not implemented. Queries using these features return a
[QueryNotSupported](reference.md#error_QueryNotSupported) error.
- For some `COUNT DISTINCT` queries, you'll encounter a [QueryNotSupported](reference.md#error_QueryNotSupported) error
diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md
index 010bbff2a270..5e80e318b8c8 100644
--- a/docs/multi-stage-query/reference.md
+++ b/docs/multi-stage-query/reference.md
@@ -247,6 +247,7 @@ The following table lists the context parameters for the MSQ task engine:
| `faultTolerance` | SELECT, INSERT, REPLACE Whether to turn on fault tolerance mode or not. Failed workers are retried based on [Limits](#limits). Cannot be used when `durableShuffleStorage` is explicitly set to false. | `false` |
| `selectDestination` | SELECT Controls where the final result of the select query is written. Use `taskReport`(the default) to write select results to the task report. This is not scalable since task reports size explodes for large results Use `durableStorage` to write results to durable storage location. For large results sets, its recommended to use `durableStorage` . To configure durable storage see [`this`](#durable-storage) section. | `taskReport` |
| `waitTillSegmentsLoad` | INSERT, REPLACE If set, the ingest query waits for the generated segment to be loaded before exiting, else the ingest query exits without waiting. The task and live reports contain the information about the status of loading segments if this flag is set. This will ensure that any future queries made after the ingestion exits will include results from the ingestion. The drawback is that the controller task will stall till the segments are loaded. | `false` |
+| `includeSegmentSource` | SELECT, INSERT, REPLACE Controls the sources, which will be queried for results in addition to the segments present on deep storage. Can be `NONE` or `REALTIME`. If this value is `NONE`, only non-realtime (published and used) segments will be downloaded from deep storage. If this value is `REALTIME`, results will also be included from realtime tasks. | `NONE` |
## Joins
diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md
index 77a79170bec8..28e8a9fa9646 100644
--- a/docs/operations/metrics.md
+++ b/docs/operations/metrics.md
@@ -62,6 +62,7 @@ Most metric values reset each emission period, as specified in `druid.monitoring
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
+|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/segments/count`|This metric is not enabled by default. See the `QueryMetrics` Interface for reference regarding enabling this metric. Number of segments that will be touched by the query. In the broker, it makes a plan to distribute the query to realtime tasks and historicals based on a snapshot of segment distribution state. If there are some segments moved after this snapshot is created, certain historicals and realtime tasks can report those segments as missing to the broker. The broker will resend the query to the new servers that serve those segments after move. In this case, those segments can be counted more than once in this metric.||Varies|
|`query/priority`|Assigned lane and priority, only if Laning strategy is enabled. Refer to [Laning strategies](../configuration/index.md#laning-strategies)|`lane`, `dataSource`, `type`|0|
|`sqlQuery/time`|Milliseconds taken to complete a SQL query.|`id`, `nativeQueryIds`, `dataSource`, `remoteAddress`, `success`, `engine`|< 1s|
@@ -97,6 +98,7 @@ Most metric values reset each emission period, as specified in `druid.monitoring
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
+|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
### Real-time
diff --git a/docs/querying/query-context.md b/docs/querying/query-context.md
index 326753970fbb..1ac3af1127c4 100644
--- a/docs/querying/query-context.md
+++ b/docs/querying/query-context.md
@@ -52,6 +52,8 @@ Unless otherwise noted, the following parameters apply to all query types.
|`finalize` | `N/A` | Flag indicating whether to "finalize" aggregation results. Primarily used for debugging. For instance, the `hyperUnique` aggregator returns the full HyperLogLog sketch instead of the estimated cardinality when this flag is set to `false` |
|`maxScatterGatherBytes`| `druid.server.http.maxScatterGatherBytes` | Maximum number of bytes gathered from data processes such as Historicals and realtime processes to execute a query. This parameter can be used to further reduce `maxScatterGatherBytes` limit at query time. See [Broker configuration](../configuration/index.md#broker) for more details.|
|`maxQueuedBytes` | `druid.broker.http.maxQueuedBytes` | Maximum number of bytes queued per query before exerting backpressure on the channel to the data server. Similar to `maxScatterGatherBytes`, except unlike that configuration, this one will trigger backpressure rather than query failure. Zero means disabled.|
+|`maxSubqueryRows`| `druid.server.http.maxSubqueryRows` | Upper limit on the number of rows a subquery can generate. See [Broker configuration](../configuration/index.md#broker) and [subquery guardrails](../configuration/index.md#Guardrails for materialization of subqueries) for more details.|
+|`maxSubqueryBytes`| `druid.server.http.maxSubqueryBytes` | Upper limit on the number of bytes a subquery can generate. See [Broker configuration](../configuration/index.md#broker) and [subquery guardrails](../configuration/index.md#Guardrails for materialization of subqueries) for more details.|
|`serializeDateTimeAsLong`| `false` | If true, DateTime is serialized as long in the result returned by Broker and the data transportation between Broker and compute process|
|`serializeDateTimeAsLongInner`| `false` | If true, DateTime is serialized as long in the data transportation between Broker and compute process|
|`enableParallelMerge`|`true`|Enable parallel result merging on the Broker. Note that `druid.processing.merge.useParallelMergePool` must be enabled for this setting to be set to `true`. See [Broker configuration](../configuration/index.md#broker) for more details.|
diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorBase.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorBase.java
index 4a61f0271eeb..a6c23551598e 100644
--- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorBase.java
+++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorBase.java
@@ -21,8 +21,6 @@
import com.google.common.collect.ImmutableList;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
@@ -36,12 +34,12 @@
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
@@ -71,12 +69,10 @@ public SqlAggFunction calciteFunction()
@Override
public Aggregation toDruidAggregation(
PlannerContext plannerContext,
- RowSignature rowSignature,
VirtualColumnRegistry virtualColumnRegistry,
- RexBuilder rexBuilder,
String name,
AggregateCall aggregateCall,
- Project project,
+ InputAccessor inputAccessor,
List existingAggregations,
boolean finalizeAggregations
)
@@ -88,13 +84,8 @@ public Aggregation toDruidAggregation(
// fetch sum column expression
DruidExpression sumColumn = Expressions.toDruidExpression(
plannerContext,
- rowSignature,
- Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(0)
- )
+ inputAccessor.getInputRowSignature(),
+ inputAccessor.getField(aggregateCall.getArgList().get(0))
);
if (sumColumn == null) {
@@ -114,12 +105,7 @@ public Aggregation toDruidAggregation(
Integer size = null;
if (aggregateCall.getArgList().size() >= 2) {
- RexNode sizeArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(1)
- );
+ RexNode sizeArg = inputAccessor.getField(aggregateCall.getArgList().get(1));
size = ((Number) RexLiteral.value(sizeArg)).intValue();
}
@@ -128,12 +114,7 @@ public Aggregation toDruidAggregation(
Integer scale = null;
if (aggregateCall.getArgList().size() >= 3) {
- RexNode scaleArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(2)
- );
+ RexNode scaleArg = inputAccessor.getField(aggregateCall.getArgList().get(2));
scale = ((Number) RexLiteral.value(scaleArg)).intValue();
}
@@ -141,12 +122,7 @@ public Aggregation toDruidAggregation(
Boolean useStrictNumberParsing = null;
if (aggregateCall.getArgList().size() >= 4) {
- RexNode useStrictNumberParsingArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(3)
- );
+ RexNode useStrictNumberParsingArg = inputAccessor.getField(aggregateCall.getArgList().get(3));
useStrictNumberParsing = RexLiteral.booleanValue(useStrictNumberParsingArg);
}
diff --git a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestGenerateSketchSqlAggregator.java b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestGenerateSketchSqlAggregator.java
index ca0a4acc603f..ebb6c7f4b141 100644
--- a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestGenerateSketchSqlAggregator.java
+++ b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestGenerateSketchSqlAggregator.java
@@ -20,8 +20,6 @@
package org.apache.druid.query.aggregation.tdigestsketch.sql;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
@@ -36,13 +34,12 @@
import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchAggregatorFactory;
import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchUtils;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.Aggregations;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
-import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
@@ -63,25 +60,18 @@ public SqlAggFunction calciteFunction()
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
- final RowSignature rowSignature,
final VirtualColumnRegistry virtualColumnRegistry,
- final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
- final Project project,
+ final InputAccessor inputAccessor,
final List existingAggregations,
final boolean finalizeAggregations
)
{
- final RexNode inputOperand = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(0)
- );
+ final RexNode inputOperand = inputAccessor.getField(aggregateCall.getArgList().get(0));
final DruidExpression input = Aggregations.toDruidExpressionForNumericAggregator(
plannerContext,
- rowSignature,
+ inputAccessor.getInputRowSignature(),
inputOperand
);
if (input == null) {
@@ -93,12 +83,7 @@ public Aggregation toDruidAggregation(
Integer compression = TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION;
if (aggregateCall.getArgList().size() > 1) {
- RexNode compressionOperand = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(1)
- );
+ RexNode compressionOperand = inputAccessor.getField(aggregateCall.getArgList().get(1));
if (!compressionOperand.isA(SqlKind.LITERAL)) {
// compressionOperand must be a literal in order to plan.
return null;
diff --git a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchQuantileSqlAggregator.java b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchQuantileSqlAggregator.java
index 379e889d3835..ee63444f6d71 100644
--- a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchQuantileSqlAggregator.java
+++ b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchQuantileSqlAggregator.java
@@ -21,8 +21,6 @@
import com.google.common.collect.ImmutableList;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
@@ -39,13 +37,12 @@
import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchToQuantilePostAggregator;
import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchUtils;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.Aggregations;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
-import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
@@ -66,12 +63,10 @@ public SqlAggFunction calciteFunction()
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
- final RowSignature rowSignature,
final VirtualColumnRegistry virtualColumnRegistry,
- final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
- final Project project,
+ final InputAccessor inputAccessor,
final List existingAggregations,
final boolean finalizeAggregations
)
@@ -79,13 +74,8 @@ public Aggregation toDruidAggregation(
// This is expected to be a tdigest sketch
final DruidExpression input = Aggregations.toDruidExpressionForNumericAggregator(
plannerContext,
- rowSignature,
- Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(0)
- )
+ inputAccessor.getInputRowSignature(),
+ inputAccessor.getField(aggregateCall.getArgList().get(0))
);
if (input == null) {
return null;
@@ -95,12 +85,7 @@ public Aggregation toDruidAggregation(
final String sketchName = StringUtils.format("%s:agg", name);
// this is expected to be quantile fraction
- final RexNode quantileArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(1)
- );
+ final RexNode quantileArg = inputAccessor.getField(aggregateCall.getArgList().get(1));
if (!quantileArg.isA(SqlKind.LITERAL)) {
// Quantile must be a literal in order to plan.
@@ -110,12 +95,7 @@ public Aggregation toDruidAggregation(
final double quantile = ((Number) RexLiteral.value(quantileArg)).floatValue();
Integer compression = TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION;
if (aggregateCall.getArgList().size() > 2) {
- final RexNode compressionArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(2)
- );
+ final RexNode compressionArg = inputAccessor.getField(aggregateCall.getArgList().get(2));
compression = ((Number) RexLiteral.value(compressionArg)).intValue();
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java
index c6dd3e7afa02..d221b72ac1c6 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java
@@ -20,9 +20,7 @@
package org.apache.druid.query.aggregation.datasketches.hll.sql;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
@@ -36,7 +34,6 @@
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
@@ -44,6 +41,7 @@
import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
@@ -66,38 +64,26 @@ protected HllSketchBaseSqlAggregator(boolean finalizeSketch, StringEncoding stri
@Override
public Aggregation toDruidAggregation(
PlannerContext plannerContext,
- RowSignature rowSignature,
VirtualColumnRegistry virtualColumnRegistry,
- RexBuilder rexBuilder,
String name,
AggregateCall aggregateCall,
- Project project,
+ InputAccessor inputAccessor,
List existingAggregations,
boolean finalizeAggregations
)
{
// Don't use Aggregations.getArgumentsForSimpleAggregator, since it won't let us use direct column access
// for string columns.
- final RexNode columnRexNode = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(0)
- );
+ final RexNode columnRexNode = inputAccessor.getField(aggregateCall.getArgList().get(0));
- final DruidExpression columnArg = Expressions.toDruidExpression(plannerContext, rowSignature, columnRexNode);
+ final DruidExpression columnArg = Expressions.toDruidExpression(plannerContext, inputAccessor.getInputRowSignature(), columnRexNode);
if (columnArg == null) {
return null;
}
final int logK;
if (aggregateCall.getArgList().size() >= 2) {
- final RexNode logKarg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(1)
- );
+ final RexNode logKarg = inputAccessor.getField(aggregateCall.getArgList().get(1));
if (!logKarg.isA(SqlKind.LITERAL)) {
// logK must be a literal in order to plan.
@@ -111,12 +97,7 @@ public Aggregation toDruidAggregation(
final String tgtHllType;
if (aggregateCall.getArgList().size() >= 3) {
- final RexNode tgtHllTypeArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(2)
- );
+ final RexNode tgtHllTypeArg = inputAccessor.getField(aggregateCall.getArgList().get(2));
if (!tgtHllTypeArg.isA(SqlKind.LITERAL)) {
// tgtHllType must be a literal in order to plan.
@@ -132,9 +113,10 @@ public Aggregation toDruidAggregation(
final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name;
if (columnArg.isDirectColumnAccess()
- && rowSignature.getColumnType(columnArg.getDirectColumn())
- .map(type -> type.is(ValueType.COMPLEX))
- .orElse(false)) {
+ && inputAccessor.getInputRowSignature()
+ .getColumnType(columnArg.getDirectColumn())
+ .map(type -> type.is(ValueType.COMPLEX))
+ .orElse(false)) {
aggregatorFactory = new HllSketchMergeAggregatorFactory(
aggregatorName,
columnArg.getDirectColumn(),
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java
index 6c1b5720af49..08c7a1b123fd 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java
@@ -21,8 +21,6 @@
import com.google.common.collect.ImmutableList;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
@@ -37,14 +35,13 @@
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToQuantilePostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.Aggregations;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
-import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.expression.OperatorConversions;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
@@ -75,25 +72,18 @@ public SqlAggFunction calciteFunction()
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
- final RowSignature rowSignature,
final VirtualColumnRegistry virtualColumnRegistry,
- final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
- final Project project,
+ final InputAccessor inputAccessor,
final List existingAggregations,
final boolean finalizeAggregations
)
{
final DruidExpression input = Aggregations.toDruidExpressionForNumericAggregator(
plannerContext,
- rowSignature,
- Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(0)
- )
+ inputAccessor.getInputRowSignature(),
+ inputAccessor.getField(aggregateCall.getArgList().get(0))
);
if (input == null) {
return null;
@@ -101,12 +91,7 @@ public Aggregation toDruidAggregation(
final AggregatorFactory aggregatorFactory;
final String histogramName = StringUtils.format("%s:agg", name);
- final RexNode probabilityArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(1)
- );
+ final RexNode probabilityArg = inputAccessor.getField(aggregateCall.getArgList().get(1));
if (!probabilityArg.isA(SqlKind.LITERAL)) {
// Probability must be a literal in order to plan.
@@ -117,12 +102,7 @@ public Aggregation toDruidAggregation(
final int k;
if (aggregateCall.getArgList().size() >= 3) {
- final RexNode resolutionArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(2)
- );
+ final RexNode resolutionArg = inputAccessor.getField(aggregateCall.getArgList().get(2));
if (!resolutionArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java
index 049e1284a911..8331ab720640 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java
@@ -21,8 +21,6 @@
import com.google.common.collect.ImmutableList;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
@@ -35,14 +33,13 @@
import org.apache.druid.query.aggregation.datasketches.SketchQueryContext;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.Aggregations;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
-import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.expression.OperatorConversions;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
@@ -71,25 +68,18 @@ public SqlAggFunction calciteFunction()
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
- final RowSignature rowSignature,
final VirtualColumnRegistry virtualColumnRegistry,
- final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
- final Project project,
+ final InputAccessor inputAccessor,
final List existingAggregations,
final boolean finalizeAggregations
)
{
final DruidExpression input = Aggregations.toDruidExpressionForNumericAggregator(
plannerContext,
- rowSignature,
- Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(0)
- )
+ inputAccessor.getInputRowSignature(),
+ inputAccessor.getField(aggregateCall.getArgList().get(0))
);
if (input == null) {
return null;
@@ -100,12 +90,7 @@ public Aggregation toDruidAggregation(
final int k;
if (aggregateCall.getArgList().size() >= 2) {
- final RexNode resolutionArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(1)
- );
+ final RexNode resolutionArg = inputAccessor.getField(aggregateCall.getArgList().get(1));
if (!resolutionArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java
index 6564b276c971..bf35cd665ae8 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java
@@ -20,9 +20,7 @@
package org.apache.druid.query.aggregation.datasketches.theta.sql;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
@@ -34,7 +32,6 @@
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
@@ -42,6 +39,7 @@
import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
@@ -60,38 +58,26 @@ protected ThetaSketchBaseSqlAggregator(boolean finalizeSketch)
@Override
public Aggregation toDruidAggregation(
PlannerContext plannerContext,
- RowSignature rowSignature,
VirtualColumnRegistry virtualColumnRegistry,
- RexBuilder rexBuilder,
String name,
AggregateCall aggregateCall,
- Project project,
+ InputAccessor inputAccessor,
List existingAggregations,
boolean finalizeAggregations
)
{
// Don't use Aggregations.getArgumentsForSimpleAggregator, since it won't let us use direct column access
// for string columns.
- final RexNode columnRexNode = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(0)
- );
+ final RexNode columnRexNode = inputAccessor.getField(aggregateCall.getArgList().get(0));
- final DruidExpression columnArg = Expressions.toDruidExpression(plannerContext, rowSignature, columnRexNode);
+ final DruidExpression columnArg = Expressions.toDruidExpression(plannerContext, inputAccessor.getInputRowSignature(), columnRexNode);
if (columnArg == null) {
return null;
}
final int sketchSize;
if (aggregateCall.getArgList().size() >= 2) {
- final RexNode sketchSizeArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(1)
- );
+ final RexNode sketchSizeArg = inputAccessor.getField(aggregateCall.getArgList().get(1));
if (!sketchSizeArg.isA(SqlKind.LITERAL)) {
// logK must be a literal in order to plan.
@@ -107,9 +93,10 @@ public Aggregation toDruidAggregation(
final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name;
if (columnArg.isDirectColumnAccess()
- && rowSignature.getColumnType(columnArg.getDirectColumn())
- .map(type -> type.is(ValueType.COMPLEX))
- .orElse(false)) {
+ && inputAccessor.getInputRowSignature()
+ .getColumnType(columnArg.getDirectColumn())
+ .map(type -> type.is(ValueType.COMPLEX))
+ .orElse(false)) {
aggregatorFactory = new SketchMergeAggregatorFactory(
aggregatorName,
columnArg.getDirectColumn(),
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregator.java
index 9d6ddac89a89..a9b1aaa627d0 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregator.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregator.java
@@ -20,9 +20,7 @@
package org.apache.druid.query.aggregation.datasketches.tuple.sql;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
@@ -38,7 +36,6 @@
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
@@ -46,6 +43,7 @@
import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
@@ -69,12 +67,10 @@ public SqlAggFunction calciteFunction()
@Override
public Aggregation toDruidAggregation(
PlannerContext plannerContext,
- RowSignature rowSignature,
VirtualColumnRegistry virtualColumnRegistry,
- RexBuilder rexBuilder,
String name,
AggregateCall aggregateCall,
- Project project,
+ InputAccessor inputAccessor,
List existingAggregations,
boolean finalizeAggregations
)
@@ -86,12 +82,7 @@ public Aggregation toDruidAggregation(
final int nominalEntries;
final int metricExpressionEndIndex;
final int lastArgIndex = argList.size() - 1;
- final RexNode potentialNominalEntriesArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- argList.get(lastArgIndex)
- );
+ final RexNode potentialNominalEntriesArg = inputAccessor.getField(argList.get(lastArgIndex));
if (potentialNominalEntriesArg.isA(SqlKind.LITERAL) &&
RexLiteral.value(potentialNominalEntriesArg) instanceof Number) {
@@ -107,16 +98,11 @@ public Aggregation toDruidAggregation(
for (int i = 0; i <= metricExpressionEndIndex; i++) {
final String fieldName;
- final RexNode columnRexNode = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- argList.get(i)
- );
+ final RexNode columnRexNode = inputAccessor.getField(argList.get(i));
final DruidExpression columnArg = Expressions.toDruidExpression(
plannerContext,
- rowSignature,
+ inputAccessor.getInputRowSignature(),
columnRexNode
);
if (columnArg == null) {
@@ -124,9 +110,10 @@ public Aggregation toDruidAggregation(
}
if (columnArg.isDirectColumnAccess() &&
- rowSignature.getColumnType(columnArg.getDirectColumn())
- .map(type -> type.is(ValueType.COMPLEX))
- .orElse(false)) {
+ inputAccessor.getInputRowSignature()
+ .getColumnType(columnArg.getDirectColumn())
+ .map(type -> type.is(ValueType.COMPLEX))
+ .orElse(false)) {
fieldName = columnArg.getDirectColumn();
} else {
final RelDataType dataType = columnRexNode.getType();
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
index 349f1a57d1c0..20ea97aec3e5 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
@@ -369,7 +369,7 @@ public void testAvgDailyCountDistinctHllSketch()
final List expectedResults = ImmutableList.of(
new Object[]{
- 1L
+ 1.0
}
);
@@ -429,11 +429,11 @@ public void testAvgDailyCountDistinctHllSketch()
.setAggregatorSpecs(
NullHandling.replaceWithDefault()
? Arrays.asList(
- new LongSumAggregatorFactory("_a0:sum", "a0"),
+ new DoubleSumAggregatorFactory("_a0:sum", "a0"),
new CountAggregatorFactory("_a0:count")
)
: Arrays.asList(
- new LongSumAggregatorFactory("_a0:sum", "a0"),
+ new DoubleSumAggregatorFactory("_a0:sum", "a0"),
new FilteredAggregatorFactory(
new CountAggregatorFactory("_a0:count"),
notNull("a0")
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
index 3946ce558b19..3a079e064783 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
@@ -278,7 +278,7 @@ public void testAvgDailyCountDistinctThetaSketch()
final List expectedResults = ImmutableList.of(
new Object[]{
- 1L
+ 1.0
}
);
@@ -334,11 +334,11 @@ public void testAvgDailyCountDistinctThetaSketch()
.setAggregatorSpecs(
NullHandling.replaceWithDefault()
? Arrays.asList(
- new LongSumAggregatorFactory("_a0:sum", "a0"),
+ new DoubleSumAggregatorFactory("_a0:sum", "a0"),
new CountAggregatorFactory("_a0:count")
)
: Arrays.asList(
- new LongSumAggregatorFactory("_a0:sum", "a0"),
+ new DoubleSumAggregatorFactory("_a0:sum", "a0"),
new FilteredAggregatorFactory(
new CountAggregatorFactory("_a0:count"),
notNull("a0")
diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java
index 600af931f031..85cc60d2e76b 100644
--- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java
+++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java
@@ -182,15 +182,27 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo
return;
}
- String[] splits = decodedUserSecret.split(":");
- if (splits.length != 2) {
+ /* From https://www.rfc-editor.org/rfc/rfc7617.html, we can assume that userid won't include a colon but password
+ can.
+
+ The user-id and password MUST NOT contain any control characters (see
+ "CTL" in Appendix B.1 of [RFC5234]).
+
+ Furthermore, a user-id containing a colon character is invalid, as
+ the first colon in a user-pass string separates user-id and password
+ from one another; text after the first colon is part of the password.
+ User-ids containing colons cannot be encoded in user-pass strings.
+
+ */
+ int split = decodedUserSecret.indexOf(':');
+ if (split < 0) {
// The decoded user secret is not of the right format
httpResp.sendError(HttpServletResponse.SC_UNAUTHORIZED);
return;
}
- String user = splits[0];
- char[] password = splits[1].toCharArray();
+ String user = decodedUserSecret.substring(0, split);
+ char[] password = decodedUserSecret.substring(split + 1).toCharArray();
// If any authentication error occurs we send a 401 response immediately and do not proceed further down the filter chain.
// If the authentication result is null and skipOnFailure property is false, we send a 401 response and do not proceed
diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java
index 84bfdcf56b1c..bf0cf1778a14 100644
--- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java
+++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java
@@ -112,55 +112,21 @@ public void testGoodPassword() throws IOException, ServletException
}
@Test
- public void testGoodPasswordWithValidator() throws IOException, ServletException
+ public void testGoodNonEmptyPasswordWithValidator() throws IOException, ServletException
{
- CredentialsValidator validator = EasyMock.createMock(CredentialsValidator.class);
- BasicHTTPAuthenticator authenticatorWithValidator = new BasicHTTPAuthenticator(
- CACHE_MANAGER_PROVIDER,
- "basic",
- "basic",
- null,
- null,
- false,
- null, null,
- false,
- validator
- );
-
- String header = StringUtils.utf8Base64("userA:helloworld");
- header = StringUtils.format("Basic %s", header);
-
- EasyMock
- .expect(
- validator.validateCredentials(EasyMock.eq("basic"), EasyMock.eq("basic"), EasyMock.eq("userA"), EasyMock.aryEq("helloworld".toCharArray()))
- )
- .andReturn(
- new AuthenticationResult("userA", "basic", "basic", null)
- )
- .times(1);
- EasyMock.replay(validator);
-
- HttpServletRequest req = EasyMock.createMock(HttpServletRequest.class);
- EasyMock.expect(req.getHeader("Authorization")).andReturn(header);
- req.setAttribute(
- AuthConfig.DRUID_AUTHENTICATION_RESULT,
- new AuthenticationResult("userA", "basic", "basic", null)
- );
- EasyMock.expectLastCall().times(1);
- EasyMock.replay(req);
-
- HttpServletResponse resp = EasyMock.createMock(HttpServletResponse.class);
- EasyMock.replay(resp);
-
- FilterChain filterChain = EasyMock.createMock(FilterChain.class);
- filterChain.doFilter(req, resp);
- EasyMock.expectLastCall().times(1);
- EasyMock.replay(filterChain);
+ testGoodPasswordWithValidator("userA", "helloworld");
+ }
- Filter authenticatorFilter = authenticatorWithValidator.getFilter();
- authenticatorFilter.doFilter(req, resp, filterChain);
+ @Test
+ public void testGoodEmptyPasswordWithValidator() throws IOException, ServletException
+ {
+ testGoodPasswordWithValidator("userA", "");
+ }
- EasyMock.verify(req, resp, validator, filterChain);
+ @Test
+ public void testGoodColonInPasswordWithValidator() throws IOException, ServletException
+ {
+ testGoodPasswordWithValidator("userA", "hello:hello");
}
@Test
@@ -396,4 +362,55 @@ public void testMissingHeader() throws IOException, ServletException
EasyMock.verify(req, resp, filterChain);
}
+
+ private void testGoodPasswordWithValidator(String username, String password) throws IOException, ServletException
+ {
+ CredentialsValidator validator = EasyMock.createMock(CredentialsValidator.class);
+ BasicHTTPAuthenticator authenticatorWithValidator = new BasicHTTPAuthenticator(
+ CACHE_MANAGER_PROVIDER,
+ "basic",
+ "basic",
+ null,
+ null,
+ false,
+ null, null,
+ false,
+ validator
+ );
+
+ String header = StringUtils.utf8Base64(username + ":" + password);
+ header = StringUtils.format("Basic %s", header);
+
+ EasyMock
+ .expect(
+ validator.validateCredentials(EasyMock.eq("basic"), EasyMock.eq("basic"), EasyMock.eq(username), EasyMock.aryEq(password.toCharArray()))
+ )
+ .andReturn(
+ new AuthenticationResult(username, "basic", "basic", null)
+ )
+ .times(1);
+ EasyMock.replay(validator);
+
+ HttpServletRequest req = EasyMock.createMock(HttpServletRequest.class);
+ EasyMock.expect(req.getHeader("Authorization")).andReturn(header);
+ req.setAttribute(
+ AuthConfig.DRUID_AUTHENTICATION_RESULT,
+ new AuthenticationResult(username, "basic", "basic", null)
+ );
+ EasyMock.expectLastCall().times(1);
+ EasyMock.replay(req);
+
+ HttpServletResponse resp = EasyMock.createMock(HttpServletResponse.class);
+ EasyMock.replay(resp);
+
+ FilterChain filterChain = EasyMock.createMock(FilterChain.class);
+ filterChain.doFilter(req, resp);
+ EasyMock.expectLastCall().times(1);
+ EasyMock.replay(filterChain);
+
+ Filter authenticatorFilter = authenticatorWithValidator.getFilter();
+ authenticatorFilter.doFilter(req, resp, filterChain);
+
+ EasyMock.verify(req, resp, validator, filterChain);
+ }
}
diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregator.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregator.java
index 0ec265595e11..6a1ca49067e7 100644
--- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregator.java
+++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregator.java
@@ -20,8 +20,6 @@
package org.apache.druid.query.aggregation.bloom.sql;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
@@ -38,13 +36,13 @@
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.dimension.ExtractionDimensionSpec;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
@@ -65,25 +63,18 @@ public SqlAggFunction calciteFunction()
@Override
public Aggregation toDruidAggregation(
PlannerContext plannerContext,
- RowSignature rowSignature,
VirtualColumnRegistry virtualColumnRegistry,
- RexBuilder rexBuilder,
String name,
AggregateCall aggregateCall,
- Project project,
+ InputAccessor inputAccessor,
List existingAggregations,
boolean finalizeAggregations
)
{
- final RexNode inputOperand = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(0)
- );
+ final RexNode inputOperand = inputAccessor.getField(aggregateCall.getArgList().get(0));
final DruidExpression input = Expressions.toDruidExpression(
plannerContext,
- rowSignature,
+ inputAccessor.getInputRowSignature(),
inputOperand
);
if (input == null) {
@@ -92,12 +83,7 @@ public Aggregation toDruidAggregation(
final AggregatorFactory aggregatorFactory;
final String aggName = StringUtils.format("%s:agg", name);
- final RexNode maxNumEntriesOperand = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(1)
- );
+ final RexNode maxNumEntriesOperand = inputAccessor.getField(aggregateCall.getArgList().get(1));
if (!maxNumEntriesOperand.isA(SqlKind.LITERAL)) {
// maxNumEntriesOperand must be a literal in order to plan.
diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java
index 3f0bd14f8449..fdc61796c4d9 100644
--- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java
+++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java
@@ -21,8 +21,6 @@
import com.google.common.collect.ImmutableList;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
@@ -38,13 +36,12 @@
import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogramAggregatorFactory;
import org.apache.druid.query.aggregation.histogram.QuantilePostAggregator;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.Aggregations;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
-import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
@@ -65,25 +62,18 @@ public SqlAggFunction calciteFunction()
@Override
public Aggregation toDruidAggregation(
PlannerContext plannerContext,
- RowSignature rowSignature,
VirtualColumnRegistry virtualColumnRegistry,
- RexBuilder rexBuilder,
String name,
AggregateCall aggregateCall,
- Project project,
+ InputAccessor inputAccessor,
List existingAggregations,
boolean finalizeAggregations
)
{
final DruidExpression input = Aggregations.toDruidExpressionForNumericAggregator(
plannerContext,
- rowSignature,
- Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(0)
- )
+ inputAccessor.getInputRowSignature(),
+ inputAccessor.getField(aggregateCall.getArgList().get(0))
);
if (input == null) {
return null;
@@ -91,12 +81,7 @@ public Aggregation toDruidAggregation(
final AggregatorFactory aggregatorFactory;
final String histogramName = StringUtils.format("%s:agg", name);
- final RexNode probabilityArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(1)
- );
+ final RexNode probabilityArg = inputAccessor.getField(aggregateCall.getArgList().get(1));
if (!probabilityArg.isA(SqlKind.LITERAL)) {
// Probability must be a literal in order to plan.
@@ -107,12 +92,7 @@ public Aggregation toDruidAggregation(
final int numBuckets;
if (aggregateCall.getArgList().size() >= 3) {
- final RexNode numBucketsArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(2)
- );
+ final RexNode numBucketsArg = inputAccessor.getField(aggregateCall.getArgList().get(2));
if (!numBucketsArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
@@ -126,12 +106,7 @@ public Aggregation toDruidAggregation(
final double lowerLimit;
if (aggregateCall.getArgList().size() >= 4) {
- final RexNode lowerLimitArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(3)
- );
+ final RexNode lowerLimitArg = inputAccessor.getField(aggregateCall.getArgList().get(3));
if (!lowerLimitArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
@@ -145,12 +120,7 @@ public Aggregation toDruidAggregation(
final double upperLimit;
if (aggregateCall.getArgList().size() >= 5) {
- final RexNode upperLimitArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(4)
- );
+ final RexNode upperLimitArg = inputAccessor.getField(aggregateCall.getArgList().get(4));
if (!upperLimitArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
@@ -164,12 +134,7 @@ public Aggregation toDruidAggregation(
final FixedBucketsHistogram.OutlierHandlingMode outlierHandlingMode;
if (aggregateCall.getArgList().size() >= 6) {
- final RexNode outlierHandlingModeArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(5)
- );
+ final RexNode outlierHandlingModeArg = inputAccessor.getField(aggregateCall.getArgList().get(5));
if (!outlierHandlingModeArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java
index 9ba7604d7095..a3fe8dc5458a 100644
--- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java
+++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java
@@ -21,8 +21,6 @@
import com.google.common.collect.ImmutableList;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
@@ -39,14 +37,13 @@
import org.apache.druid.query.aggregation.histogram.ApproximateHistogramFoldingAggregatorFactory;
import org.apache.druid.query.aggregation.histogram.QuantilePostAggregator;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.Aggregations;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
-import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
@@ -67,25 +64,18 @@ public SqlAggFunction calciteFunction()
@Override
public Aggregation toDruidAggregation(
final PlannerContext plannerContext,
- final RowSignature rowSignature,
final VirtualColumnRegistry virtualColumnRegistry,
- final RexBuilder rexBuilder,
final String name,
final AggregateCall aggregateCall,
- final Project project,
+ final InputAccessor inputAccessor,
final List existingAggregations,
final boolean finalizeAggregations
)
{
final DruidExpression input = Aggregations.toDruidExpressionForNumericAggregator(
plannerContext,
- rowSignature,
- Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(0)
- )
+ inputAccessor.getInputRowSignature(),
+ inputAccessor.getField(aggregateCall.getArgList().get(0))
);
if (input == null) {
return null;
@@ -93,12 +83,7 @@ public Aggregation toDruidAggregation(
final AggregatorFactory aggregatorFactory;
final String histogramName = StringUtils.format("%s:agg", name);
- final RexNode probabilityArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(1)
- );
+ final RexNode probabilityArg = inputAccessor.getField(aggregateCall.getArgList().get(1));
if (!probabilityArg.isA(SqlKind.LITERAL)) {
// Probability must be a literal in order to plan.
@@ -109,12 +94,7 @@ public Aggregation toDruidAggregation(
final int resolution;
if (aggregateCall.getArgList().size() >= 3) {
- final RexNode resolutionArg = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(2)
- );
+ final RexNode resolutionArg = inputAccessor.getField(aggregateCall.getArgList().get(2));
if (!resolutionArg.isA(SqlKind.LITERAL)) {
// Resolution must be a literal in order to plan.
@@ -170,7 +150,10 @@ public Aggregation toDruidAggregation(
// No existing match found. Create a new one.
if (input.isDirectColumnAccess()) {
- if (rowSignature.getColumnType(input.getDirectColumn()).map(type -> type.is(ValueType.COMPLEX)).orElse(false)) {
+ if (inputAccessor.getInputRowSignature()
+ .getColumnType(input.getDirectColumn())
+ .map(type -> type.is(ValueType.COMPLEX))
+ .orElse(false)) {
aggregatorFactory = new ApproximateHistogramFoldingAggregatorFactory(
histogramName,
input.getDirectColumn(),
diff --git a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/ExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/ExtractionNamespace.java
index 86eb310b4df9..c52021bd18f7 100644
--- a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/ExtractionNamespace.java
+++ b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/ExtractionNamespace.java
@@ -41,4 +41,12 @@ default long getMaxHeapPercentage()
{
return -1L;
}
+
+ // For larger clusters, when they all startup at the same time and have lookups in the db,
+ // it overwhelms the database, this allows implementations to introduce a jitter, which
+ // should spread out the load.
+ default long getJitterMills()
+ {
+ return 0;
+ }
}
diff --git a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/JdbcExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/JdbcExtractionNamespace.java
index 1495370a4519..32ceccd1a82a 100644
--- a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/JdbcExtractionNamespace.java
+++ b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/JdbcExtractionNamespace.java
@@ -34,6 +34,7 @@
import javax.validation.constraints.Min;
import javax.validation.constraints.NotNull;
import java.util.Objects;
+import java.util.concurrent.ThreadLocalRandom;
/**
*
@@ -61,6 +62,8 @@ public class JdbcExtractionNamespace implements ExtractionNamespace
private final Period pollPeriod;
@JsonProperty
private final long maxHeapPercentage;
+ @JsonProperty
+ private final int jitterSeconds;
@JsonCreator
public JdbcExtractionNamespace(
@@ -73,6 +76,7 @@ public JdbcExtractionNamespace(
@JsonProperty(value = "filter") @Nullable final String filter,
@Min(0) @JsonProperty(value = "pollPeriod") @Nullable final Period pollPeriod,
@JsonProperty(value = "maxHeapPercentage") @Nullable final Long maxHeapPercentage,
+ @JsonProperty(value = "jitterSeconds") @Nullable Integer jitterSeconds,
@JacksonInject JdbcAccessSecurityConfig securityConfig
)
{
@@ -95,6 +99,7 @@ public JdbcExtractionNamespace(
} else {
this.pollPeriod = pollPeriod;
}
+ this.jitterSeconds = jitterSeconds == null ? 0 : jitterSeconds;
this.maxHeapPercentage = maxHeapPercentage == null ? DEFAULT_MAX_HEAP_PERCENTAGE : maxHeapPercentage;
}
@@ -162,6 +167,15 @@ public long getMaxHeapPercentage()
return maxHeapPercentage;
}
+ @Override
+ public long getJitterMills()
+ {
+ if (jitterSeconds == 0) {
+ return jitterSeconds;
+ }
+ return 1000L * ThreadLocalRandom.current().nextInt(jitterSeconds + 1);
+ }
+
@Override
public String toString()
{
diff --git a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java
index 63471afe3d17..61e580563f8c 100644
--- a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java
+++ b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java
@@ -180,9 +180,9 @@ private Future> schedule(final T namespace)
final long updateMs = namespace.getPollMs();
Runnable command = this::updateCache;
if (updateMs > 0) {
- return cacheManager.scheduledExecutorService().scheduleAtFixedRate(command, 0, updateMs, TimeUnit.MILLISECONDS);
+ return cacheManager.scheduledExecutorService().scheduleAtFixedRate(command, namespace.getJitterMills(), updateMs, TimeUnit.MILLISECONDS);
} else {
- return cacheManager.scheduledExecutorService().schedule(command, 0, TimeUnit.MILLISECONDS);
+ return cacheManager.scheduledExecutorService().schedule(command, namespace.getJitterMills(), TimeUnit.MILLISECONDS);
}
}
diff --git a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/query/lookup/namespace/JdbcExtractionNamespaceUrlCheckTest.java b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/query/lookup/namespace/JdbcExtractionNamespaceUrlCheckTest.java
index 44bb67eac00d..f4fffef5fff3 100644
--- a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/query/lookup/namespace/JdbcExtractionNamespaceUrlCheckTest.java
+++ b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/query/lookup/namespace/JdbcExtractionNamespaceUrlCheckTest.java
@@ -63,7 +63,7 @@ public String getConnectURI()
"some filter",
new Period(10),
null,
- new JdbcAccessSecurityConfig()
+ 0, new JdbcAccessSecurityConfig()
{
@Override
public Set getAllowedProperties()
@@ -101,6 +101,7 @@ public String getConnectURI()
"some filter",
new Period(10),
null,
+ 0,
new JdbcAccessSecurityConfig()
{
@Override
@@ -137,6 +138,7 @@ public String getConnectURI()
"some filter",
new Period(10),
null,
+ 0,
new JdbcAccessSecurityConfig()
{
@Override
@@ -175,6 +177,7 @@ public String getConnectURI()
"some filter",
new Period(10),
null,
+ 0,
new JdbcAccessSecurityConfig()
{
@Override
@@ -217,6 +220,7 @@ public String getConnectURI()
"some filter",
new Period(10),
null,
+ 0,
new JdbcAccessSecurityConfig()
{
@Override
@@ -255,6 +259,7 @@ public String getConnectURI()
"some filter",
new Period(10),
10L,
+ 0,
new JdbcAccessSecurityConfig()
{
@Override
@@ -291,7 +296,7 @@ public String getConnectURI()
"some filter",
new Period(10),
null,
- new JdbcAccessSecurityConfig()
+ 0, new JdbcAccessSecurityConfig()
{
@Override
public Set getAllowedProperties()
@@ -329,6 +334,7 @@ public String getConnectURI()
"some filter",
new Period(10),
null,
+ 0,
new JdbcAccessSecurityConfig()
{
@Override
@@ -373,6 +379,7 @@ public String getConnectURI()
"some filter",
new Period(10),
null,
+ 0,
new JdbcAccessSecurityConfig()
{
@Override
@@ -415,6 +422,7 @@ public String getConnectURI()
"some filter",
new Period(10),
null,
+ 0,
new JdbcAccessSecurityConfig()
{
@Override
diff --git a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/JdbcCacheGeneratorTest.java b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/JdbcCacheGeneratorTest.java
index ff27b50fd86e..1eb74630fda0 100644
--- a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/JdbcCacheGeneratorTest.java
+++ b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/JdbcCacheGeneratorTest.java
@@ -137,6 +137,7 @@ private static JdbcExtractionNamespace createJdbcExtractionNamespace(
"filter",
Period.ZERO,
null,
+ 0,
new JdbcAccessSecurityConfig()
);
}
diff --git a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/cache/CacheSchedulerTest.java b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/cache/CacheSchedulerTest.java
index 44289b048339..fd96529ae992 100644
--- a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/cache/CacheSchedulerTest.java
+++ b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/cache/CacheSchedulerTest.java
@@ -458,6 +458,7 @@ public String getConnectURI()
"some filter",
new Period(10_000),
null,
+ 0,
new JdbcAccessSecurityConfig()
{
@Override
diff --git a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java
index b6a37240cea4..e0c651724d75 100644
--- a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java
+++ b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java
@@ -328,6 +328,7 @@ public void testMappingWithoutFilter()
null,
new Period(0),
null,
+ 0,
new JdbcAccessSecurityConfig()
);
try (CacheScheduler.Entry entry = scheduler.schedule(extractionNamespace)) {
@@ -361,6 +362,7 @@ public void testMappingWithFilter()
FILTER_COLUMN + "='1'",
new Period(0),
null,
+ 0,
new JdbcAccessSecurityConfig()
);
try (CacheScheduler.Entry entry = scheduler.schedule(extractionNamespace)) {
@@ -399,6 +401,45 @@ public void testSkipOld()
}
}
+ @Test
+ public void testRandomJitter()
+ {
+ JdbcExtractionNamespace extractionNamespace = new JdbcExtractionNamespace(
+ derbyConnectorRule.getMetadataConnectorConfig(),
+ TABLE_NAME,
+ KEY_NAME,
+ VAL_NAME,
+ tsColumn,
+ FILTER_COLUMN + "='1'",
+ new Period(0),
+ null,
+ 120,
+ new JdbcAccessSecurityConfig()
+ );
+ long jitter = extractionNamespace.getJitterMills();
+ // jitter will be a random value between 0 and 120 seconds.
+ Assert.assertTrue(jitter >= 0 && jitter <= 120000);
+ }
+
+ @Test
+ public void testRandomJitterNotSpecified()
+ {
+ JdbcExtractionNamespace extractionNamespace = new JdbcExtractionNamespace(
+ derbyConnectorRule.getMetadataConnectorConfig(),
+ TABLE_NAME,
+ KEY_NAME,
+ VAL_NAME,
+ tsColumn,
+ FILTER_COLUMN + "='1'",
+ new Period(0),
+ null,
+ null,
+ new JdbcAccessSecurityConfig()
+ );
+ // jitter will be a random value between 0 and 120 seconds.
+ Assert.assertEquals(0, extractionNamespace.getJitterMills());
+ }
+
@Test(timeout = 60_000L)
public void testFindNew()
throws InterruptedException
@@ -436,6 +477,7 @@ public void testSerde() throws IOException
"some filter",
new Period(10),
null,
+ 0,
securityConfig
);
final ObjectMapper mapper = new DefaultObjectMapper();
@@ -461,6 +503,7 @@ private CacheScheduler.Entry ensureEntry()
null,
new Period(10),
null,
+ 0,
new JdbcAccessSecurityConfig()
);
CacheScheduler.Entry entry = scheduler.schedule(extractionNamespace);
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 c7b10f245c1d..6f46007d93c0 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
@@ -24,6 +24,7 @@
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
import com.google.common.util.concurrent.FutureCallback;
@@ -39,6 +40,7 @@
import it.unimi.dsi.fastutil.ints.IntList;
import it.unimi.dsi.fastutil.ints.IntSet;
import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.druid.client.ImmutableSegmentLoadInfo;
import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.data.input.StringTuple;
import org.apache.druid.data.input.impl.DimensionSchema;
@@ -140,6 +142,7 @@
import org.apache.druid.msq.input.stage.StageInputSlice;
import org.apache.druid.msq.input.stage.StageInputSpec;
import org.apache.druid.msq.input.stage.StageInputSpecSlicer;
+import org.apache.druid.msq.input.table.DataSegmentWithLocation;
import org.apache.druid.msq.input.table.TableInputSpec;
import org.apache.druid.msq.input.table.TableInputSpecSlicer;
import org.apache.druid.msq.kernel.GlobalSortTargetSizeShuffleSpec;
@@ -164,6 +167,7 @@
import org.apache.druid.msq.shuffle.input.DurableStorageInputChannelFactory;
import org.apache.druid.msq.shuffle.input.WorkerInputChannelFactory;
import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation;
+import org.apache.druid.msq.util.ArrayIngestMode;
import org.apache.druid.msq.util.DimensionSchemaUtils;
import org.apache.druid.msq.util.IntervalUtils;
import org.apache.druid.msq.util.MSQFutureUtils;
@@ -187,6 +191,7 @@
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.DruidNode;
+import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.sql.calcite.planner.ColumnMapping;
import org.apache.druid.sql.calcite.planner.ColumnMappings;
import org.apache.druid.sql.calcite.rel.DruidQuery;
@@ -1163,14 +1168,73 @@ private QueryKit makeQueryControllerToolKit()
private DataSegmentTimelineView makeDataSegmentTimelineView()
{
+ final SegmentSource includeSegmentSource = MultiStageQueryContext.getSegmentSources(
+ task.getQuerySpec()
+ .getQuery()
+ .context()
+ );
+
+ final boolean includeRealtime = SegmentSource.shouldQueryRealtimeServers(includeSegmentSource);
+
return (dataSource, intervals) -> {
- final Collection dataSegments =
+ final Iterable realtimeAndHistoricalSegments;
+
+ // Fetch the realtime segments and segments loaded on the historical. Do this first so that we don't miss any
+ // segment if they get handed off between the two calls. Segments loaded on historicals are deduplicated below,
+ // since we are only interested in realtime segments for now.
+ if (includeRealtime) {
+ realtimeAndHistoricalSegments = context.coordinatorClient().fetchServerViewSegments(dataSource, intervals);
+ } else {
+ realtimeAndHistoricalSegments = ImmutableList.of();
+ }
+
+ // Fetch all published, used segments (all non-realtime segments) from the metadata store.
+ final Collection publishedUsedSegments =
FutureUtils.getUnchecked(context.coordinatorClient().fetchUsedSegments(dataSource, intervals), true);
- if (dataSegments.isEmpty()) {
+ int realtimeCount = 0;
+
+ // Deduplicate segments, giving preference to published used segments.
+ // We do this so that if any segments have been handed off in between the two metadata calls above,
+ // we directly fetch it from deep storage.
+ Set unifiedSegmentView = new HashSet<>(publishedUsedSegments);
+
+ // Iterate over the realtime segments and segments loaded on the historical
+ for (ImmutableSegmentLoadInfo segmentLoadInfo : realtimeAndHistoricalSegments) {
+ ImmutableSet servers = segmentLoadInfo.getServers();
+ // Filter out only realtime servers. We don't want to query historicals for now, but we can in the future.
+ // This check can be modified then.
+ Set realtimeServerMetadata
+ = servers.stream()
+ .filter(druidServerMetadata -> includeSegmentSource.getUsedServerTypes()
+ .contains(druidServerMetadata.getType())
+ )
+ .collect(Collectors.toSet());
+ if (!realtimeServerMetadata.isEmpty()) {
+ realtimeCount += 1;
+ DataSegmentWithLocation dataSegmentWithLocation = new DataSegmentWithLocation(
+ segmentLoadInfo.getSegment(),
+ realtimeServerMetadata
+ );
+ unifiedSegmentView.add(dataSegmentWithLocation);
+ } else {
+ // We don't have any segments of the required segment source, ignore the segment
+ }
+ }
+
+ if (includeRealtime) {
+ log.info(
+ "Fetched total [%d] segments from coordinator: [%d] from metadata stoure, [%d] from server view",
+ unifiedSegmentView.size(),
+ publishedUsedSegments.size(),
+ realtimeCount
+ );
+ }
+
+ if (unifiedSegmentView.isEmpty()) {
return Optional.empty();
} else {
- return Optional.of(SegmentTimeline.forSegments(dataSegments));
+ return Optional.of(SegmentTimeline.forSegments(unifiedSegmentView));
}
};
}
@@ -1936,6 +2000,17 @@ private static Pair, List> makeDimensio
final Query> query
)
{
+ // Log a warning unconditionally if arrayIngestMode is MVD, since the behaviour is incorrect, and is subject to
+ // deprecation and removal in future
+ if (MultiStageQueryContext.getArrayIngestMode(query.context()) == ArrayIngestMode.MVD) {
+ log.warn(
+ "'%s' is set to 'mvd' in the query's context. This ingests the string arrays as multi-value "
+ + "strings instead of arrays, and is preserved for legacy reasons when MVDs were the only way to ingest string "
+ + "arrays in Druid. It is incorrect behaviour and will likely be removed in the future releases of Druid",
+ MultiStageQueryContext.CTX_ARRAY_INGEST_MODE
+ );
+ }
+
final List dimensions = new ArrayList<>();
final List aggregators = new ArrayList<>();
@@ -2013,7 +2088,8 @@ private static Pair, List> makeDimensio
DimensionSchemaUtils.createDimensionSchema(
outputColumnName,
type,
- MultiStageQueryContext.useAutoColumnSchemas(query.context())
+ MultiStageQueryContext.useAutoColumnSchemas(query.context()),
+ MultiStageQueryContext.getArrayIngestMode(query.context())
)
);
} else if (!isRollupQuery) {
@@ -2062,7 +2138,8 @@ private static void populateDimensionsAndAggregators(
DimensionSchemaUtils.createDimensionSchema(
outputColumn,
type,
- MultiStageQueryContext.useAutoColumnSchemas(context)
+ MultiStageQueryContext.useAutoColumnSchemas(context),
+ MultiStageQueryContext.getArrayIngestMode(context)
)
);
}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java
new file mode 100644
index 000000000000..d9d789e3d2ba
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProvider.java
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.exec;
+
+import com.fasterxml.jackson.databind.JavaType;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.client.coordinator.CoordinatorClient;
+import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.discovery.DataServerClient;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.RetryUtils;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.guava.Yielders;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.msq.counters.ChannelCounters;
+import org.apache.druid.msq.input.table.RichSegmentDescriptor;
+import org.apache.druid.query.Queries;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryInterruptedException;
+import org.apache.druid.query.QueryToolChest;
+import org.apache.druid.query.QueryToolChestWarehouse;
+import org.apache.druid.query.SegmentDescriptor;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.aggregation.MetricManipulationFn;
+import org.apache.druid.query.aggregation.MetricManipulatorFns;
+import org.apache.druid.query.context.DefaultResponseContext;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.rpc.FixedSetServiceLocator;
+import org.apache.druid.rpc.RpcException;
+import org.apache.druid.rpc.ServiceClientFactory;
+import org.apache.druid.rpc.ServiceLocation;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.utils.CollectionUtils;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.Function;
+
+/**
+ * Class responsible for querying dataservers and retriving results for a given query. Also queries the coordinator
+ * to check if a segment has been handed off.
+ */
+public class LoadedSegmentDataProvider
+{
+ private static final Logger log = new Logger(LoadedSegmentDataProvider.class);
+ private static final int DEFAULT_NUM_TRIES = 5;
+ private final String dataSource;
+ private final ChannelCounters channelCounters;
+ private final ServiceClientFactory serviceClientFactory;
+ private final CoordinatorClient coordinatorClient;
+ private final ObjectMapper objectMapper;
+ private final QueryToolChestWarehouse warehouse;
+ private final ScheduledExecutorService queryCancellationExecutor;
+
+ public LoadedSegmentDataProvider(
+ String dataSource,
+ ChannelCounters channelCounters,
+ ServiceClientFactory serviceClientFactory,
+ CoordinatorClient coordinatorClient,
+ ObjectMapper objectMapper,
+ QueryToolChestWarehouse warehouse,
+ ScheduledExecutorService queryCancellationExecutor
+ )
+ {
+ this.dataSource = dataSource;
+ this.channelCounters = channelCounters;
+ this.serviceClientFactory = serviceClientFactory;
+ this.coordinatorClient = coordinatorClient;
+ this.objectMapper = objectMapper;
+ this.warehouse = warehouse;
+ this.queryCancellationExecutor = queryCancellationExecutor;
+ }
+
+ @VisibleForTesting
+ DataServerClient makeDataServerClient(ServiceLocation serviceLocation)
+ {
+ return new DataServerClient(serviceClientFactory, serviceLocation, objectMapper, queryCancellationExecutor);
+ }
+
+ /**
+ * Performs some necessary transforms to the query, so that the dataserver is able to understand it first.
+ * - Changing the datasource to a {@link TableDataSource}
+ * - Limiting the query to a single required segment with {@link Queries#withSpecificSegments(Query, List)}
+ *
+ * Then queries a data server and returns a {@link Yielder} for the results, retrying if needed. If a dataserver
+ * indicates that the segment was not found, checks with the coordinator to see if the segment was handed off.
+ * - If the segment was handed off, returns with a {@link DataServerQueryStatus#HANDOFF} status.
+ * - If the segment was not handed off, retries with the known list of servers and throws an exception if the retry
+ * count is exceeded.
+ * - If the servers could not be found, checks if the segment was handed-off. If it was, returns with a
+ * {@link DataServerQueryStatus#HANDOFF} status. Otherwise, throws an exception.
+ *
+ * Also applies {@link QueryToolChest#makePreComputeManipulatorFn(Query, MetricManipulationFn)} and reports channel
+ * metrics on the returned results.
+ *
+ * @param result return type for the query from the data server
+ * @param type of the result rows after parsing from QueryType object
+ */
+ public Pair> fetchRowsFromDataServer(
+ Query query,
+ RichSegmentDescriptor segmentDescriptor,
+ Function, Sequence> mappingFunction,
+ Closer closer
+ ) throws IOException
+ {
+ final Query preparedQuery = Queries.withSpecificSegments(
+ query.withDataSource(new TableDataSource(dataSource)),
+ ImmutableList.of(segmentDescriptor)
+ );
+
+ final Set servers = segmentDescriptor.getServers();
+ final FixedSetServiceLocator fixedSetServiceLocator = FixedSetServiceLocator.forDruidServerMetadata(servers);
+ final QueryToolChest> toolChest = warehouse.getToolChest(query);
+ final Function preComputeManipulatorFn =
+ toolChest.makePreComputeManipulatorFn(query, MetricManipulatorFns.deserializing());
+
+ final JavaType queryResultType = toolChest.getBaseResultType();
+ final int numRetriesOnMissingSegments = preparedQuery.context().getNumRetriesOnMissingSegments(DEFAULT_NUM_TRIES);
+
+ log.debug("Querying severs[%s] for segment[%s], retries:[%d]", servers, segmentDescriptor, numRetriesOnMissingSegments);
+ final ResponseContext responseContext = new DefaultResponseContext();
+
+ Pair> statusSequencePair;
+ try {
+ // We need to check for handoff to decide if we need to retry. Therefore, we handle it here instead of inside
+ // the client.
+ statusSequencePair = RetryUtils.retry(
+ () -> {
+ ServiceLocation serviceLocation = CollectionUtils.getOnlyElement(
+ fixedSetServiceLocator.locate().get().getLocations(),
+ serviceLocations -> {
+ throw DruidException.defensive("Should only have one location");
+ }
+ );
+ DataServerClient dataServerClient = makeDataServerClient(serviceLocation);
+ Sequence sequence = dataServerClient.run(preparedQuery, responseContext, queryResultType, closer)
+ .map(preComputeManipulatorFn);
+ final List missingSegments = getMissingSegments(responseContext);
+ // Only one segment is fetched, so this should be empty if it was fetched
+ if (missingSegments.isEmpty()) {
+ log.debug("Successfully fetched rows from server for segment[%s]", segmentDescriptor);
+ // Segment was found
+ Yielder yielder = closer.register(
+ Yielders.each(mappingFunction.apply(sequence)
+ .map(row -> {
+ channelCounters.incrementRowCount();
+ return row;
+ }))
+ );
+ return Pair.of(DataServerQueryStatus.SUCCESS, yielder);
+ } else {
+ Boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor);
+ if (Boolean.TRUE.equals(wasHandedOff)) {
+ log.debug("Segment[%s] was handed off.", segmentDescriptor);
+ return Pair.of(DataServerQueryStatus.HANDOFF, null);
+ } else {
+ log.error("Segment[%s] could not be found on data server, but segment was not handed off.", segmentDescriptor);
+ throw new IOE(
+ "Segment[%s] could not be found on data server, but segment was not handed off.",
+ segmentDescriptor
+ );
+ }
+ }
+ },
+ throwable -> !(throwable instanceof QueryInterruptedException && throwable.getCause() instanceof InterruptedException),
+ numRetriesOnMissingSegments
+ );
+
+ return statusSequencePair;
+ }
+ catch (QueryInterruptedException e) {
+ if (e.getCause() instanceof RpcException) {
+ // In the case that all the realtime servers for a segment are gone (for example, if they were scaled down),
+ // we would also be unable to fetch the segment. Check if the segment was handed off, just in case, instead of
+ // failing the query.
+ boolean wasHandedOff = checkSegmentHandoff(coordinatorClient, dataSource, segmentDescriptor);
+ if (wasHandedOff) {
+ log.debug("Segment[%s] was handed off.", segmentDescriptor);
+ return Pair.of(DataServerQueryStatus.HANDOFF, null);
+ }
+ }
+ throw DruidException.forPersona(DruidException.Persona.OPERATOR)
+ .ofCategory(DruidException.Category.RUNTIME_FAILURE)
+ .build(e, "Exception while fetching rows for query from dataservers[%s]", servers);
+ }
+ catch (Exception e) {
+ Throwables.propagateIfPossible(e, IOE.class);
+ throw DruidException.forPersona(DruidException.Persona.OPERATOR)
+ .ofCategory(DruidException.Category.RUNTIME_FAILURE)
+ .build(e, "Exception while fetching rows for query from dataservers[%s]", servers);
+ }
+ }
+
+ /**
+ * Retreives the list of missing segments from the response context.
+ */
+ private static List getMissingSegments(final ResponseContext responseContext)
+ {
+ List missingSegments = responseContext.getMissingSegments();
+ if (missingSegments == null) {
+ return ImmutableList.of();
+ }
+ return missingSegments;
+ }
+
+ /**
+ * Queries the coordinator to check if a segment has been handed off.
+ *
+ * See {@link org.apache.druid.server.http.DataSourcesResource#isHandOffComplete(String, String, int, String)}
+ */
+ private static boolean checkSegmentHandoff(
+ CoordinatorClient coordinatorClient,
+ String dataSource,
+ SegmentDescriptor segmentDescriptor
+ ) throws IOE
+ {
+ Boolean wasHandedOff;
+ try {
+ wasHandedOff = FutureUtils.get(coordinatorClient.isHandoffComplete(dataSource, segmentDescriptor), true);
+ }
+ catch (Exception e) {
+ throw new IOE(e, "Could not contact coordinator for segment[%s]", segmentDescriptor);
+ }
+ return Boolean.TRUE.equals(wasHandedOff);
+ }
+
+ /**
+ * Represents the status of fetching a segment from a data server
+ */
+ public enum DataServerQueryStatus
+ {
+ /**
+ * Segment was found on the data server and fetched successfully.
+ */
+ SUCCESS,
+ /**
+ * Segment was not found on the realtime server as it has been handed off to a historical. Only returned while
+ * querying a realtime server.
+ */
+ HANDOFF
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java
new file mode 100644
index 000000000000..48ed57be8701
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderFactory.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.exec;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.client.coordinator.CoordinatorClient;
+import org.apache.druid.java.util.common.RE;
+import org.apache.druid.java.util.common.concurrent.ScheduledExecutors;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.msq.counters.ChannelCounters;
+import org.apache.druid.query.QueryToolChestWarehouse;
+import org.apache.druid.rpc.ServiceClientFactory;
+
+import java.io.Closeable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Creates new instances of {@link LoadedSegmentDataProvider} and manages the cancellation threadpool.
+ */
+public class LoadedSegmentDataProviderFactory implements Closeable
+{
+ private static final Logger log = new Logger(LoadedSegmentDataProviderFactory.class);
+ private static final int DEFAULT_THREAD_COUNT = 4;
+ private final CoordinatorClient coordinatorClient;
+ private final ServiceClientFactory serviceClientFactory;
+ private final ObjectMapper objectMapper;
+ private final QueryToolChestWarehouse warehouse;
+ private final ScheduledExecutorService queryCancellationExecutor;
+
+ public LoadedSegmentDataProviderFactory(
+ CoordinatorClient coordinatorClient,
+ ServiceClientFactory serviceClientFactory,
+ ObjectMapper objectMapper,
+ QueryToolChestWarehouse warehouse
+ )
+ {
+ this.coordinatorClient = coordinatorClient;
+ this.serviceClientFactory = serviceClientFactory;
+ this.objectMapper = objectMapper;
+ this.warehouse = warehouse;
+ this.queryCancellationExecutor = ScheduledExecutors.fixed(DEFAULT_THREAD_COUNT, "query-cancellation-executor");
+ }
+
+ public LoadedSegmentDataProvider createLoadedSegmentDataProvider(
+ String dataSource,
+ ChannelCounters channelCounters
+ )
+ {
+ return new LoadedSegmentDataProvider(
+ dataSource,
+ channelCounters,
+ serviceClientFactory,
+ coordinatorClient,
+ objectMapper,
+ warehouse,
+ queryCancellationExecutor
+ );
+ }
+
+ @Override
+ public void close()
+ {
+ // Wait for all query cancellations to be complete.
+ log.info("Waiting for any data server queries to be canceled.");
+ queryCancellationExecutor.shutdown();
+ try {
+ if (!queryCancellationExecutor.awaitTermination(1, TimeUnit.MINUTES)) {
+ log.error("Unable to cancel all ongoing queries.");
+ }
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RE(e);
+ }
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java
new file mode 100644
index 000000000000..22f3a5df973c
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentSource.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.exec;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.server.coordination.ServerType;
+
+import java.util.Set;
+
+/**
+ * Decides the types of data servers contacted by MSQ tasks to fetch results.
+ */
+public enum SegmentSource
+{
+ /**
+ * Include only segments from deep storage.
+ */
+ NONE(ImmutableSet.of()),
+ /**
+ * Include segments from realtime tasks as well as segments from deep storage.
+ */
+ REALTIME(ImmutableSet.of(ServerType.REALTIME, ServerType.INDEXER_EXECUTOR));
+
+ /**
+ * The type of dataservers (if any) to include. This does not include segments queried from deep storage, which are
+ * always included in queries.
+ */
+ private final Set usedServerTypes;
+
+ SegmentSource(Set usedServerTypes)
+ {
+ this.usedServerTypes = usedServerTypes;
+ }
+
+ public Set getUsedServerTypes()
+ {
+ return usedServerTypes;
+ }
+
+ /**
+ * Whether realtime servers should be included for the segmentSource.
+ */
+ public static boolean shouldQueryRealtimeServers(SegmentSource segmentSource)
+ {
+ return REALTIME.equals(segmentSource);
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java
index d017feb099fb..a3d4fde6c1a5 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java
@@ -74,6 +74,7 @@ public interface WorkerContext
DruidNode selfNode();
Bouncer processorBouncer();
+ LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory();
default File tempDir(int stageNumber, String id)
{
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java
index d522c3a7f169..6ee45bc158e8 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java
@@ -295,6 +295,7 @@ public Optional runTask(final Closer closer) throws Exception
{
this.controllerClient = context.makeControllerClient(task.getControllerTaskId());
closer.register(controllerClient::close);
+ closer.register(context.loadedSegmentDataProviderFactory());
context.registerWorker(this, closer); // Uses controllerClient, so must be called after that is initialized
this.workerClient = new ExceptionWrappingWorkerClient(context.makeWorkerClient());
@@ -1103,7 +1104,7 @@ private void makeInputSliceReader()
.put(
SegmentsInputSlice.class,
new SegmentsInputSliceReader(
- frameContext.dataSegmentProvider(),
+ frameContext,
MultiStageQueryContext.isReindex(QueryContext.of(task().getContext()))
)
)
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java
index c35832992f93..d522a8a7d88f 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java
@@ -20,6 +20,7 @@
package org.apache.druid.msq.indexing;
import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory;
import org.apache.druid.msq.exec.WorkerMemoryParameters;
import org.apache.druid.msq.kernel.FrameContext;
import org.apache.druid.msq.querykit.DataSegmentProvider;
@@ -38,17 +39,20 @@ public class IndexerFrameContext implements FrameContext
private final IndexIO indexIO;
private final DataSegmentProvider dataSegmentProvider;
private final WorkerMemoryParameters memoryParameters;
+ private final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory;
public IndexerFrameContext(
IndexerWorkerContext context,
IndexIO indexIO,
DataSegmentProvider dataSegmentProvider,
+ LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory,
WorkerMemoryParameters memoryParameters
)
{
this.context = context;
this.indexIO = indexIO;
this.dataSegmentProvider = dataSegmentProvider;
+ this.loadedSegmentDataProviderFactory = loadedSegmentDataProviderFactory;
this.memoryParameters = memoryParameters;
}
@@ -76,6 +80,13 @@ public DataSegmentProvider dataSegmentProvider()
return dataSegmentProvider;
}
+ @Override
+ public LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory()
+ {
+ return loadedSegmentDataProviderFactory;
+ }
+
+
@Override
public File tempDir()
{
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java
index 43d067dd6c90..709b019891f0 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java
@@ -27,12 +27,14 @@
import org.apache.druid.frame.processor.Bouncer;
import org.apache.druid.guice.annotations.EscalatedGlobal;
import org.apache.druid.guice.annotations.Self;
+import org.apache.druid.guice.annotations.Smile;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.msq.exec.ControllerClient;
+import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory;
import org.apache.druid.msq.exec.TaskDataSegmentProvider;
import org.apache.druid.msq.exec.Worker;
import org.apache.druid.msq.exec.WorkerClient;
@@ -43,6 +45,7 @@
import org.apache.druid.msq.indexing.client.WorkerChatHandler;
import org.apache.druid.msq.kernel.FrameContext;
import org.apache.druid.msq.kernel.QueryDefinition;
+import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.rpc.ServiceClientFactory;
import org.apache.druid.rpc.ServiceLocations;
import org.apache.druid.rpc.ServiceLocator;
@@ -68,6 +71,7 @@ public class IndexerWorkerContext implements WorkerContext
private final Injector injector;
private final IndexIO indexIO;
private final TaskDataSegmentProvider dataSegmentProvider;
+ private final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory;
private final ServiceClientFactory clientFactory;
@GuardedBy("this")
@@ -81,6 +85,7 @@ public IndexerWorkerContext(
final Injector injector,
final IndexIO indexIO,
final TaskDataSegmentProvider dataSegmentProvider,
+ final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory,
final ServiceClientFactory clientFactory
)
{
@@ -88,6 +93,7 @@ public IndexerWorkerContext(
this.injector = injector;
this.indexIO = indexIO;
this.dataSegmentProvider = dataSegmentProvider;
+ this.loadedSegmentDataProviderFactory = loadedSegmentDataProviderFactory;
this.clientFactory = clientFactory;
}
@@ -99,12 +105,24 @@ public static IndexerWorkerContext createProductionInstance(final TaskToolbox to
.manufacturate(new File(toolbox.getIndexingTmpDir(), "segment-fetch"));
final ServiceClientFactory serviceClientFactory =
injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class));
+ final ObjectMapper smileMapper = injector.getInstance(Key.get(ObjectMapper.class, Smile.class));
+ final QueryToolChestWarehouse warehouse = injector.getInstance(QueryToolChestWarehouse.class);
return new IndexerWorkerContext(
toolbox,
injector,
indexIO,
- new TaskDataSegmentProvider(toolbox.getCoordinatorClient(), segmentCacheManager, indexIO),
+ new TaskDataSegmentProvider(
+ toolbox.getCoordinatorClient(),
+ segmentCacheManager,
+ indexIO
+ ),
+ new LoadedSegmentDataProviderFactory(
+ toolbox.getCoordinatorClient(),
+ serviceClientFactory,
+ smileMapper,
+ warehouse
+ ),
serviceClientFactory
);
}
@@ -227,6 +245,7 @@ public FrameContext frameContext(QueryDefinition queryDef, int stageNumber)
this,
indexIO,
dataSegmentProvider,
+ loadedSegmentDataProviderFactory,
WorkerMemoryParameters.createProductionInstanceForWorker(injector, queryDef, stageNumber)
);
}
@@ -249,6 +268,12 @@ public Bouncer processorBouncer()
return injector.getInstance(Bouncer.class);
}
+ @Override
+ public LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory()
+ {
+ return loadedSegmentDataProviderFactory;
+ }
+
private synchronized OverlordClient makeOverlordClient()
{
if (overlordClient == null) {
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java
index 3dbd3da0a026..714e8dc3a639 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java
@@ -42,6 +42,7 @@
import org.apache.druid.msq.input.NilInputSource;
import org.apache.druid.msq.input.ReadableInput;
import org.apache.druid.msq.input.ReadableInputs;
+import org.apache.druid.msq.input.table.RichSegmentDescriptor;
import org.apache.druid.msq.input.table.SegmentWithDescriptor;
import org.apache.druid.msq.util.DimensionSchemaUtils;
import org.apache.druid.segment.RowBasedSegment;
@@ -118,10 +119,9 @@ private static Iterator inputSourceSegmentIterator(
new DimensionsSpec(
signature.getColumnNames().stream().map(
column ->
- DimensionSchemaUtils.createDimensionSchema(
+ DimensionSchemaUtils.createDimensionSchemaForExtern(
column,
- signature.getColumnType(column).orElse(null),
- false
+ signature.getColumnType(column).orElse(null)
)
).collect(Collectors.toList())
),
@@ -159,7 +159,8 @@ private static Iterator inputSourceSegmentIterator(
);
return new SegmentWithDescriptor(
() -> ResourceHolder.fromCloseable(segment),
- segmentId.toDescriptor()
+ null,
+ new RichSegmentDescriptor(segmentId.toDescriptor(), null, null)
);
}
);
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java
index 143fb49692f2..25f06c7cd40c 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java
@@ -27,9 +27,9 @@
import org.apache.druid.msq.input.InputSliceReader;
import org.apache.druid.msq.input.ReadableInput;
import org.apache.druid.msq.input.ReadableInputs;
+import org.apache.druid.msq.input.table.RichSegmentDescriptor;
import org.apache.druid.msq.input.table.SegmentWithDescriptor;
import org.apache.druid.query.InlineDataSource;
-import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.segment.InlineSegmentWrangler;
import org.apache.druid.segment.SegmentWrangler;
import org.apache.druid.timeline.SegmentId;
@@ -43,7 +43,8 @@
public class InlineInputSliceReader implements InputSliceReader
{
public static final String SEGMENT_ID = "__inline";
- private static final SegmentDescriptor DUMMY_SEGMENT_DESCRIPTOR = SegmentId.dummy(SEGMENT_ID).toDescriptor();
+ private static final RichSegmentDescriptor DUMMY_SEGMENT_DESCRIPTOR
+ = new RichSegmentDescriptor(SegmentId.dummy(SEGMENT_ID).toDescriptor(), null, null);
private final SegmentWrangler segmentWrangler;
@@ -74,6 +75,7 @@ public ReadableInputs attach(
segment -> ReadableInput.segment(
new SegmentWithDescriptor(
() -> ResourceHolder.fromCloseable(segment),
+ null,
DUMMY_SEGMENT_DESCRIPTOR
)
)
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java
index 648527ce0061..b601b043ac13 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java
@@ -29,6 +29,7 @@
import org.apache.druid.msq.input.InputSliceReader;
import org.apache.druid.msq.input.ReadableInput;
import org.apache.druid.msq.input.ReadableInputs;
+import org.apache.druid.msq.input.table.RichSegmentDescriptor;
import org.apache.druid.msq.input.table.SegmentWithDescriptor;
import org.apache.druid.query.LookupDataSource;
import org.apache.druid.segment.Segment;
@@ -99,7 +100,8 @@ public ReadableInputs attach(
return ResourceHolder.fromCloseable(segment);
},
- SegmentId.dummy(lookupName).toDescriptor()
+ null,
+ new RichSegmentDescriptor(SegmentId.dummy(lookupName).toDescriptor(), null, null)
)
)
)
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java
new file mode 100644
index 000000000000..0e83e9c3edee
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.input.table;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.google.common.base.Preconditions;
+import org.apache.druid.jackson.CommaListJoinDeserializer;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.timeline.CompactionState;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.partition.ShardSpec;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Data segment including the locations which contain the segment. Used if MSQ needs to fetch the segment from a server
+ * instead of from deep storage.
+ */
+public class DataSegmentWithLocation extends DataSegment
+{
+ private final Set servers;
+
+ @JsonCreator
+ public DataSegmentWithLocation(
+ @JsonProperty("dataSource") String dataSource,
+ @JsonProperty("interval") Interval interval,
+ @JsonProperty("version") String version,
+ // use `Map` *NOT* `LoadSpec` because we want to do lazy materialization to prevent dependency pollution
+ @JsonProperty("loadSpec") @Nullable Map loadSpec,
+ @JsonProperty("dimensions")
+ @JsonDeserialize(using = CommaListJoinDeserializer.class)
+ @Nullable
+ List dimensions,
+ @JsonProperty("metrics")
+ @JsonDeserialize(using = CommaListJoinDeserializer.class)
+ @Nullable
+ List metrics,
+ @JsonProperty("shardSpec") @Nullable ShardSpec shardSpec,
+ @JsonProperty("lastCompactionState") @Nullable CompactionState lastCompactionState,
+ @JsonProperty("binaryVersion") Integer binaryVersion,
+ @JsonProperty("size") long size,
+ @JsonProperty("servers") Set servers,
+ @JacksonInject PruneSpecsHolder pruneSpecsHolder
+ )
+ {
+ super(dataSource, interval, version, loadSpec, dimensions, metrics, shardSpec, lastCompactionState, binaryVersion, size, pruneSpecsHolder);
+ this.servers = Preconditions.checkNotNull(servers, "servers");
+ }
+
+ public DataSegmentWithLocation(
+ DataSegment dataSegment,
+ Set servers
+ )
+ {
+ super(
+ dataSegment.getDataSource(),
+ dataSegment.getInterval(),
+ dataSegment.getVersion(),
+ dataSegment.getLoadSpec(),
+ dataSegment.getDimensions(),
+ dataSegment.getMetrics(),
+ dataSegment.getShardSpec(),
+ dataSegment.getBinaryVersion(),
+ dataSegment.getSize()
+ );
+ this.servers = servers;
+ }
+
+ @JsonProperty("servers")
+ public Set getServers()
+ {
+ return servers;
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java
index 3ca48ef9cbdf..04e4e601b073 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/RichSegmentDescriptor.java
@@ -23,36 +23,54 @@
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
import org.apache.druid.query.SegmentDescriptor;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.utils.CollectionUtils;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.Objects;
+import java.util.Set;
/**
- * Like {@link SegmentDescriptor}, but provides both the full interval and the clipped interval for a segment.
- * (SegmentDescriptor only provides the clipped interval.)
- *
+ * Like {@link SegmentDescriptor}, but provides both the full interval and the clipped interval for a segment
+ * (SegmentDescriptor only provides the clipped interval.), as well as the metadata of the servers it is loaded on.
+ *
* To keep the serialized form lightweight, the full interval is only serialized if it is different from the
* clipped interval.
- *
+ *
* It is possible to deserialize this class as {@link SegmentDescriptor}. However, going the other direction is
- * not a good idea, because the {@link #fullInterval} will not end up being set correctly.
+ * not a good idea, because the {@link #fullInterval} and {@link #servers} will not end up being set correctly.
*/
public class RichSegmentDescriptor extends SegmentDescriptor
{
@Nullable
private final Interval fullInterval;
+ private final Set servers;
public RichSegmentDescriptor(
final Interval fullInterval,
final Interval interval,
final String version,
- final int partitionNumber
+ final int partitionNumber,
+ final Set servers
)
{
super(interval, version, partitionNumber);
this.fullInterval = interval.equals(Preconditions.checkNotNull(fullInterval, "fullInterval")) ? null : fullInterval;
+ this.servers = servers;
+ }
+
+ public RichSegmentDescriptor(
+ SegmentDescriptor segmentDescriptor,
+ @Nullable Interval fullInterval,
+ Set servers
+ )
+ {
+ super(segmentDescriptor.getInterval(), segmentDescriptor.getVersion(), segmentDescriptor.getPartitionNumber());
+ this.fullInterval = fullInterval;
+ this.servers = servers;
}
@JsonCreator
@@ -60,17 +78,33 @@ static RichSegmentDescriptor fromJson(
@JsonProperty("fi") @Nullable final Interval fullInterval,
@JsonProperty("itvl") final Interval interval,
@JsonProperty("ver") final String version,
- @JsonProperty("part") final int partitionNumber
+ @JsonProperty("part") final int partitionNumber,
+ @JsonProperty("servers") @Nullable final Set servers
)
{
return new RichSegmentDescriptor(
fullInterval != null ? fullInterval : interval,
interval,
version,
- partitionNumber
+ partitionNumber,
+ servers == null ? ImmutableSet.of() : servers
);
}
+ /**
+ * Returns true if the location the segment is loaded is available, and false if it is not.
+ */
+ public boolean isLoadedOnServer()
+ {
+ return !CollectionUtils.isNullOrEmpty(getServers());
+ }
+
+ @JsonProperty("servers")
+ public Set getServers()
+ {
+ return servers;
+ }
+
public Interval getFullInterval()
{
return fullInterval == null ? getInterval() : fullInterval;
@@ -97,13 +131,13 @@ public boolean equals(Object o)
return false;
}
RichSegmentDescriptor that = (RichSegmentDescriptor) o;
- return Objects.equals(fullInterval, that.fullInterval);
+ return Objects.equals(fullInterval, that.fullInterval) && Objects.equals(servers, that.servers);
}
@Override
public int hashCode()
{
- return Objects.hash(super.hashCode(), fullInterval);
+ return Objects.hash(super.hashCode(), fullInterval, servers);
}
@Override
@@ -111,6 +145,7 @@ public String toString()
{
return "RichSegmentDescriptor{" +
"fullInterval=" + (fullInterval == null ? getInterval() : fullInterval) +
+ ", servers=" + getServers() +
", interval=" + getInterval() +
", version='" + getVersion() + '\'' +
", partitionNumber=" + getPartitionNumber() +
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java
index 020b9f2a5bb0..137129ed338b 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java
@@ -21,10 +21,19 @@
import com.google.common.base.Preconditions;
import org.apache.druid.collections.ResourceHolder;
-import org.apache.druid.query.SegmentDescriptor;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.msq.exec.LoadedSegmentDataProvider;
+import org.apache.druid.query.Query;
import org.apache.druid.segment.Segment;
+import javax.annotation.Nullable;
+import java.io.IOException;
import java.util.Objects;
+import java.util.function.Function;
import java.util.function.Supplier;
/**
@@ -33,30 +42,36 @@
public class SegmentWithDescriptor
{
private final Supplier extends ResourceHolder> segmentSupplier;
- private final SegmentDescriptor descriptor;
+ @Nullable
+ private final LoadedSegmentDataProvider loadedSegmentDataProvider;
+ private final RichSegmentDescriptor descriptor;
/**
* Create a new instance.
*
- * @param segmentSupplier supplier of a {@link ResourceHolder} of segment. The {@link ResourceHolder#close()} logic
- * must include a delegated call to {@link Segment#close()}.
- * @param descriptor segment descriptor
+ * @param segmentSupplier supplier of a {@link ResourceHolder} of segment. The {@link ResourceHolder#close()}
+ * logic must include a delegated call to {@link Segment#close()}.
+ * @param loadedSegmentDataProvider {@link LoadedSegmentDataProvider} which fetches the corresponding results from a
+ * data server where the segment is loaded. The call will fetch the
+ * @param descriptor segment descriptor
*/
public SegmentWithDescriptor(
final Supplier extends ResourceHolder> segmentSupplier,
- final SegmentDescriptor descriptor
+ final @Nullable LoadedSegmentDataProvider loadedSegmentDataProvider,
+ final RichSegmentDescriptor descriptor
)
{
this.segmentSupplier = Preconditions.checkNotNull(segmentSupplier, "segment");
+ this.loadedSegmentDataProvider = loadedSegmentDataProvider;
this.descriptor = Preconditions.checkNotNull(descriptor, "descriptor");
}
/**
* The physical segment.
- *
+ *
* Named "getOrLoad" because the segment may be generated by a lazy supplier. In this case, the segment is acquired
* as part of the call to this method.
- *
+ *
* It is not necessary to call {@link Segment#close()} on the returned segment. Calling {@link ResourceHolder#close()}
* is enough.
*/
@@ -65,10 +80,23 @@ public ResourceHolder getOrLoad()
return segmentSupplier.get();
}
+ public Pair> fetchRowsFromDataServer(
+ Query query,
+ Function, Sequence> mappingFunction,
+ Closer closer
+ ) throws IOException
+ {
+ if (loadedSegmentDataProvider == null) {
+ throw DruidException.defensive("loadedSegmentDataProvider was null. Fetching segments from servers is not "
+ + "supported for segment[%s]", descriptor);
+ }
+ return loadedSegmentDataProvider.fetchRowsFromDataServer(query, descriptor, mappingFunction, closer);
+ }
+
/**
* The segment descriptor associated with this physical segment.
*/
- public SegmentDescriptor getDescriptor()
+ public RichSegmentDescriptor getDescriptor()
{
return descriptor;
}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java
index 5334c4cb2abd..8bc67dbb4e88 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSliceReader.java
@@ -23,10 +23,12 @@
import org.apache.druid.msq.counters.ChannelCounters;
import org.apache.druid.msq.counters.CounterNames;
import org.apache.druid.msq.counters.CounterTracker;
+import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory;
import org.apache.druid.msq.input.InputSlice;
import org.apache.druid.msq.input.InputSliceReader;
import org.apache.druid.msq.input.ReadableInput;
import org.apache.druid.msq.input.ReadableInputs;
+import org.apache.druid.msq.kernel.FrameContext;
import org.apache.druid.msq.querykit.DataSegmentProvider;
import org.apache.druid.timeline.SegmentId;
@@ -40,11 +42,13 @@
public class SegmentsInputSliceReader implements InputSliceReader
{
private final DataSegmentProvider dataSegmentProvider;
+ private final LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory;
private final boolean isReindex;
- public SegmentsInputSliceReader(final DataSegmentProvider dataSegmentProvider, final boolean isReindex)
+ public SegmentsInputSliceReader(final FrameContext frameContext, final boolean isReindex)
{
- this.dataSegmentProvider = dataSegmentProvider;
+ this.dataSegmentProvider = frameContext.dataSegmentProvider();
+ this.loadedSegmentDataProviderFactory = frameContext.loadedSegmentDataProviderFactory();
this.isReindex = isReindex;
}
@@ -94,6 +98,7 @@ private Iterator dataSegmentIterator(
return new SegmentWithDescriptor(
dataSegmentProvider.fetchSegment(segmentId, channelCounters, isReindex),
+ descriptor.isLoadedOnServer() ? loadedSegmentDataProviderFactory.createLoadedSegmentDataProvider(dataSource, channelCounters) : null,
descriptor
);
}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java
index 37a97d33be5c..91f2e681e1ea 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java
@@ -166,7 +166,8 @@ public RichSegmentDescriptor toRichSegmentDescriptor()
segment.getInterval(),
interval,
segment.getVersion(),
- segment.getShardSpec().getPartitionNum()
+ segment.getShardSpec().getPartitionNum(),
+ segment instanceof DataSegmentWithLocation ? ((DataSegmentWithLocation) segment).getServers() : null
);
}
}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java
index 2339ac5537a0..49871cecc1d4 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java
@@ -20,6 +20,7 @@
package org.apache.druid.msq.kernel;
import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory;
import org.apache.druid.msq.exec.WorkerMemoryParameters;
import org.apache.druid.msq.querykit.DataSegmentProvider;
import org.apache.druid.query.groupby.GroupingEngine;
@@ -43,6 +44,7 @@ public interface FrameContext
RowIngestionMeters rowIngestionMeters();
DataSegmentProvider dataSegmentProvider();
+ LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory();
File tempDir();
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java
index 3974b7e1e1d6..f67f30d0c5c6 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java
@@ -82,7 +82,12 @@ public ReturnOrAwait runIncrementally(final IntSet readableInputs) throw
final ReturnOrAwait retVal;
if (baseInput.hasSegment()) {
- retVal = runWithSegment(baseInput.getSegment());
+ SegmentWithDescriptor segment = baseInput.getSegment();
+ if (segment.getDescriptor().isLoadedOnServer()) {
+ retVal = runWithLoadedSegment(baseInput.getSegment());
+ } else {
+ retVal = runWithSegment(baseInput.getSegment());
+ }
} else {
retVal = runWithInputChannel(baseInput.getChannel(), baseInput.getChannelFrameReader());
}
@@ -105,6 +110,7 @@ protected FrameWriterFactory getFrameWriterFactory()
}
protected abstract ReturnOrAwait runWithSegment(SegmentWithDescriptor segment) throws IOException;
+ protected abstract ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment) throws IOException;
protected abstract ReturnOrAwait runWithInputChannel(
ReadableFrameChannel inputChannel,
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java
index 0e931c7f8ef0..91ee4a487885 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java
@@ -32,7 +32,7 @@ public interface DataSegmentProvider
* Returns a supplier that fetches the segment corresponding to the provided segmentId from deep storage. The segment
* is not actually fetched until you call {@link Supplier#get()}. Once you call this, make sure to also call
* {@link ResourceHolder#close()}.
- *
+ *
* It is not necessary to call {@link ResourceHolder#close()} if you never call {@link Supplier#get()}.
*/
Supplier> fetchSegment(
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java
index d8481bf7a094..16eaef63c497 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java
@@ -51,6 +51,7 @@
import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryDataSource;
import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.UnionDataSource;
import org.apache.druid.query.UnnestDataSource;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.planning.DataSourceAnalysis;
@@ -170,6 +171,18 @@ public static DataSourcePlan forDataSource(
minStageNumber,
broadcast
);
+ } else if (dataSource instanceof UnionDataSource) {
+ return forUnion(
+ queryKit,
+ queryId,
+ queryContext,
+ (UnionDataSource) dataSource,
+ querySegmentSpec,
+ filter,
+ maxWorkerCount,
+ minStageNumber,
+ broadcast
+ );
} else if (dataSource instanceof JoinDataSource) {
final JoinAlgorithm preferredJoinAlgorithm = PlannerContext.getJoinAlgorithm(queryContext);
final JoinAlgorithm deducedJoinAlgorithm = deduceJoinAlgorithm(
@@ -458,6 +471,54 @@ private static DataSourcePlan forUnnest(
);
}
+ private static DataSourcePlan forUnion(
+ final QueryKit queryKit,
+ final String queryId,
+ final QueryContext queryContext,
+ final UnionDataSource unionDataSource,
+ final QuerySegmentSpec querySegmentSpec,
+ @Nullable DimFilter filter,
+ final int maxWorkerCount,
+ final int minStageNumber,
+ final boolean broadcast
+ )
+ {
+ // This is done to prevent loss of generality since MSQ can plan any type of DataSource.
+ List children = unionDataSource.getDataSources();
+
+ final QueryDefinitionBuilder subqueryDefBuilder = QueryDefinition.builder();
+ final List newChildren = new ArrayList<>();
+ final List inputSpecs = new ArrayList<>();
+ final IntSet broadcastInputs = new IntOpenHashSet();
+
+ for (DataSource child : children) {
+ DataSourcePlan childDataSourcePlan = forDataSource(
+ queryKit,
+ queryId,
+ queryContext,
+ child,
+ querySegmentSpec,
+ filter,
+ maxWorkerCount,
+ Math.max(minStageNumber, subqueryDefBuilder.getNextStageNumber()),
+ broadcast
+ );
+
+ int shift = inputSpecs.size();
+
+ newChildren.add(shiftInputNumbers(childDataSourcePlan.getNewDataSource(), shift));
+ inputSpecs.addAll(childDataSourcePlan.getInputSpecs());
+ childDataSourcePlan.getSubQueryDefBuilder().ifPresent(subqueryDefBuilder::addAll);
+ childDataSourcePlan.getBroadcastInputs().forEach(inp -> broadcastInputs.add(inp + shift));
+ }
+ return new DataSourcePlan(
+ new UnionDataSource(newChildren),
+ inputSpecs,
+ broadcastInputs,
+ subqueryDefBuilder
+ );
+ }
+
/**
* Build a plan for broadcast hash-join.
*/
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java
index 63f5ad6650ae..1e9eedc4c436 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java
@@ -33,11 +33,14 @@
import org.apache.druid.frame.write.FrameWriter;
import org.apache.druid.frame.write.FrameWriterFactory;
import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.Unit;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.guava.Yielders;
import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.msq.exec.LoadedSegmentDataProvider;
import org.apache.druid.msq.input.ReadableInput;
import org.apache.druid.msq.input.table.SegmentWithDescriptor;
import org.apache.druid.msq.querykit.BaseLeafFrameProcessor;
@@ -62,6 +65,7 @@
*/
public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
{
+ private static final Logger log = new Logger(GroupByPreShuffleFrameProcessor.class);
private final GroupByQuery query;
private final GroupingEngine groupingEngine;
private final ColumnSelectorFactory frameWriterColumnSelectorFactory;
@@ -95,6 +99,29 @@ public GroupByPreShuffleFrameProcessor(
);
}
+ @Override
+ protected ReturnOrAwait runWithLoadedSegment(SegmentWithDescriptor segment) throws IOException
+ {
+ if (resultYielder == null) {
+ Pair> statusSequencePair =
+ segment.fetchRowsFromDataServer(groupingEngine.prepareGroupByQuery(query), Function.identity(), closer);
+ if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) {
+ log.info("Segment[%s] was handed off, falling back to fetching the segment from deep storage.",
+ segment.getDescriptor());
+ return runWithSegment(segment);
+ }
+ resultYielder = statusSequencePair.rhs;
+ }
+
+ populateFrameWriterAndFlushIfNeeded();
+
+ if (resultYielder == null || resultYielder.isDone()) {
+ return ReturnOrAwait.returnObject(Unit.instance());
+ } else {
+ return ReturnOrAwait.runAgain();
+ }
+ }
+
@Override
protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException
{
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java
index 99ea8037b7bb..1541d314f215 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java
@@ -22,6 +22,7 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import it.unimi.dsi.fastutil.ints.IntSet;
import org.apache.druid.collections.ResourceHolder;
@@ -40,20 +41,26 @@
import org.apache.druid.frame.write.InvalidNullByteException;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.Unit;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.guava.Yielders;
import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.msq.exec.LoadedSegmentDataProvider;
import org.apache.druid.msq.input.ParseExceptionUtils;
import org.apache.druid.msq.input.ReadableInput;
import org.apache.druid.msq.input.external.ExternalSegment;
import org.apache.druid.msq.input.table.SegmentWithDescriptor;
import org.apache.druid.msq.querykit.BaseLeafFrameProcessor;
import org.apache.druid.msq.querykit.QueryKitUtils;
+import org.apache.druid.query.IterableRowsCursorHelper;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.query.scan.ScanResultValue;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.SpecificSegmentSpec;
import org.apache.druid.segment.ColumnSelectorFactory;
@@ -65,11 +72,13 @@
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.filter.Filters;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
import javax.annotation.Nullable;
+import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -82,8 +91,10 @@
*/
public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor
{
+ private static final Logger log = new Logger(ScanQueryFrameProcessor.class);
private final ScanQuery query;
private final AtomicLong runningCountForLimit;
+ private final ObjectMapper jsonMapper;
private final SettableLongVirtualColumn partitionBoostVirtualColumn;
private final VirtualColumns frameWriterVirtualColumns;
private final Closer closer = Closer.create();
@@ -112,6 +123,7 @@ public ScanQueryFrameProcessor(
);
this.query = query;
this.runningCountForLimit = runningCountForLimit;
+ this.jsonMapper = jsonMapper;
this.partitionBoostVirtualColumn = new SettableLongVirtualColumn(QueryKitUtils.PARTITION_BOOST_COLUMN);
final List frameWriterVirtualColumns = new ArrayList<>();
@@ -152,6 +164,63 @@ public void cleanup() throws IOException
closer.close();
}
+ public static Sequence mappingFunction(Sequence inputSequence)
+ {
+ return inputSequence.flatMap(resultRow -> {
+ List> events = (List>) resultRow.getEvents();
+ return Sequences.simple(events);
+ }).map(List::toArray);
+ }
+
+ @Override
+ protected ReturnOrAwait runWithLoadedSegment(final SegmentWithDescriptor segment) throws IOException
+ {
+ if (cursor == null) {
+ final Pair> statusSequencePair =
+ segment.fetchRowsFromDataServer(
+ query,
+ ScanQueryFrameProcessor::mappingFunction,
+ closer
+ );
+ if (LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF.equals(statusSequencePair.lhs)) {
+ log.info("Segment[%s] was handed off, falling back to fetching the segment from deep storage.",
+ segment.getDescriptor());
+ return runWithSegment(segment);
+ }
+
+ RowSignature rowSignature = ScanQueryKit.getAndValidateSignature(query, jsonMapper);
+ Pair cursorFromIterable = IterableRowsCursorHelper.getCursorFromYielder(
+ statusSequencePair.rhs,
+ rowSignature
+ );
+
+ closer.register(cursorFromIterable.rhs);
+ final Yielder cursorYielder = Yielders.each(Sequences.simple(ImmutableList.of(cursorFromIterable.lhs)));
+
+ if (cursorYielder.isDone()) {
+ // No cursors!
+ cursorYielder.close();
+ return ReturnOrAwait.returnObject(Unit.instance());
+ } else {
+ final long rowsFlushed = setNextCursor(cursorYielder.get(), null);
+ assert rowsFlushed == 0; // There's only ever one cursor when running with a segment
+ closer.register(cursorYielder);
+ }
+ }
+
+ populateFrameWriterAndFlushIfNeededWithExceptionHandling();
+
+ if (cursor.isDone()) {
+ flushFrameWriter();
+ }
+
+ if (cursor.isDone() && (frameWriter == null || frameWriter.getNumRows() == 0)) {
+ return ReturnOrAwait.returnObject(Unit.instance());
+ } else {
+ return ReturnOrAwait.runAgain();
+ }
+ }
+
@Override
protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException
{
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 e6578388a40e..cb331760ca34 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
@@ -114,6 +114,7 @@ public boolean featureAvailable(EngineFeature feature, PlannerContext plannerCon
case TIME_BOUNDARY_QUERY:
case GROUPING_SETS:
case WINDOW_FUNCTIONS:
+ case ALLOW_TOP_LEVEL_UNION_ALL:
return false;
case UNNEST:
case CAN_SELECT:
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/ArrayIngestMode.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/ArrayIngestMode.java
new file mode 100644
index 000000000000..ff6b4718ad85
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/ArrayIngestMode.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.util;
+
+/**
+ * Values that the query context flag 'arrayIngestMode' can take to specify the behaviour of ingestion of arrays via
+ * MSQ's INSERT queries
+ */
+public enum ArrayIngestMode
+{
+ /**
+ * Disables the ingestion of arrays via MSQ's INSERT queries.
+ */
+ NONE,
+
+ /**
+ * String arrays are ingested as MVDs. This is to preserve the legacy behaviour of Druid and will be removed in the
+ * future, since MVDs are not true array types and the behaviour is incorrect.
+ * This also disables the ingestion of numeric arrays
+ */
+ MVD,
+
+ /**
+ * Allows numeric and string arrays to be ingested as arrays. This should be the preferred method of ingestion,
+ * unless bound by compatibility reasons to use 'mvd'
+ */
+ ARRAY
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java
index 2efc94740ac7..98d94518bde8 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java
@@ -24,7 +24,9 @@
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.error.InvalidInput;
import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.AutoTypeColumnSchema;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.column.ColumnCapabilities;
@@ -40,15 +42,31 @@
*/
public class DimensionSchemaUtils
{
+
+ /**
+ * Creates a dimension schema for creating {@link org.apache.druid.data.input.InputSourceReader}.
+ */
+ public static DimensionSchema createDimensionSchemaForExtern(final String column, @Nullable final ColumnType type)
+ {
+ return createDimensionSchema(
+ column,
+ type,
+ false,
+ // Least restrictive mode since we do not have any type restrictions while reading the extern files.
+ ArrayIngestMode.ARRAY
+ );
+ }
+
public static DimensionSchema createDimensionSchema(
final String column,
@Nullable final ColumnType type,
- boolean useAutoType
+ boolean useAutoType,
+ ArrayIngestMode arrayIngestMode
)
{
if (useAutoType) {
// for complex types that are not COMPLEX, we still want to use the handler since 'auto' typing
- // only works for the 'standard' built-in typesg
+ // only works for the 'standard' built-in types
if (type != null && type.is(ValueType.COMPLEX) && !ColumnType.NESTED_DATA.equals(type)) {
final ColumnCapabilities capabilities = ColumnCapabilitiesImpl.createDefault().setType(type);
return DimensionHandlerUtils.getHandlerFromCapabilities(column, capabilities, null)
@@ -57,35 +75,54 @@ public static DimensionSchema createDimensionSchema(
return new AutoTypeColumnSchema(column);
} else {
- // if schema information not available, create a string dimension
+ // if schema information is not available, create a string dimension
if (type == null) {
return new StringDimensionSchema(column);
- }
-
- switch (type.getType()) {
- case STRING:
- return new StringDimensionSchema(column);
- case LONG:
- return new LongDimensionSchema(column);
- case FLOAT:
- return new FloatDimensionSchema(column);
- case DOUBLE:
- return new DoubleDimensionSchema(column);
- case ARRAY:
- switch (type.getElementType().getType()) {
- case STRING:
- return new StringDimensionSchema(column, DimensionSchema.MultiValueHandling.ARRAY, null);
- case LONG:
- case FLOAT:
- case DOUBLE:
- return new AutoTypeColumnSchema(column);
- default:
- throw new ISE("Cannot create dimension for type [%s]", type.toString());
+ } else if (type.getType() == ValueType.STRING) {
+ return new StringDimensionSchema(column);
+ } else if (type.getType() == ValueType.LONG) {
+ return new LongDimensionSchema(column);
+ } else if (type.getType() == ValueType.FLOAT) {
+ return new FloatDimensionSchema(column);
+ } else if (type.getType() == ValueType.DOUBLE) {
+ return new DoubleDimensionSchema(column);
+ } else if (type.getType() == ValueType.ARRAY) {
+ ValueType elementType = type.getElementType().getType();
+ if (elementType == ValueType.STRING) {
+ if (arrayIngestMode == ArrayIngestMode.NONE) {
+ throw InvalidInput.exception(
+ "String arrays can not be ingested when '%s' is set to '%s'. Either set '%s' in query context "
+ + "to 'array' to ingest the string array as an array, or ingest it as an MVD by explicitly casting the "
+ + "array to an MVD with ARRAY_TO_MV function.",
+ MultiStageQueryContext.CTX_ARRAY_INGEST_MODE,
+ StringUtils.toLowerCase(arrayIngestMode.name()),
+ MultiStageQueryContext.CTX_ARRAY_INGEST_MODE
+ );
+ } else if (arrayIngestMode == ArrayIngestMode.MVD) {
+ return new StringDimensionSchema(column, DimensionSchema.MultiValueHandling.ARRAY, null);
+ } else {
+ // arrayIngestMode == ArrayIngestMode.ARRAY would be true
+ return new AutoTypeColumnSchema(column);
+ }
+ } else if (elementType.isNumeric()) {
+ // ValueType == LONG || ValueType == FLOAT || ValueType == DOUBLE
+ if (arrayIngestMode == ArrayIngestMode.ARRAY) {
+ return new AutoTypeColumnSchema(column);
+ } else {
+ throw InvalidInput.exception(
+ "Numeric arrays can only be ingested when '%s' is set to 'array' in the MSQ query's context. "
+ + "Current value of the parameter [%s]",
+ MultiStageQueryContext.CTX_ARRAY_INGEST_MODE,
+ StringUtils.toLowerCase(arrayIngestMode.name())
+ );
}
- default:
- final ColumnCapabilities capabilities = ColumnCapabilitiesImpl.createDefault().setType(type);
- return DimensionHandlerUtils.getHandlerFromCapabilities(column, capabilities, null)
- .getDimensionSchema(capabilities);
+ } else {
+ throw new ISE("Cannot create dimension for type [%s]", type.toString());
+ }
+ } else {
+ final ColumnCapabilities capabilities = ColumnCapabilitiesImpl.createDefault().setType(type);
+ return DimensionHandlerUtils.getHandlerFromCapabilities(column, capabilities, null)
+ .getDimensionSchema(capabilities);
}
}
}
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 98dcd471d0fe..613fac6203c2 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
@@ -27,6 +27,7 @@
import com.opencsv.RFC4180ParserBuilder;
import org.apache.druid.msq.exec.ClusterStatisticsMergeMode;
import org.apache.druid.msq.exec.Limits;
+import org.apache.druid.msq.exec.SegmentSource;
import org.apache.druid.msq.indexing.destination.MSQSelectDestination;
import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
import org.apache.druid.msq.sql.MSQMode;
@@ -73,6 +74,10 @@
* {@link org.apache.druid.segment.AutoTypeColumnSchema} for all 'standard' type columns during segment generation,
* see {@link DimensionSchemaUtils#createDimensionSchema} for more details.
*
+ * arrayIngestMode : Tri-state query context that controls the behaviour and support of arrays that are
+ * ingested via MSQ. If set to 'none', arrays are not allowed to be ingested in MSQ. If set to 'array', array types
+ * can be ingested as expected. If set to 'mvd', numeric arrays can not be ingested, and string arrays will be
+ * ingested as MVDs (this is kept for legacy purpose).
*
**/
public class MultiStageQueryContext
@@ -90,6 +95,9 @@ public class MultiStageQueryContext
public static final String CTX_FINALIZE_AGGREGATIONS = "finalizeAggregations";
private static final boolean DEFAULT_FINALIZE_AGGREGATIONS = true;
+ public static final String CTX_INCLUDE_SEGMENT_SOURCE = "includeSegmentSource";
+ public static final SegmentSource DEFAULT_INCLUDE_SEGMENT_SOURCE = SegmentSource.NONE;
+
public static final String CTX_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage";
private static final boolean DEFAULT_DURABLE_SHUFFLE_STORAGE = false;
public static final String CTX_SELECT_DESTINATION = "selectDestination";
@@ -123,6 +131,11 @@ public class MultiStageQueryContext
public static final String CTX_INDEX_SPEC = "indexSpec";
public static final String CTX_USE_AUTO_SCHEMAS = "useAutoColumnSchemas";
+ public static final boolean DEFAULT_USE_AUTO_SCHEMAS = false;
+
+ public static final String CTX_ARRAY_INGEST_MODE = "arrayIngestMode";
+ public static final ArrayIngestMode DEFAULT_ARRAY_INGEST_MODE = ArrayIngestMode.MVD;
+
private static final Pattern LOOKS_LIKE_JSON_ARRAY = Pattern.compile("^\\s*\\[.*", Pattern.DOTALL);
@@ -191,6 +204,15 @@ public static boolean isFinalizeAggregations(final QueryContext queryContext)
);
}
+ public static SegmentSource getSegmentSources(final QueryContext queryContext)
+ {
+ return queryContext.getEnum(
+ CTX_INCLUDE_SEGMENT_SOURCE,
+ SegmentSource.class,
+ DEFAULT_INCLUDE_SEGMENT_SOURCE
+ );
+ }
+
public static WorkerAssignmentStrategy getAssignmentStrategy(final QueryContext queryContext)
{
return QueryContexts.getAsEnum(
@@ -253,7 +275,12 @@ public static IndexSpec getIndexSpec(final QueryContext queryContext, final Obje
public static boolean useAutoColumnSchemas(final QueryContext queryContext)
{
- return queryContext.getBoolean(CTX_USE_AUTO_SCHEMAS, false);
+ return queryContext.getBoolean(CTX_USE_AUTO_SCHEMAS, DEFAULT_USE_AUTO_SCHEMAS);
+ }
+
+ public static ArrayIngestMode getArrayIngestMode(final QueryContext queryContext)
+ {
+ return queryContext.getEnum(CTX_ARRAY_INGEST_MODE, ArrayIngestMode.class, DEFAULT_ARRAY_INGEST_MODE);
}
/**
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java
new file mode 100644
index 000000000000..6c6ad1b3fa9a
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/LoadedSegmentDataProviderTest.java
@@ -0,0 +1,247 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.exec;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.Futures;
+import org.apache.druid.client.coordinator.CoordinatorClient;
+import org.apache.druid.discovery.DataServerClient;
+import org.apache.druid.discovery.DruidServiceTestUtils;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.common.IOE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.msq.counters.ChannelCounters;
+import org.apache.druid.msq.input.table.RichSegmentDescriptor;
+import org.apache.druid.msq.querykit.InputNumberDataSource;
+import org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessor;
+import org.apache.druid.query.MapQueryToolChestWarehouse;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryInterruptedException;
+import org.apache.druid.query.QueryToolChest;
+import org.apache.druid.query.QueryToolChestWarehouse;
+import org.apache.druid.query.context.ResponseContext;
+import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.query.scan.ScanQueryQueryToolChest;
+import org.apache.druid.query.scan.ScanResultValue;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.rpc.RpcException;
+import org.apache.druid.rpc.ServiceClientFactory;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.druid.query.Druids.newScanQueryBuilder;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+@RunWith(MockitoJUnitRunner.class)
+public class LoadedSegmentDataProviderTest
+{
+ private static final String DATASOURCE1 = "dataSource1";
+ private static final DruidServerMetadata DRUID_SERVER_1 = new DruidServerMetadata(
+ "name1",
+ "host1:5050",
+ null,
+ 100L,
+ ServerType.REALTIME,
+ "tier1",
+ 0
+ );
+ private static final RichSegmentDescriptor SEGMENT_1 = new RichSegmentDescriptor(
+ Intervals.of("2003/2004"),
+ Intervals.of("2003/2004"),
+ "v1",
+ 1,
+ ImmutableSet.of(DRUID_SERVER_1)
+ );
+ private DataServerClient dataServerClient;
+ private CoordinatorClient coordinatorClient;
+ private ScanResultValue scanResultValue;
+ private ScanQuery query;
+ private LoadedSegmentDataProvider target;
+
+ @Before
+ public void setUp()
+ {
+ dataServerClient = mock(DataServerClient.class);
+ coordinatorClient = mock(CoordinatorClient.class);
+ scanResultValue = new ScanResultValue(
+ null,
+ ImmutableList.of(),
+ ImmutableList.of(
+ ImmutableList.of("abc", "123"),
+ ImmutableList.of("ghi", "456"),
+ ImmutableList.of("xyz", "789")
+ )
+ );
+ query = newScanQueryBuilder()
+ .dataSource(new InputNumberDataSource(1))
+ .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003/2004"))))
+ .columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1")
+ .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+ .context(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 1))
+ .build();
+ QueryToolChestWarehouse queryToolChestWarehouse = new MapQueryToolChestWarehouse(
+ ImmutableMap., QueryToolChest>builder()
+ .put(ScanQuery.class, new ScanQueryQueryToolChest(null, null))
+ .build()
+ );
+ target = spy(
+ new LoadedSegmentDataProvider(
+ DATASOURCE1,
+ new ChannelCounters(),
+ mock(ServiceClientFactory.class),
+ coordinatorClient,
+ DruidServiceTestUtils.newJsonMapper(),
+ queryToolChestWarehouse,
+ Execs.scheduledSingleThreaded("query-cancellation-executor")
+ )
+ );
+ doReturn(dataServerClient).when(target).makeDataServerClient(any());
+ }
+
+ @Test
+ public void testFetchRowsFromServer() throws IOException
+ {
+ doReturn(Sequences.simple(ImmutableList.of(scanResultValue))).when(dataServerClient).run(any(), any(), any(), any());
+
+ Pair> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer(
+ query,
+ SEGMENT_1,
+ ScanQueryFrameProcessor::mappingFunction,
+ Closer.create()
+ );
+
+ Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS, dataServerQueryStatusYielderPair.lhs);
+ List> events = (List>) scanResultValue.getEvents();
+ Yielder yielder = dataServerQueryStatusYielderPair.rhs;
+ events.forEach(
+ event -> {
+ Assert.assertArrayEquals(event.toArray(), yielder.get());
+ yielder.next(null);
+ }
+ );
+ }
+
+ @Test
+ public void testHandoff() throws IOException
+ {
+ doAnswer(invocation -> {
+ ResponseContext responseContext = invocation.getArgument(1);
+ responseContext.addMissingSegments(ImmutableList.of(SEGMENT_1));
+ return Sequences.empty();
+ }).when(dataServerClient).run(any(), any(), any(), any());
+ doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1);
+
+ Pair> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer(
+ query,
+ SEGMENT_1,
+ ScanQueryFrameProcessor::mappingFunction,
+ Closer.create()
+ );
+
+ Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF, dataServerQueryStatusYielderPair.lhs);
+ Assert.assertNull(dataServerQueryStatusYielderPair.rhs);
+ }
+
+ @Test
+ public void testServerNotFoundWithoutHandoffShouldThrowException()
+ {
+ doThrow(
+ new QueryInterruptedException(new RpcException("Could not connect to server"))
+ ).when(dataServerClient).run(any(), any(), any(), any());
+
+ doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1);
+
+ ScanQuery queryWithRetry = query.withOverriddenContext(ImmutableMap.of(QueryContexts.NUM_RETRIES_ON_MISSING_SEGMENTS_KEY, 3));
+
+ Assert.assertThrows(DruidException.class, () ->
+ target.fetchRowsFromDataServer(
+ queryWithRetry,
+ SEGMENT_1,
+ ScanQueryFrameProcessor::mappingFunction,
+ Closer.create()
+ )
+ );
+
+ verify(dataServerClient, times(3)).run(any(), any(), any(), any());
+ }
+
+ @Test
+ public void testServerNotFoundButHandoffShouldReturnWithStatus() throws IOException
+ {
+ doThrow(
+ new QueryInterruptedException(new RpcException("Could not connect to server"))
+ ).when(dataServerClient).run(any(), any(), any(), any());
+
+ doReturn(Futures.immediateFuture(Boolean.TRUE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1);
+
+ Pair> dataServerQueryStatusYielderPair = target.fetchRowsFromDataServer(
+ query,
+ SEGMENT_1,
+ ScanQueryFrameProcessor::mappingFunction,
+ Closer.create()
+ );
+
+ Assert.assertEquals(LoadedSegmentDataProvider.DataServerQueryStatus.HANDOFF, dataServerQueryStatusYielderPair.lhs);
+ Assert.assertNull(dataServerQueryStatusYielderPair.rhs);
+ }
+
+ @Test
+ public void testQueryFail()
+ {
+ doAnswer(invocation -> {
+ ResponseContext responseContext = invocation.getArgument(1);
+ responseContext.addMissingSegments(ImmutableList.of(SEGMENT_1));
+ return Sequences.empty();
+ }).when(dataServerClient).run(any(), any(), any(), any());
+ doReturn(Futures.immediateFuture(Boolean.FALSE)).when(coordinatorClient).isHandoffComplete(DATASOURCE1, SEGMENT_1);
+
+ Assert.assertThrows(IOE.class, () ->
+ target.fetchRowsFromDataServer(
+ query,
+ SEGMENT_1,
+ ScanQueryFrameProcessor::mappingFunction,
+ Closer.create()
+ )
+ );
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java
new file mode 100644
index 000000000000..d2696f232820
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQArraysTest.java
@@ -0,0 +1,727 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.exec;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.data.input.impl.JsonInputFormat;
+import org.apache.druid.data.input.impl.LocalInputSource;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.msq.indexing.MSQSpec;
+import org.apache.druid.msq.indexing.MSQTuningConfig;
+import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination;
+import org.apache.druid.msq.test.MSQTestBase;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.NestedDataTestUtils;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.expression.TestExprMacroTable;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.external.ExternalDataSource;
+import org.apache.druid.sql.calcite.filtration.Filtration;
+import org.apache.druid.sql.calcite.planner.ColumnMapping;
+import org.apache.druid.sql.calcite.planner.ColumnMappings;
+import org.apache.druid.timeline.SegmentId;
+import org.apache.druid.utils.CompressionUtils;
+import org.hamcrest.CoreMatchers;
+import org.junit.Test;
+import org.junit.internal.matchers.ThrowableMessageMatcher;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tests INSERT and SELECT behaviour of MSQ with arrays and MVDs
+ */
+@RunWith(Parameterized.class)
+public class MSQArraysTest extends MSQTestBase
+{
+
+ @Parameterized.Parameters(name = "{index}:with context {0}")
+ public static Collection data()
+ {
+ Object[][] data = new Object[][]{
+ {DEFAULT, DEFAULT_MSQ_CONTEXT},
+ {DURABLE_STORAGE, DURABLE_STORAGE_MSQ_CONTEXT},
+ {FAULT_TOLERANCE, FAULT_TOLERANCE_MSQ_CONTEXT},
+ {PARALLEL_MERGE, PARALLEL_MERGE_MSQ_CONTEXT}
+ };
+ return Arrays.asList(data);
+ }
+
+ @Parameterized.Parameter(0)
+ public String contextName;
+
+ @Parameterized.Parameter(1)
+ public Map context;
+
+ /**
+ * Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest
+ * string arrays
+ */
+ @Test
+ public void testInsertStringArrayWithArrayIngestModeNone()
+ {
+
+ final Map adjustedContext = new HashMap<>(context);
+ adjustedContext.put(MultiStageQueryContext.CTX_ARRAY_INGEST_MODE, "none");
+
+ testIngestQuery().setSql(
+ "INSERT INTO foo1 SELECT MV_TO_ARRAY(dim3) AS dim3 FROM foo GROUP BY 1 PARTITIONED BY ALL TIME")
+ .setQueryContext(adjustedContext)
+ .setExpectedExecutionErrorMatcher(CoreMatchers.allOf(
+ CoreMatchers.instanceOf(ISE.class),
+ ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
+ "String arrays can not be ingested when 'arrayIngestMode' is set to 'none'"))
+ ))
+ .verifyExecutionError();
+ }
+
+
+ /**
+ * Tests the behaviour of INSERT query when arrayIngestMode is set to mvd (default) and the only array type to be
+ * ingested is string array
+ */
+ @Test
+ public void testInsertOnFoo1WithMultiValueToArrayGroupByWithDefaultContext()
+ {
+ RowSignature rowSignature = RowSignature.builder()
+ .add("__time", ColumnType.LONG)
+ .add("dim3", ColumnType.STRING)
+ .build();
+
+ testIngestQuery().setSql(
+ "INSERT INTO foo1 SELECT MV_TO_ARRAY(dim3) AS dim3 FROM foo GROUP BY 1 PARTITIONED BY ALL TIME")
+ .setExpectedDataSource("foo1")
+ .setExpectedRowSignature(rowSignature)
+ .setQueryContext(context)
+ .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
+ .setExpectedResultRows(expectedMultiValueFooRowsToArray())
+ .verifyResults();
+ }
+
+ /**
+ * Tests the INSERT query when 'auto' type is set
+ */
+ @Test
+ public void testInsertArraysAutoType() throws IOException
+ {
+ List expectedRows = Arrays.asList(
+ new Object[]{1672531200000L, null, null, null},
+ new Object[]{1672531200000L, null, new Object[]{1L, 2L, 3L}, new Object[]{1.1, 2.2, 3.3}},
+ new Object[]{1672531200000L, new Object[]{"d", "e"}, new Object[]{1L, 4L}, new Object[]{2.2, 3.3, 4.0}},
+ new Object[]{1672531200000L, new Object[]{"a", "b"}, null, null},
+ new Object[]{1672531200000L, new Object[]{"a", "b"}, new Object[]{1L, 2L, 3L}, new Object[]{1.1, 2.2, 3.3}},
+ new Object[]{1672531200000L, new Object[]{"b", "c"}, new Object[]{1L, 2L, 3L, 4L}, new Object[]{1.1, 3.3}},
+ new Object[]{1672531200000L, new Object[]{"a", "b", "c"}, new Object[]{2L, 3L}, new Object[]{3.3, 4.4, 5.5}},
+ new Object[]{1672617600000L, null, null, null},
+ new Object[]{1672617600000L, null, new Object[]{1L, 2L, 3L}, new Object[]{1.1, 2.2, 3.3}},
+ new Object[]{1672617600000L, new Object[]{"d", "e"}, new Object[]{1L, 4L}, new Object[]{2.2, 3.3, 4.0}},
+ new Object[]{1672617600000L, new Object[]{"a", "b"}, null, null},
+ new Object[]{1672617600000L, new Object[]{"a", "b"}, new Object[]{1L, 2L, 3L}, new Object[]{1.1, 2.2, 3.3}},
+ new Object[]{1672617600000L, new Object[]{"b", "c"}, new Object[]{1L, 2L, 3L, 4L}, new Object[]{1.1, 3.3}},
+ new Object[]{1672617600000L, new Object[]{"a", "b", "c"}, new Object[]{2L, 3L}, new Object[]{3.3, 4.4, 5.5}}
+ );
+
+ RowSignature rowSignature = RowSignature.builder()
+ .add("__time", ColumnType.LONG)
+ .add("arrayString", ColumnType.STRING_ARRAY)
+ .add("arrayLong", ColumnType.LONG_ARRAY)
+ .add("arrayDouble", ColumnType.DOUBLE_ARRAY)
+ .build();
+
+ final Map adjustedContext = new HashMap<>(context);
+ adjustedContext.put(MultiStageQueryContext.CTX_USE_AUTO_SCHEMAS, true);
+
+ final File tmpFile = temporaryFolder.newFile();
+ final InputStream resourceStream = NestedDataTestUtils.class.getClassLoader()
+ .getResourceAsStream(NestedDataTestUtils.ARRAY_TYPES_DATA_FILE);
+ final InputStream decompressing = CompressionUtils.decompress(
+ resourceStream,
+ NestedDataTestUtils.ARRAY_TYPES_DATA_FILE
+ );
+ Files.copy(decompressing, tmpFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+ decompressing.close();
+
+ final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(tmpFile);
+
+ testIngestQuery().setSql(" INSERT INTO foo1 SELECT\n"
+ + " TIME_PARSE(\"timestamp\") as __time,\n"
+ + " arrayString,\n"
+ + " arrayLong,\n"
+ + " arrayDouble\n"
+ + "FROM TABLE(\n"
+ + " EXTERN(\n"
+ + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n"
+ + " '{\"type\": \"json\"}',\n"
+ + " '[{\"name\": \"timestamp\", \"type\": \"STRING\"}, {\"name\": \"arrayString\", \"type\": \"COMPLEX\"}, {\"name\": \"arrayLong\", \"type\": \"COMPLEX\"}, {\"name\": \"arrayDouble\", \"type\": \"COMPLEX\"}]'\n"
+ + " )\n"
+ + ") PARTITIONED BY ALL")
+ .setQueryContext(adjustedContext)
+ .setExpectedResultRows(expectedRows)
+ .setExpectedDataSource("foo1")
+ .setExpectedRowSignature(rowSignature)
+ .verifyResults();
+ }
+
+ /**
+ * Tests the behaviour of INSERT query when arrayIngestMode is set to mvd and the user tries to ingest numeric array
+ * types as well
+ */
+ @Test
+ public void testInsertArraysWithStringArraysAsMVDs() throws IOException
+ {
+ RowSignature rowSignatureWithoutTimeAndStringColumns =
+ RowSignature.builder()
+ .add("arrayLong", ColumnType.LONG_ARRAY)
+ .add("arrayLongNulls", ColumnType.LONG_ARRAY)
+ .add("arrayDouble", ColumnType.DOUBLE_ARRAY)
+ .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY)
+ .build();
+
+
+ RowSignature fileSignature = RowSignature.builder()
+ .add("timestamp", ColumnType.STRING)
+ .add("arrayString", ColumnType.STRING_ARRAY)
+ .add("arrayStringNulls", ColumnType.STRING_ARRAY)
+ .addAll(rowSignatureWithoutTimeAndStringColumns)
+ .build();
+
+ final Map adjustedContext = new HashMap<>(context);
+ adjustedContext.put(MultiStageQueryContext.CTX_ARRAY_INGEST_MODE, "mvd");
+
+ final File tmpFile = temporaryFolder.newFile();
+ final InputStream resourceStream = NestedDataTestUtils.class.getClassLoader()
+ .getResourceAsStream(NestedDataTestUtils.ARRAY_TYPES_DATA_FILE);
+ final InputStream decompressing = CompressionUtils.decompress(
+ resourceStream,
+ NestedDataTestUtils.ARRAY_TYPES_DATA_FILE
+ );
+ Files.copy(decompressing, tmpFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+ decompressing.close();
+
+ final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(tmpFile);
+
+ testIngestQuery().setSql(" INSERT INTO foo1 SELECT\n"
+ + " TIME_PARSE(\"timestamp\") as __time,\n"
+ + " arrayString,\n"
+ + " arrayStringNulls,\n"
+ + " arrayLong,\n"
+ + " arrayLongNulls,\n"
+ + " arrayDouble,\n"
+ + " arrayDoubleNulls\n"
+ + "FROM TABLE(\n"
+ + " EXTERN(\n"
+ + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n"
+ + " '{\"type\": \"json\"}',\n"
+ + " '" + queryFramework().queryJsonMapper().writeValueAsString(fileSignature) + "'\n"
+ + " )\n"
+ + ") PARTITIONED BY ALL")
+ .setQueryContext(adjustedContext)
+ .setExpectedExecutionErrorMatcher(CoreMatchers.allOf(
+ CoreMatchers.instanceOf(ISE.class),
+ ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
+ "Numeric arrays can only be ingested when"))
+ ))
+ .verifyExecutionError();
+ }
+
+ /**
+ * Tests the behaviour of INSERT query when arrayIngestMode is set to array and the user tries to ingest all
+ * array types
+ */
+ @Test
+ public void testInsertArraysAsArrays() throws IOException
+ {
+ final List expectedRows = Arrays.asList(
+ new Object[]{
+ 1672531200000L,
+ null,
+ null,
+ new Object[]{1L, 2L, 3L},
+ new Object[]{},
+ new Object[]{1.1d, 2.2d, 3.3d},
+ null
+ },
+ new Object[]{
+ 1672531200000L,
+ null,
+ new Object[]{"a", "b"},
+ null,
+ new Object[]{2L, 3L},
+ null,
+ new Object[]{null}
+ },
+ new Object[]{
+ 1672531200000L,
+ new Object[]{"d", "e"},
+ new Object[]{"b", "b"},
+ new Object[]{1L, 4L},
+ new Object[]{1L},
+ new Object[]{2.2d, 3.3d, 4.0d},
+ null
+ },
+ new Object[]{
+ 1672531200000L,
+ new Object[]{"a", "b"},
+ null,
+ null,
+ new Object[]{null, 2L, 9L},
+ null,
+ new Object[]{999.0d, 5.5d, null}
+ },
+ new Object[]{
+ 1672531200000L,
+ new Object[]{"a", "b"},
+ new Object[]{"a", "b"},
+ new Object[]{1L, 2L, 3L},
+ new Object[]{1L, null, 3L},
+ new Object[]{1.1d, 2.2d, 3.3d},
+ new Object[]{1.1d, 2.2d, null}
+ },
+ new Object[]{
+ 1672531200000L,
+ new Object[]{"b", "c"},
+ new Object[]{"d", null, "b"},
+ new Object[]{1L, 2L, 3L, 4L},
+ new Object[]{1L, 2L, 3L},
+ new Object[]{1.1d, 3.3d},
+ new Object[]{null, 2.2d, null}
+ },
+ new Object[]{
+ 1672531200000L,
+ new Object[]{"a", "b", "c"},
+ new Object[]{null, "b"},
+ new Object[]{2L, 3L},
+ null,
+ new Object[]{3.3d, 4.4d, 5.5d},
+ new Object[]{999.0d, null, 5.5d}
+ },
+ new Object[]{
+ 1672617600000L,
+ null,
+ null,
+ new Object[]{1L, 2L, 3L},
+ null,
+ new Object[]{1.1d, 2.2d, 3.3d},
+ new Object[]{}
+ },
+ new Object[]{
+ 1672617600000L,
+ null,
+ new Object[]{"a", "b"},
+ null,
+ new Object[]{2L, 3L},
+ null,
+ new Object[]{null, 1.1d}
+ },
+ new Object[]{
+ 1672617600000L,
+ new Object[]{"d", "e"},
+ new Object[]{"b", "b"},
+ new Object[]{1L, 4L},
+ new Object[]{null},
+ new Object[]{2.2d, 3.3d, 4.0},
+ null
+ },
+ new Object[]{
+ 1672617600000L,
+ new Object[]{"a", "b"},
+ new Object[]{null},
+ null,
+ new Object[]{null, 2L, 9L},
+ null,
+ new Object[]{999.0d, 5.5d, null}
+ },
+ new Object[]{
+ 1672617600000L,
+ new Object[]{"a", "b"},
+ new Object[]{},
+ new Object[]{1L, 2L, 3L},
+ new Object[]{1L, null, 3L},
+ new Object[]{1.1d, 2.2d, 3.3d},
+ new Object[]{1.1d, 2.2d, null}
+ },
+ new Object[]{
+ 1672617600000L,
+ new Object[]{"b", "c"},
+ new Object[]{"d", null, "b"},
+ new Object[]{1L, 2L, 3L, 4L},
+ new Object[]{1L, 2L, 3L},
+ new Object[]{1.1d, 3.3d},
+ new Object[]{null, 2.2d, null}
+ },
+ new Object[]{
+ 1672617600000L,
+ new Object[]{"a", "b", "c"},
+ new Object[]{null, "b"},
+ new Object[]{2L, 3L},
+ null,
+ new Object[]{3.3d, 4.4d, 5.5d},
+ new Object[]{999.0d, null, 5.5d}
+ }
+ );
+
+ RowSignature rowSignatureWithoutTimeColumn =
+ RowSignature.builder()
+ .add("arrayString", ColumnType.STRING_ARRAY)
+ .add("arrayStringNulls", ColumnType.STRING_ARRAY)
+ .add("arrayLong", ColumnType.LONG_ARRAY)
+ .add("arrayLongNulls", ColumnType.LONG_ARRAY)
+ .add("arrayDouble", ColumnType.DOUBLE_ARRAY)
+ .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY)
+ .build();
+
+ RowSignature fileSignature = RowSignature.builder()
+ .add("timestamp", ColumnType.STRING)
+ .addAll(rowSignatureWithoutTimeColumn)
+ .build();
+
+ RowSignature rowSignature = RowSignature.builder()
+ .add("__time", ColumnType.LONG)
+ .addAll(rowSignatureWithoutTimeColumn)
+ .build();
+
+ final Map adjustedContext = new HashMap<>(context);
+ adjustedContext.put(MultiStageQueryContext.CTX_ARRAY_INGEST_MODE, "array");
+
+ final File tmpFile = temporaryFolder.newFile();
+ final InputStream resourceStream = NestedDataTestUtils.class.getClassLoader()
+ .getResourceAsStream(NestedDataTestUtils.ARRAY_TYPES_DATA_FILE);
+ final InputStream decompressing = CompressionUtils.decompress(
+ resourceStream,
+ NestedDataTestUtils.ARRAY_TYPES_DATA_FILE
+ );
+ Files.copy(decompressing, tmpFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+ decompressing.close();
+
+ final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(tmpFile);
+
+ testIngestQuery().setSql(" INSERT INTO foo1 SELECT\n"
+ + " TIME_PARSE(\"timestamp\") as __time,\n"
+ + " arrayString,\n"
+ + " arrayStringNulls,\n"
+ + " arrayLong,\n"
+ + " arrayLongNulls,\n"
+ + " arrayDouble,\n"
+ + " arrayDoubleNulls\n"
+ + "FROM TABLE(\n"
+ + " EXTERN(\n"
+ + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n"
+ + " '{\"type\": \"json\"}',\n"
+ + " '" + queryFramework().queryJsonMapper().writeValueAsString(fileSignature) + "'\n"
+ + " )\n"
+ + ") PARTITIONED BY ALL")
+ .setQueryContext(adjustedContext)
+ .setExpectedResultRows(expectedRows)
+ .setExpectedDataSource("foo1")
+ .setExpectedRowSignature(rowSignature)
+ .verifyResults();
+ }
+
+ @Test
+ public void testSelectOnArraysWithArrayIngestModeAsNone() throws IOException
+ {
+ testSelectOnArrays("none");
+ }
+
+ @Test
+ public void testSelectOnArraysWithArrayIngestModeAsMVD() throws IOException
+ {
+ testSelectOnArrays("mvd");
+ }
+
+ @Test
+ public void testSelectOnArraysWithArrayIngestModeAsArray() throws IOException
+ {
+ testSelectOnArrays("array");
+ }
+
+ // Tests the behaviour of the select with the given arrayIngestMode. The expectation should be the same, since the
+ // arrayIngestMode should only determine how the array gets ingested at the end.
+ public void testSelectOnArrays(String arrayIngestMode) throws IOException
+ {
+ final List expectedRows = Arrays.asList(
+ new Object[]{
+ 1672531200000L,
+ Arrays.asList("a", "b"),
+ Arrays.asList("a", "b"),
+ Arrays.asList(1L, 2L, 3L),
+ Arrays.asList(1L, null, 3L),
+ Arrays.asList(1.1d, 2.2d, 3.3d),
+ Arrays.asList(1.1d, 2.2d, null)
+ },
+ new Object[]{
+ 1672531200000L,
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList(null, "b"),
+ Arrays.asList(2L, 3L),
+ null,
+ Arrays.asList(3.3d, 4.4d, 5.5d),
+ Arrays.asList(999.0d, null, 5.5d),
+ },
+ new Object[]{
+ 1672531200000L,
+ Arrays.asList("b", "c"),
+ Arrays.asList("d", null, "b"),
+ Arrays.asList(1L, 2L, 3L, 4L),
+ Arrays.asList(1L, 2L, 3L),
+ Arrays.asList(1.1d, 3.3d),
+ Arrays.asList(null, 2.2d, null)
+ },
+ new Object[]{
+ 1672531200000L,
+ Arrays.asList("d", "e"),
+ Arrays.asList("b", "b"),
+ Arrays.asList(1L, 4L),
+ Collections.singletonList(1L),
+ Arrays.asList(2.2d, 3.3d, 4.0d),
+ null
+ },
+ new Object[]{
+ 1672531200000L,
+ null,
+ null,
+ Arrays.asList(1L, 2L, 3L),
+ Collections.emptyList(),
+ Arrays.asList(1.1d, 2.2d, 3.3d),
+ null
+ },
+ new Object[]{
+ 1672531200000L,
+ Arrays.asList("a", "b"),
+ null,
+ null,
+ Arrays.asList(null, 2L, 9L),
+ null,
+ Arrays.asList(999.0d, 5.5d, null)
+ },
+ new Object[]{
+ 1672531200000L,
+ null,
+ Arrays.asList("a", "b"),
+ null,
+ Arrays.asList(2L, 3L),
+ null,
+ Collections.singletonList(null)
+ },
+ new Object[]{
+ 1672617600000L,
+ Arrays.asList("a", "b"),
+ Collections.emptyList(),
+ Arrays.asList(1L, 2L, 3L),
+ Arrays.asList(1L, null, 3L),
+ Arrays.asList(1.1d, 2.2d, 3.3d),
+ Arrays.asList(1.1d, 2.2d, null)
+ },
+ new Object[]{
+ 1672617600000L,
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList(null, "b"),
+ Arrays.asList(2L, 3L),
+ null,
+ Arrays.asList(3.3d, 4.4d, 5.5d),
+ Arrays.asList(999.0d, null, 5.5d)
+ },
+ new Object[]{
+ 1672617600000L,
+ Arrays.asList("b", "c"),
+ Arrays.asList("d", null, "b"),
+ Arrays.asList(1L, 2L, 3L, 4L),
+ Arrays.asList(1L, 2L, 3L),
+ Arrays.asList(1.1d, 3.3d),
+ Arrays.asList(null, 2.2d, null)
+ },
+ new Object[]{
+ 1672617600000L,
+ Arrays.asList("d", "e"),
+ Arrays.asList("b", "b"),
+ Arrays.asList(1L, 4L),
+ Collections.singletonList(null),
+ Arrays.asList(2.2d, 3.3d, 4.0),
+ null
+ },
+ new Object[]{
+ 1672617600000L,
+ null,
+ null,
+ Arrays.asList(1L, 2L, 3L),
+ null,
+ Arrays.asList(1.1d, 2.2d, 3.3d),
+ Collections.emptyList()
+ },
+ new Object[]{
+ 1672617600000L,
+ Arrays.asList("a", "b"),
+ Collections.singletonList(null),
+ null,
+ Arrays.asList(null, 2L, 9L),
+ null,
+ Arrays.asList(999.0d, 5.5d, null)
+ },
+ new Object[]{
+ 1672617600000L,
+ null,
+ Arrays.asList("a", "b"),
+ null,
+ Arrays.asList(2L, 3L),
+ null,
+ Arrays.asList(null, 1.1d),
+ }
+ );
+
+ RowSignature rowSignatureWithoutTimeColumn =
+ RowSignature.builder()
+ .add("arrayString", ColumnType.STRING_ARRAY)
+ .add("arrayStringNulls", ColumnType.STRING_ARRAY)
+ .add("arrayLong", ColumnType.LONG_ARRAY)
+ .add("arrayLongNulls", ColumnType.LONG_ARRAY)
+ .add("arrayDouble", ColumnType.DOUBLE_ARRAY)
+ .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY)
+ .build();
+
+ RowSignature fileSignature = RowSignature.builder()
+ .add("timestamp", ColumnType.STRING)
+ .addAll(rowSignatureWithoutTimeColumn)
+ .build();
+
+ RowSignature rowSignature = RowSignature.builder()
+ .add("__time", ColumnType.LONG)
+ .addAll(rowSignatureWithoutTimeColumn)
+ .build();
+
+ RowSignature scanSignature = RowSignature.builder()
+ .add("arrayDouble", ColumnType.DOUBLE_ARRAY)
+ .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY)
+ .add("arrayLong", ColumnType.LONG_ARRAY)
+ .add("arrayLongNulls", ColumnType.LONG_ARRAY)
+ .add("arrayString", ColumnType.STRING_ARRAY)
+ .add("arrayStringNulls", ColumnType.STRING_ARRAY)
+ .add("v0", ColumnType.LONG)
+ .build();
+
+ final Map adjustedContext = new HashMap<>(context);
+ adjustedContext.put(MultiStageQueryContext.CTX_ARRAY_INGEST_MODE, arrayIngestMode);
+
+ final File tmpFile = temporaryFolder.newFile();
+ final InputStream resourceStream = NestedDataTestUtils.class.getClassLoader()
+ .getResourceAsStream(NestedDataTestUtils.ARRAY_TYPES_DATA_FILE);
+ final InputStream decompressing = CompressionUtils.decompress(
+ resourceStream,
+ NestedDataTestUtils.ARRAY_TYPES_DATA_FILE
+ );
+ Files.copy(decompressing, tmpFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+ decompressing.close();
+
+ final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(tmpFile);
+
+ Query> expectedQuery = newScanQueryBuilder()
+ .dataSource(new ExternalDataSource(
+ new LocalInputSource(null, null, ImmutableList.of(tmpFile)),
+ new JsonInputFormat(null, null, null, null, null),
+ fileSignature
+ ))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .columns(
+ "arrayDouble",
+ "arrayDoubleNulls",
+ "arrayLong",
+ "arrayLongNulls",
+ "arrayString",
+ "arrayStringNulls",
+ "v0"
+ )
+ .virtualColumns(new ExpressionVirtualColumn(
+ "v0",
+ "timestamp_parse(\"timestamp\",null,'UTC')",
+ ColumnType.LONG,
+ TestExprMacroTable.INSTANCE
+ ))
+ .context(defaultScanQueryContext(adjustedContext, scanSignature))
+ .build();
+
+ testSelectQuery().setSql("SELECT\n"
+ + " TIME_PARSE(\"timestamp\") as __time,\n"
+ + " arrayString,\n"
+ + " arrayStringNulls,\n"
+ + " arrayLong,\n"
+ + " arrayLongNulls,\n"
+ + " arrayDouble,\n"
+ + " arrayDoubleNulls\n"
+ + "FROM TABLE(\n"
+ + " EXTERN(\n"
+ + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n"
+ + " '{\"type\": \"json\"}',\n"
+ + " '" + queryFramework().queryJsonMapper().writeValueAsString(fileSignature) + "'\n"
+ + " )\n"
+ + ")")
+ .setQueryContext(adjustedContext)
+ .setExpectedMSQSpec(MSQSpec
+ .builder()
+ .query(expectedQuery)
+ .columnMappings(new ColumnMappings(ImmutableList.of(
+ new ColumnMapping("v0", "__time"),
+ new ColumnMapping("arrayString", "arrayString"),
+ new ColumnMapping("arrayStringNulls", "arrayStringNulls"),
+ new ColumnMapping("arrayLong", "arrayLong"),
+ new ColumnMapping("arrayLongNulls", "arrayLongNulls"),
+ new ColumnMapping("arrayDouble", "arrayDouble"),
+ new ColumnMapping("arrayDoubleNulls", "arrayDoubleNulls")
+ )))
+ .tuningConfig(MSQTuningConfig.defaultConfig())
+ .destination(TaskReportMSQDestination.INSTANCE)
+ .build()
+ )
+ .setExpectedRowSignature(rowSignature)
+ .setExpectedResultRows(expectedRows)
+ .verifyResults();
+ }
+
+
+ private List expectedMultiValueFooRowsToArray()
+ {
+ List expectedRows = new ArrayList<>();
+ expectedRows.add(new Object[]{0L, null});
+ if (!useDefault) {
+ expectedRows.add(new Object[]{0L, ""});
+ }
+
+ expectedRows.addAll(ImmutableList.of(
+ new Object[]{0L, ImmutableList.of("a", "b")},
+ new Object[]{0L, ImmutableList.of("b", "c")},
+ new Object[]{0L, "d"}
+ ));
+ return expectedRows;
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java
index 42bb1506a307..4b77dd78b339 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java
@@ -20,6 +20,8 @@
package org.apache.druid.msq.exec;
import com.google.common.collect.ImmutableMap;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.error.DruidExceptionMatcher;
import org.apache.druid.indexing.common.actions.SegmentAllocateAction;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
@@ -330,4 +332,46 @@ public void testTooManyInputFiles() throws IOException
.setExpectedMSQFault(new TooManyInputFilesFault(numFiles, Limits.MAX_INPUT_FILES_PER_WORKER, 2))
.verifyResults();
}
+
+ @Test
+ public void testUnionAllWithDifferentColumnNames()
+ {
+ // This test fails till MSQ can support arbitrary column names and column types for UNION ALL
+ testIngestQuery()
+ .setSql(
+ "INSERT INTO druid.dst "
+ + "SELECT dim2, dim1, m1 FROM foo2 "
+ + "UNION ALL "
+ + "SELECT dim1, dim2, m1 FROM foo "
+ + "PARTITIONED BY ALL TIME")
+ .setExpectedValidationErrorMatcher(
+ new DruidExceptionMatcher(
+ DruidException.Persona.ADMIN,
+ DruidException.Category.INVALID_INPUT,
+ "general"
+ ).expectMessageContains("SQL requires union between two tables and column names queried for each table are different "
+ + "Left: [dim2, dim1, m1], Right: [dim1, dim2, m1]."))
+ .verifyPlanningErrors();
+ }
+
+ @Test
+ public void testTopLevelUnionAllWithJoins()
+ {
+ // This test fails becaues it is a top level UNION ALL which cannot be planned using MSQ. It will be supported once
+ // we support arbitrary types and column names for UNION ALL
+ testSelectQuery()
+ .setSql(
+ "(SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) "
+ + "UNION ALL "
+ + "(SELECT SUM(cnt) FROM foo)"
+ )
+ .setExpectedValidationErrorMatcher(
+ new DruidExceptionMatcher(
+ DruidException.Persona.ADMIN,
+ DruidException.Category.INVALID_INPUT,
+ "general"
+ ).expectMessageContains(
+ "SQL requires union between inputs that are not simple table scans and involve a filter or aliasing"))
+ .verifyPlanningErrors();
+ }
}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java
index e54027c2449b..b43dd72e88c8 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java
@@ -38,7 +38,6 @@
import org.apache.druid.msq.test.MSQTestBase;
import org.apache.druid.msq.test.MSQTestFileUtils;
import org.apache.druid.msq.util.MultiStageQueryContext;
-import org.apache.druid.query.NestedDataTestUtils;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
@@ -46,7 +45,6 @@
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.timeline.SegmentId;
-import org.apache.druid.utils.CompressionUtils;
import org.hamcrest.CoreMatchers;
import org.junit.Test;
import org.junit.internal.matchers.ThrowableMessageMatcher;
@@ -54,16 +52,11 @@
import org.junit.runners.Parameterized;
import org.mockito.Mockito;
-import javax.annotation.Nonnull;
import java.io.File;
import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Files;
-import java.nio.file.StandardCopyOption;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
-import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -735,22 +728,6 @@ public void testInsertOnFoo1WithMultiValueMeasureGroupBy()
}
- @Test
- public void testInsertOnFoo1WithMultiValueToArrayGroupBy()
- {
- RowSignature rowSignature = RowSignature.builder()
- .add("__time", ColumnType.LONG)
- .add("dim3", ColumnType.STRING).build();
-
- testIngestQuery().setSql(
- "INSERT INTO foo1 SELECT MV_TO_ARRAY(dim3) AS dim3 FROM foo GROUP BY 1 PARTITIONED BY ALL TIME")
- .setExpectedDataSource("foo1")
- .setExpectedRowSignature(rowSignature)
- .setQueryContext(context)
- .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
- .setExpectedResultRows(expectedMultiValueFooRowsToArray())
- .verifyResults();
- }
@Test
public void testInsertOnFoo1WithAutoTypeArrayGroupBy()
@@ -1407,251 +1384,6 @@ public void testCorrectNumberOfWorkersUsedAutoModeWithBytesLimit() throws IOExce
.verifyResults();
}
- @Test
- public void testInsertArraysAutoType() throws IOException
- {
- List expectedRows = Arrays.asList(
- new Object[]{1672531200000L, null, null, null},
- new Object[]{1672531200000L, null, new Object[]{1L, 2L, 3L}, new Object[]{1.1, 2.2, 3.3}},
- new Object[]{1672531200000L, new Object[]{"d", "e"}, new Object[]{1L, 4L}, new Object[]{2.2, 3.3, 4.0}},
- new Object[]{1672531200000L, new Object[]{"a", "b"}, null, null},
- new Object[]{1672531200000L, new Object[]{"a", "b"}, new Object[]{1L, 2L, 3L}, new Object[]{1.1, 2.2, 3.3}},
- new Object[]{1672531200000L, new Object[]{"b", "c"}, new Object[]{1L, 2L, 3L, 4L}, new Object[]{1.1, 3.3}},
- new Object[]{1672531200000L, new Object[]{"a", "b", "c"}, new Object[]{2L, 3L}, new Object[]{3.3, 4.4, 5.5}},
- new Object[]{1672617600000L, null, null, null},
- new Object[]{1672617600000L, null, new Object[]{1L, 2L, 3L}, new Object[]{1.1, 2.2, 3.3}},
- new Object[]{1672617600000L, new Object[]{"d", "e"}, new Object[]{1L, 4L}, new Object[]{2.2, 3.3, 4.0}},
- new Object[]{1672617600000L, new Object[]{"a", "b"}, null, null},
- new Object[]{1672617600000L, new Object[]{"a", "b"}, new Object[]{1L, 2L, 3L}, new Object[]{1.1, 2.2, 3.3}},
- new Object[]{1672617600000L, new Object[]{"b", "c"}, new Object[]{1L, 2L, 3L, 4L}, new Object[]{1.1, 3.3}},
- new Object[]{1672617600000L, new Object[]{"a", "b", "c"}, new Object[]{2L, 3L}, new Object[]{3.3, 4.4, 5.5}}
- );
-
- RowSignature rowSignature = RowSignature.builder()
- .add("__time", ColumnType.LONG)
- .add("arrayString", ColumnType.STRING_ARRAY)
- .add("arrayLong", ColumnType.LONG_ARRAY)
- .add("arrayDouble", ColumnType.DOUBLE_ARRAY)
- .build();
-
- final Map adjustedContext = new HashMap<>(context);
- adjustedContext.put(MultiStageQueryContext.CTX_USE_AUTO_SCHEMAS, true);
-
- final File tmpFile = temporaryFolder.newFile();
- final InputStream resourceStream = NestedDataTestUtils.class.getClassLoader().getResourceAsStream(NestedDataTestUtils.ARRAY_TYPES_DATA_FILE);
- final InputStream decompressing = CompressionUtils.decompress(resourceStream, NestedDataTestUtils.ARRAY_TYPES_DATA_FILE);
- Files.copy(decompressing, tmpFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
- decompressing.close();
-
- final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(tmpFile);
-
- testIngestQuery().setSql(" INSERT INTO foo1 SELECT\n"
- + " TIME_PARSE(\"timestamp\") as __time,\n"
- + " arrayString,\n"
- + " arrayLong,\n"
- + " arrayDouble\n"
- + "FROM TABLE(\n"
- + " EXTERN(\n"
- + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n"
- + " '{\"type\": \"json\"}',\n"
- + " '[{\"name\": \"timestamp\", \"type\": \"STRING\"}, {\"name\": \"arrayString\", \"type\": \"COMPLEX\"}, {\"name\": \"arrayLong\", \"type\": \"COMPLEX\"}, {\"name\": \"arrayDouble\", \"type\": \"COMPLEX\"}]'\n"
- + " )\n"
- + ") PARTITIONED BY day")
- .setQueryContext(adjustedContext)
- .setExpectedResultRows(expectedRows)
- .setExpectedDataSource("foo1")
- .setExpectedRowSignature(rowSignature)
- .verifyResults();
- }
-
- @Test
- public void testInsertArrays() throws IOException
- {
- List expectedRows = Arrays.asList(
- new Object[]{
- 1672531200000L,
- null,
- null,
- new Object[]{1L, 2L, 3L},
- new Object[]{},
- new Object[]{1.1d, 2.2d, 3.3d},
- null
- },
- new Object[]{
- 1672531200000L,
- null,
- Arrays.asList("a", "b"),
- null,
- new Object[]{2L, 3L},
- null,
- new Object[]{null}
- },
- new Object[]{
- 1672531200000L,
- Arrays.asList("a", "b"),
- null,
- null,
- new Object[]{null, 2L, 9L},
- null,
- new Object[]{999.0d, 5.5d, null}
- },
- new Object[]{
- 1672531200000L,
- Arrays.asList("a", "b"),
- Arrays.asList("a", "b"),
- new Object[]{1L, 2L, 3L},
- new Object[]{1L, null, 3L},
- new Object[]{1.1d, 2.2d, 3.3d},
- new Object[]{1.1d, 2.2d, null}
- },
- new Object[]{
- 1672531200000L,
- Arrays.asList("a", "b", "c"),
- Arrays.asList(null, "b"),
- new Object[]{2L, 3L},
- null,
- new Object[]{3.3d, 4.4d, 5.5d},
- new Object[]{999.0d, null, 5.5d}
- },
- new Object[]{
- 1672531200000L,
- Arrays.asList("b", "c"),
- Arrays.asList("d", null, "b"),
- new Object[]{1L, 2L, 3L, 4L},
- new Object[]{1L, 2L, 3L},
- new Object[]{1.1d, 3.3d},
- new Object[]{null, 2.2d, null}
- },
- new Object[]{
- 1672531200000L,
- Arrays.asList("d", "e"),
- Arrays.asList("b", "b"),
- new Object[]{1L, 4L},
- new Object[]{1L},
- new Object[]{2.2d, 3.3d, 4.0d},
- null
- },
- new Object[]{
- 1672617600000L,
- null,
- null,
- new Object[]{1L, 2L, 3L},
- null,
- new Object[]{1.1d, 2.2d, 3.3d},
- new Object[]{}
- },
- new Object[]{
- 1672617600000L,
- null,
- Arrays.asList("a", "b"),
- null,
- new Object[]{2L, 3L},
- null,
- new Object[]{null, 1.1d}
- },
- new Object[]{
- 1672617600000L,
- Arrays.asList("a", "b"),
- null,
- null,
- new Object[]{null, 2L, 9L},
- null,
- new Object[]{999.0d, 5.5d, null}
- },
- new Object[]{
- 1672617600000L,
- Arrays.asList("a", "b"),
- Collections.emptyList(),
- new Object[]{1L, 2L, 3L},
- new Object[]{1L, null, 3L},
- new Object[]{1.1d, 2.2d, 3.3d},
- new Object[]{1.1d, 2.2d, null}
- },
- new Object[]{
- 1672617600000L,
- Arrays.asList("a", "b", "c"),
- Arrays.asList(null, "b"),
- new Object[]{2L, 3L},
- null,
- new Object[]{3.3d, 4.4d, 5.5d},
- new Object[]{999.0d, null, 5.5d}
- },
- new Object[]{
- 1672617600000L,
- Arrays.asList("b", "c"),
- Arrays.asList("d", null, "b"),
- new Object[]{1L, 2L, 3L, 4L},
- new Object[]{1L, 2L, 3L},
- new Object[]{1.1d, 3.3d},
- new Object[]{null, 2.2d, null}
- },
- new Object[]{
- 1672617600000L,
- Arrays.asList("d", "e"),
- Arrays.asList("b", "b"),
- new Object[]{1L, 4L},
- new Object[]{null},
- new Object[]{2.2d, 3.3d, 4.0},
- null
- }
- );
-
- RowSignature rowSignatureWithoutTimeAndStringColumns =
- RowSignature.builder()
- .add("arrayLong", ColumnType.LONG_ARRAY)
- .add("arrayLongNulls", ColumnType.LONG_ARRAY)
- .add("arrayDouble", ColumnType.DOUBLE_ARRAY)
- .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY)
- .build();
-
-
- RowSignature fileSignature = RowSignature.builder()
- .add("timestamp", ColumnType.STRING)
- .add("arrayString", ColumnType.STRING_ARRAY)
- .add("arrayStringNulls", ColumnType.STRING_ARRAY)
- .addAll(rowSignatureWithoutTimeAndStringColumns)
- .build();
-
- // MSQ writes strings instead of string arrays
- RowSignature rowSignature = RowSignature.builder()
- .add("__time", ColumnType.LONG)
- .add("arrayString", ColumnType.STRING)
- .add("arrayStringNulls", ColumnType.STRING)
- .addAll(rowSignatureWithoutTimeAndStringColumns)
- .build();
-
- final Map adjustedContext = new HashMap<>(context);
- final File tmpFile = temporaryFolder.newFile();
- final InputStream resourceStream = NestedDataTestUtils.class.getClassLoader().getResourceAsStream(NestedDataTestUtils.ARRAY_TYPES_DATA_FILE);
- final InputStream decompressing = CompressionUtils.decompress(resourceStream, NestedDataTestUtils.ARRAY_TYPES_DATA_FILE);
- Files.copy(decompressing, tmpFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
- decompressing.close();
-
- final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(tmpFile);
-
- testIngestQuery().setSql(" INSERT INTO foo1 SELECT\n"
- + " TIME_PARSE(\"timestamp\") as __time,\n"
- + " arrayString,\n"
- + " arrayStringNulls,\n"
- + " arrayLong,\n"
- + " arrayLongNulls,\n"
- + " arrayDouble,\n"
- + " arrayDoubleNulls\n"
- + "FROM TABLE(\n"
- + " EXTERN(\n"
- + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n"
- + " '{\"type\": \"json\"}',\n"
- + " '" + queryFramework().queryJsonMapper().writeValueAsString(fileSignature) + "'\n"
- + " )\n"
- + ") PARTITIONED BY day")
- .setQueryContext(adjustedContext)
- .setExpectedResultRows(expectedRows)
- .setExpectedDataSource("foo1")
- .setExpectedRowSignature(rowSignature)
- .verifyResults();
- }
-
- @Nonnull
private List expectedFooRows()
{
List expectedRows = new ArrayList<>();
@@ -1668,7 +1400,6 @@ private List expectedFooRows()
return expectedRows;
}
- @Nonnull
private List expectedFooRowsWithAggregatedComplexColumn()
{
List expectedRows = new ArrayList<>();
@@ -1687,7 +1418,6 @@ private List expectedFooRowsWithAggregatedComplexColumn()
return expectedRows;
}
- @Nonnull
private List expectedMultiValueFooRows()
{
List expectedRows = new ArrayList<>();
@@ -1704,24 +1434,6 @@ private List expectedMultiValueFooRows()
return expectedRows;
}
- @Nonnull
- private List expectedMultiValueFooRowsToArray()
- {
- List expectedRows = new ArrayList<>();
- expectedRows.add(new Object[]{0L, null});
- if (!useDefault) {
- expectedRows.add(new Object[]{0L, ""});
- }
-
- expectedRows.addAll(ImmutableList.of(
- new Object[]{0L, ImmutableList.of("a", "b")},
- new Object[]{0L, ImmutableList.of("b", "c")},
- new Object[]{0L, "d"}
- ));
- return expectedRows;
- }
-
- @Nonnull
private List expectedMultiValueFooRowsGroupBy()
{
List expectedRows = new ArrayList<>();
@@ -1737,7 +1449,6 @@ private List expectedMultiValueFooRowsGroupBy()
return expectedRows;
}
- @Nonnull
private Set expectedFooSegments()
{
Set expectedSegments = new TreeSet<>();
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java
new file mode 100644
index 000000000000..b2c07e267e4c
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQLoadedSegmentTests.java
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.exec;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.client.ImmutableSegmentLoadInfo;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.guava.Yielders;
+import org.apache.druid.msq.indexing.MSQSpec;
+import org.apache.druid.msq.indexing.MSQTuningConfig;
+import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination;
+import org.apache.druid.msq.test.MSQTestBase;
+import org.apache.druid.msq.util.MultiStageQueryContext;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.ResultRow;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.sql.calcite.filtration.Filtration;
+import org.apache.druid.sql.calcite.planner.ColumnMapping;
+import org.apache.druid.sql.calcite.planner.ColumnMappings;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.hamcrest.CoreMatchers;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+
+public class MSQLoadedSegmentTests extends MSQTestBase
+{
+ public static final Map REALTIME_QUERY_CTX =
+ ImmutableMap.builder()
+ .putAll(DEFAULT_MSQ_CONTEXT)
+ .put(MultiStageQueryContext.CTX_INCLUDE_SEGMENT_SOURCE, SegmentSource.REALTIME.name())
+ .build();
+ public static final DataSegment LOADED_SEGMENT_1 =
+ DataSegment.builder()
+ .dataSource(CalciteTests.DATASOURCE1)
+ .interval(Intervals.of("2003-01-01T00:00:00.000Z/2004-01-01T00:00:00.000Z"))
+ .version("1")
+ .shardSpec(new LinearShardSpec(0))
+ .size(0)
+ .build();
+
+ public static final DruidServerMetadata DATA_SERVER_1 = new DruidServerMetadata(
+ "TestDataServer",
+ "hostName:9092",
+ null,
+ 2,
+ ServerType.REALTIME,
+ "tier1",
+ 2
+ );
+
+ @Before
+ public void setUp()
+ {
+ loadedSegmentsMetadata.add(new ImmutableSegmentLoadInfo(LOADED_SEGMENT_1, ImmutableSet.of(DATA_SERVER_1)));
+ }
+
+ @Test
+ public void testSelectWithLoadedSegmentsOnFoo() throws IOException
+ {
+ RowSignature resultSignature = RowSignature.builder()
+ .add("cnt", ColumnType.LONG)
+ .add("dim1", ColumnType.STRING)
+ .build();
+
+ doReturn(
+ Pair.of(
+ LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS,
+ Yielders.each(
+ Sequences.simple(
+ ImmutableList.of(
+ new Object[]{1L, "qwe"},
+ new Object[]{1L, "tyu"}
+ )
+ )
+ )
+ )
+ )
+ .when(loadedSegmentDataProvider)
+ .fetchRowsFromDataServer(any(), any(), any(), any());
+
+ testSelectQuery()
+ .setSql("select cnt, dim1 from foo")
+ .setExpectedMSQSpec(
+ MSQSpec.builder()
+ .query(
+ newScanQueryBuilder()
+ .dataSource(CalciteTests.DATASOURCE1)
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .columns("cnt", "dim1")
+ .context(defaultScanQueryContext(REALTIME_QUERY_CTX, resultSignature))
+ .build()
+ )
+ .columnMappings(ColumnMappings.identity(resultSignature))
+ .tuningConfig(MSQTuningConfig.defaultConfig())
+ .destination(TaskReportMSQDestination.INSTANCE)
+ .build()
+ )
+ .setQueryContext(REALTIME_QUERY_CTX)
+ .setExpectedRowSignature(resultSignature)
+ .setExpectedResultRows(ImmutableList.of(
+ new Object[]{1L, ""},
+ new Object[]{1L, "qwe"},
+ new Object[]{1L, "10.1"},
+ new Object[]{1L, "tyu"},
+ new Object[]{1L, "2"},
+ new Object[]{1L, "1"},
+ new Object[]{1L, "def"},
+ new Object[]{1L, "abc"}
+ ))
+ .verifyResults();
+ }
+
+ @Test
+ public void testGroupByWithLoadedSegmentsOnFoo() throws IOException
+ {
+ RowSignature rowSignature = RowSignature.builder()
+ .add("cnt", ColumnType.LONG)
+ .add("cnt1", ColumnType.LONG)
+ .build();
+
+ doReturn(
+ Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS,
+ Yielders.each(
+ Sequences.simple(
+ ImmutableList.of(
+ ResultRow.of(1L, 2L)
+ )
+ )
+ )
+ )
+ )
+ .when(loadedSegmentDataProvider)
+ .fetchRowsFromDataServer(any(), any(), any(), any());
+
+ testSelectQuery()
+ .setSql("select cnt,count(*) as cnt1 from foo group by cnt")
+ .setExpectedMSQSpec(
+ MSQSpec.builder()
+ .query(GroupByQuery.builder()
+ .setDataSource(CalciteTests.DATASOURCE1)
+ .setInterval(querySegmentSpec(Filtration
+ .eternity()))
+ .setGranularity(Granularities.ALL)
+ .setDimensions(dimensions(
+ new DefaultDimensionSpec(
+ "cnt",
+ "d0",
+ ColumnType.LONG
+ )
+ ))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory(
+ "a0")))
+ .setContext(REALTIME_QUERY_CTX)
+ .build())
+ .columnMappings(
+ new ColumnMappings(ImmutableList.of(
+ new ColumnMapping("d0", "cnt"),
+ new ColumnMapping("a0", "cnt1")))
+ )
+ .tuningConfig(MSQTuningConfig.defaultConfig())
+ .destination(TaskReportMSQDestination.INSTANCE)
+ .build()
+ )
+ .setQueryContext(REALTIME_QUERY_CTX)
+ .setExpectedRowSignature(rowSignature)
+ .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 8L}))
+ .verifyResults();
+ }
+
+ @Test
+ public void testGroupByWithOnlyLoadedSegmentsOnFoo() throws IOException
+ {
+ RowSignature rowSignature = RowSignature.builder()
+ .add("cnt", ColumnType.LONG)
+ .add("cnt1", ColumnType.LONG)
+ .build();
+
+ doReturn(
+ Pair.of(LoadedSegmentDataProvider.DataServerQueryStatus.SUCCESS,
+ Yielders.each(
+ Sequences.simple(
+ ImmutableList.of(
+ ResultRow.of(1L, 2L)))))
+ ).when(loadedSegmentDataProvider)
+ .fetchRowsFromDataServer(any(), any(), any(), any());
+
+ testSelectQuery()
+ .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt")
+ .setExpectedMSQSpec(
+ MSQSpec.builder()
+ .query(GroupByQuery.builder()
+ .setDataSource(CalciteTests.DATASOURCE1)
+ .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z"))
+ .setGranularity(Granularities.ALL)
+ .setDimensions(dimensions(
+ new DefaultDimensionSpec(
+ "cnt",
+ "d0",
+ ColumnType.LONG
+ )
+ ))
+ .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z"))))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory(
+ "a0")))
+ .setContext(REALTIME_QUERY_CTX)
+ .build())
+ .columnMappings(
+ new ColumnMappings(ImmutableList.of(
+ new ColumnMapping("d0", "cnt"),
+ new ColumnMapping("a0", "cnt1")))
+ )
+ .tuningConfig(MSQTuningConfig.defaultConfig())
+ .destination(TaskReportMSQDestination.INSTANCE)
+ .build()
+ )
+ .setQueryContext(REALTIME_QUERY_CTX)
+ .setExpectedRowSignature(rowSignature)
+ .setExpectedResultRows(ImmutableList.of(new Object[]{1L, 2L}))
+ .verifyResults();
+ }
+
+ @Test
+ public void testDataServerQueryFailedShouldFail() throws IOException
+ {
+ RowSignature rowSignature = RowSignature.builder()
+ .add("cnt", ColumnType.LONG)
+ .add("cnt1", ColumnType.LONG)
+ .build();
+
+ doThrow(
+ new ISE("Segment could not be found on data server, but segment was not handed off.")
+ )
+ .when(loadedSegmentDataProvider)
+ .fetchRowsFromDataServer(any(), any(), any(), any());
+
+ testSelectQuery()
+ .setSql("select cnt,count(*) as cnt1 from foo where (TIMESTAMP '2003-01-01 00:00:00' <= \"__time\" AND \"__time\" < TIMESTAMP '2005-01-01 00:00:00') group by cnt")
+ .setExpectedMSQSpec(
+ MSQSpec.builder()
+ .query(GroupByQuery.builder()
+ .setDataSource(CalciteTests.DATASOURCE1)
+ .setInterval(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z"))
+ .setGranularity(Granularities.ALL)
+ .setDimensions(dimensions(
+ new DefaultDimensionSpec(
+ "cnt",
+ "d0",
+ ColumnType.LONG
+ )
+ ))
+ .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2003-01-01T00:00:00.000Z/2005-01-01T00:00:00.000Z"))))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory(
+ "a0")))
+ .setContext(REALTIME_QUERY_CTX)
+ .build())
+ .columnMappings(
+ new ColumnMappings(ImmutableList.of(
+ new ColumnMapping("d0", "cnt"),
+ new ColumnMapping("a0", "cnt1")))
+ )
+ .tuningConfig(MSQTuningConfig.defaultConfig())
+ .destination(TaskReportMSQDestination.INSTANCE)
+ .build()
+ )
+ .setQueryContext(REALTIME_QUERY_CTX)
+ .setExpectedRowSignature(rowSignature)
+ .setExpectedExecutionErrorMatcher(CoreMatchers.instanceOf(ISE.class))
+ .verifyExecutionError();
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java
index ac9ca855a635..d771f7497a8c 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java
@@ -51,6 +51,7 @@
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryDataSource;
import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.UnionDataSource;
import org.apache.druid.query.UnnestDataSource;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
@@ -1929,8 +1930,8 @@ public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOExceptio
new ColumnMappings(ImmutableList.of(
new ColumnMapping("d0", "cnt"),
new ColumnMapping("a0", "cnt1")
- )
))
+ )
.tuningConfig(MSQTuningConfig.defaultConfig())
.destination(isDurableStorageDestination()
? DurableStorageMSQDestination.INSTANCE
@@ -2322,6 +2323,64 @@ public void testSelectUnnestOnQueryFoo()
.verifyResults();
}
+ @Test
+ public void testUnionAllUsingUnionDataSource()
+ {
+
+ final RowSignature rowSignature = RowSignature.builder()
+ .add("__time", ColumnType.LONG)
+ .add("dim1", ColumnType.STRING)
+ .build();
+
+ final List results = ImmutableList.of(
+ new Object[]{946684800000L, ""},
+ new Object[]{946684800000L, ""},
+ new Object[]{946771200000L, "10.1"},
+ new Object[]{946771200000L, "10.1"},
+ new Object[]{946857600000L, "2"},
+ new Object[]{946857600000L, "2"},
+ new Object[]{978307200000L, "1"},
+ new Object[]{978307200000L, "1"},
+ new Object[]{978393600000L, "def"},
+ new Object[]{978393600000L, "def"},
+ new Object[]{978480000000L, "abc"},
+ new Object[]{978480000000L, "abc"}
+ );
+ // This plans the query using DruidUnionDataSourceRule since the DruidUnionDataSourceRule#isCompatible
+ // returns true (column names, types match, and it is a union on the table data sources).
+ // It gets planned correctly, however MSQ engine cannot plan the query correctly
+ testSelectQuery()
+ .setSql("SELECT __time, dim1 FROM foo\n"
+ + "UNION ALL\n"
+ + "SELECT __time, dim1 FROM foo\n")
+ .setExpectedRowSignature(rowSignature)
+ .setExpectedMSQSpec(
+ MSQSpec.builder()
+ .query(newScanQueryBuilder()
+ .dataSource(new UnionDataSource(
+ ImmutableList.of(new TableDataSource("foo"), new TableDataSource("foo"))
+ ))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+ .legacy(false)
+ .context(defaultScanQueryContext(
+ context,
+ rowSignature
+ ))
+ .columns(ImmutableList.of("__time", "dim1"))
+ .build())
+ .columnMappings(ColumnMappings.identity(rowSignature))
+ .tuningConfig(MSQTuningConfig.defaultConfig())
+ .destination(isDurableStorageDestination()
+ ? DurableStorageMSQDestination.INSTANCE
+ : TaskReportMSQDestination.INSTANCE)
+ .build()
+ )
+ .setQueryContext(context)
+ .setExpectedResultRows(results)
+ .verifyResults();
+ }
+
@Nonnull
private List expectedMultiValueFooRowsGroup()
{
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java
index 0ea9ab45f482..2ae8d155d4dc 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java
@@ -49,6 +49,7 @@ public void setup()
injectorMock,
null,
null,
+ null,
null
);
}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java
index 8884a92a665b..935b464e0386 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/RichSegmentDescriptorTest.java
@@ -20,15 +20,28 @@
package org.apache.druid.msq.input.table;
import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableSet;
import nl.jqno.equalsverifier.EqualsVerifier;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.segment.TestHelper;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
import org.junit.Assert;
import org.junit.Test;
public class RichSegmentDescriptorTest
{
+ private static final DruidServerMetadata DRUID_SERVER_1 = new DruidServerMetadata(
+ "name1",
+ "host1",
+ null,
+ 100L,
+ ServerType.REALTIME,
+ "tier1",
+ 0
+ );
+
@Test
public void testSerdeWithFullIntervalDifferentFromInterval() throws Exception
{
@@ -37,7 +50,8 @@ public void testSerdeWithFullIntervalDifferentFromInterval() throws Exception
Intervals.of("2000/2002"),
Intervals.of("2000/2001"),
"2",
- 3
+ 3,
+ ImmutableSet.of(DRUID_SERVER_1)
);
Assert.assertEquals(
@@ -54,7 +68,8 @@ public void testSerdeWithFullIntervalSameAsInterval() throws Exception
Intervals.of("2000/2001"),
Intervals.of("2000/2001"),
"2",
- 3
+ 3,
+ ImmutableSet.of(DRUID_SERVER_1)
);
Assert.assertEquals(
@@ -71,7 +86,8 @@ public void testDeserializeRichSegmentDescriptorAsSegmentDescriptor() throws Exc
Intervals.of("2000/2002"),
Intervals.of("2000/2001"),
"2",
- 3
+ 3,
+ ImmutableSet.of(DRUID_SERVER_1)
);
Assert.assertEquals(
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java
index 29a0ebef4ba4..875bef371e90 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentWithDescriptorTest.java
@@ -19,6 +19,7 @@
package org.apache.druid.msq.input.table;
+import com.fasterxml.jackson.databind.ObjectMapper;
import nl.jqno.equalsverifier.EqualsVerifier;
import org.junit.Test;
@@ -27,6 +28,10 @@ public class SegmentWithDescriptorTest
@Test
public void testEquals()
{
- EqualsVerifier.forClass(SegmentWithDescriptor.class).usingGetClass().verify();
+ EqualsVerifier.forClass(SegmentWithDescriptor.class)
+ .withPrefabValues(ObjectMapper.class, new ObjectMapper(), new ObjectMapper())
+ .withIgnoredFields("loadedSegmentDataProvider")
+ .usingGetClass()
+ .verify();
}
}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java
index df2937f30036..55bb424512d8 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/SegmentsInputSliceTest.java
@@ -21,11 +21,14 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
import nl.jqno.equalsverifier.EqualsVerifier;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.msq.guice.MSQIndexingModule;
import org.apache.druid.msq.input.InputSlice;
import org.apache.druid.segment.TestHelper;
+import org.apache.druid.server.coordination.DruidServerMetadata;
+import org.apache.druid.server.coordination.ServerType;
import org.junit.Assert;
import org.junit.Test;
@@ -44,7 +47,18 @@ public void testSerde() throws Exception
Intervals.of("2000/P1M"),
Intervals.of("2000/P1M"),
"1",
- 0
+ 0,
+ ImmutableSet.of(
+ new DruidServerMetadata(
+ "name1",
+ "host1",
+ null,
+ 100L,
+ ServerType.REALTIME,
+ "tier1",
+ 0
+ )
+ )
)
)
);
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java
index dbcb3646e887..fd5db7e75f64 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java
@@ -146,25 +146,29 @@ public void test_sliceStatic_intervalFilter()
SEGMENT1.getInterval(),
Intervals.of("2000/P1M"),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
),
new RichSegmentDescriptor(
SEGMENT2.getInterval(),
Intervals.of("2000/P1M"),
SEGMENT2.getVersion(),
- SEGMENT2.getShardSpec().getPartitionNum()
+ SEGMENT2.getShardSpec().getPartitionNum(),
+ null
),
new RichSegmentDescriptor(
SEGMENT1.getInterval(),
Intervals.of("2000-06-01/P1M"),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
),
new RichSegmentDescriptor(
SEGMENT2.getInterval(),
Intervals.of("2000-06-01/P1M"),
SEGMENT2.getVersion(),
- SEGMENT2.getShardSpec().getPartitionNum()
+ SEGMENT2.getShardSpec().getPartitionNum(),
+ null
)
)
)
@@ -206,7 +210,8 @@ public void test_sliceStatic_dimFilter()
SEGMENT1.getInterval(),
SEGMENT1.getInterval(),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
)
)
),
@@ -237,7 +242,8 @@ public void test_sliceStatic_intervalAndDimFilter()
SEGMENT1.getInterval(),
Intervals.of("2000/P1M"),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
)
)
),
@@ -248,7 +254,8 @@ public void test_sliceStatic_intervalAndDimFilter()
SEGMENT1.getInterval(),
Intervals.of("2000-06-01/P1M"),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
)
)
)
@@ -270,13 +277,15 @@ public void test_sliceStatic_oneSlice()
SEGMENT1.getInterval(),
SEGMENT1.getInterval(),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
),
new RichSegmentDescriptor(
SEGMENT2.getInterval(),
SEGMENT2.getInterval(),
SEGMENT2.getVersion(),
- SEGMENT2.getShardSpec().getPartitionNum()
+ SEGMENT2.getShardSpec().getPartitionNum(),
+ null
)
)
)
@@ -298,7 +307,8 @@ public void test_sliceStatic_needTwoSlices()
SEGMENT1.getInterval(),
SEGMENT1.getInterval(),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
)
)
),
@@ -309,7 +319,8 @@ public void test_sliceStatic_needTwoSlices()
SEGMENT2.getInterval(),
SEGMENT2.getInterval(),
SEGMENT2.getVersion(),
- SEGMENT2.getShardSpec().getPartitionNum()
+ SEGMENT2.getShardSpec().getPartitionNum(),
+ null
)
)
)
@@ -331,7 +342,8 @@ public void test_sliceStatic_threeSlices()
SEGMENT1.getInterval(),
SEGMENT1.getInterval(),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
)
)
),
@@ -342,7 +354,8 @@ public void test_sliceStatic_threeSlices()
SEGMENT2.getInterval(),
SEGMENT2.getInterval(),
SEGMENT2.getVersion(),
- SEGMENT2.getShardSpec().getPartitionNum()
+ SEGMENT2.getShardSpec().getPartitionNum(),
+ null
)
)
),
@@ -385,13 +398,15 @@ public void test_sliceDynamic_maxOneSlice()
SEGMENT1.getInterval(),
Intervals.of("2000/P1M"),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
),
new RichSegmentDescriptor(
SEGMENT2.getInterval(),
Intervals.of("2000/P1M"),
SEGMENT2.getVersion(),
- SEGMENT2.getShardSpec().getPartitionNum()
+ SEGMENT2.getShardSpec().getPartitionNum(),
+ null
)
)
)
@@ -418,13 +433,15 @@ public void test_sliceDynamic_needOne()
SEGMENT1.getInterval(),
Intervals.of("2000/P1M"),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
),
new RichSegmentDescriptor(
SEGMENT2.getInterval(),
Intervals.of("2000/P1M"),
SEGMENT2.getVersion(),
- SEGMENT2.getShardSpec().getPartitionNum()
+ SEGMENT2.getShardSpec().getPartitionNum(),
+ null
)
)
)
@@ -451,7 +468,8 @@ public void test_sliceDynamic_needTwoDueToFiles()
SEGMENT1.getInterval(),
Intervals.of("2000/P1M"),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
)
)
),
@@ -462,7 +480,8 @@ public void test_sliceDynamic_needTwoDueToFiles()
SEGMENT2.getInterval(),
Intervals.of("2000/P1M"),
SEGMENT2.getVersion(),
- SEGMENT2.getShardSpec().getPartitionNum()
+ SEGMENT2.getShardSpec().getPartitionNum(),
+ null
)
)
)
@@ -489,7 +508,8 @@ public void test_sliceDynamic_needTwoDueToBytes()
SEGMENT1.getInterval(),
Intervals.of("2000/P1M"),
SEGMENT1.getVersion(),
- SEGMENT1.getShardSpec().getPartitionNum()
+ SEGMENT1.getShardSpec().getPartitionNum(),
+ null
)
)
),
@@ -500,7 +520,8 @@ public void test_sliceDynamic_needTwoDueToBytes()
SEGMENT2.getInterval(),
Intervals.of("2000/P1M"),
SEGMENT2.getVersion(),
- SEGMENT2.getShardSpec().getPartitionNum()
+ SEGMENT2.getShardSpec().getPartitionNum(),
+ null
)
)
)
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java
index 82301f4ddfed..abefe6a378d8 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java
@@ -20,6 +20,7 @@
package org.apache.druid.msq.test;
import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
import com.google.inject.Injector;
import com.google.inject.Module;
import org.apache.druid.guice.DruidInjectorBuilder;
@@ -80,7 +81,8 @@ public SqlEngine createEngine(
queryJsonMapper,
injector,
new MSQTestTaskActionClient(queryJsonMapper),
- workerMemoryParameters
+ workerMemoryParameters,
+ ImmutableList.of()
);
return new MSQTaskSqlEngine(indexingServiceClient, queryJsonMapper);
}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java
index d7c0ea1f2d5f..5b49c649cc0c 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java
@@ -27,6 +27,7 @@
import com.google.inject.TypeLiteral;
import org.apache.druid.collections.ReferenceCountingResourceHolder;
import org.apache.druid.collections.ResourceHolder;
+import org.apache.druid.data.input.ResourceInputSource;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.LongDimensionSchema;
import org.apache.druid.data.input.impl.StringDimensionSchema;
@@ -40,11 +41,14 @@
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.msq.exec.LoadedSegmentDataProvider;
+import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory;
import org.apache.druid.msq.guice.MSQExternalDataSourceModule;
import org.apache.druid.msq.guice.MSQIndexingModule;
import org.apache.druid.msq.querykit.DataSegmentProvider;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.ForwardingQueryProcessingPool;
+import org.apache.druid.query.NestedDataTestUtils;
import org.apache.druid.query.QueryProcessingPool;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
@@ -72,15 +76,17 @@
import org.apache.druid.server.SegmentManager;
import org.apache.druid.server.coordination.DataSegmentAnnouncer;
import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer;
+import org.apache.druid.sql.calcite.CalciteArraysQueryTest;
import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.apache.druid.sql.calcite.util.TestDataBuilder;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
import javax.annotation.Nullable;
-import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Set;
@@ -96,6 +102,10 @@
import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS1_WITH_NUMERIC_DIMS;
import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS2;
import static org.apache.druid.sql.calcite.util.TestDataBuilder.ROWS_LOTS_OF_COLUMNS;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
/**
* Helper class aiding in wiring up the Guice bindings required for MSQ engine to work with the Calcite's tests
@@ -165,6 +175,7 @@ public String getFormatString()
binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer());
binder.bind(DataSegmentProvider.class)
.toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId));
+ binder.bind(LoadedSegmentDataProviderFactory.class).toInstance(getTestLoadedSegmentDataProviderFactory());
GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig();
GroupingEngine groupingEngine = GroupByQueryRunnerTest.makeQueryRunnerFactory(
@@ -182,6 +193,24 @@ public String getFormatString()
);
}
+ private static LoadedSegmentDataProviderFactory getTestLoadedSegmentDataProviderFactory()
+ {
+ // Currently, there is no metadata in this test for loaded segments. Therefore, this should not be called.
+ // In the future, if this needs to be supported, mocks for LoadedSegmentDataProvider should be added like
+ // org.apache.druid.msq.exec.MSQLoadedSegmentTests.
+ LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class);
+ LoadedSegmentDataProvider loadedSegmentDataProvider = Mockito.mock(LoadedSegmentDataProvider.class);
+ try {
+ doThrow(new AssertionError("Test does not support loaded segment query"))
+ .when(loadedSegmentDataProvider).fetchRowsFromDataServer(any(), any(), any(), any());
+ doReturn(loadedSegmentDataProvider).when(mockFactory).createLoadedSegmentDataProvider(anyString(), any());
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ return mockFactory;
+ }
+
private static Supplier> getSupplierForSegment(SegmentId segmentId)
{
final TemporaryFolder temporaryFolder = new TemporaryFolder();
@@ -206,7 +235,7 @@ private static Supplier> getSupplierForSegment(SegmentId
.build();
index = IndexBuilder
.create()
- .tmpDir(new File(temporaryFolder.newFolder(), "1"))
+ .tmpDir(temporaryFolder.newFolder())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.schema(foo1Schema)
.rows(ROWS1)
@@ -233,7 +262,7 @@ private static Supplier> getSupplierForSegment(SegmentId
.build();
index = IndexBuilder
.create()
- .tmpDir(new File(temporaryFolder.newFolder(), "2"))
+ .tmpDir(temporaryFolder.newFolder())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.schema(indexSchemaDifferentDim3M1Types)
.rows(ROWS2)
@@ -243,7 +272,7 @@ private static Supplier> getSupplierForSegment(SegmentId
case CalciteTests.BROADCAST_DATASOURCE:
index = IndexBuilder
.create()
- .tmpDir(new File(temporaryFolder.newFolder(), "3"))
+ .tmpDir(temporaryFolder.newFolder())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.schema(INDEX_SCHEMA_NUMERIC_DIMS)
.rows(ROWS1_WITH_NUMERIC_DIMS)
@@ -252,12 +281,36 @@ private static Supplier> getSupplierForSegment(SegmentId
case DATASOURCE5:
index = IndexBuilder
.create()
- .tmpDir(new File(temporaryFolder.newFolder(), "5"))
+ .tmpDir(temporaryFolder.newFolder())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.schema(INDEX_SCHEMA_LOTS_O_COLUMNS)
.rows(ROWS_LOTS_OF_COLUMNS)
.buildMMappedIndex();
break;
+ case CalciteArraysQueryTest.DATA_SOURCE_ARRAYS:
+ index = IndexBuilder.create()
+ .tmpDir(temporaryFolder.newFolder())
+ .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
+ .schema(
+ new IncrementalIndexSchema.Builder()
+ .withTimestampSpec(NestedDataTestUtils.AUTO_SCHEMA.getTimestampSpec())
+ .withDimensionsSpec(NestedDataTestUtils.AUTO_SCHEMA.getDimensionsSpec())
+ .withMetrics(
+ new CountAggregatorFactory("cnt")
+ )
+ .withRollup(false)
+ .build()
+ )
+ .inputSource(
+ ResourceInputSource.of(
+ NestedDataTestUtils.class.getClassLoader(),
+ NestedDataTestUtils.ARRAY_TYPES_DATA_FILE
+ )
+ )
+ .inputFormat(TestDataBuilder.DEFAULT_JSON_INPUT_FORMAT)
+ .inputTmpDir(temporaryFolder.newFolder())
+ .buildMMappedIndex();
+ break;
default:
throw new ISE("Cannot query segment %s in test runner", segmentId);
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java
index ab7b1ed7d7cf..114583d31a1a 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectJoinQueryMSQTest.java
@@ -20,6 +20,7 @@
package org.apache.druid.msq.test;
import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
import com.google.inject.Injector;
import com.google.inject.Module;
import org.apache.calcite.rel.RelRoot;
@@ -128,7 +129,8 @@ public SqlEngine createEngine(
queryJsonMapper,
injector,
new MSQTestTaskActionClient(queryJsonMapper),
- workerMemoryParameters
+ workerMemoryParameters,
+ ImmutableList.of()
);
return new MSQTaskSqlEngine(indexingServiceClient, queryJsonMapper)
{
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java
index 5ee3ba875388..e9c54cfc54b4 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java
@@ -85,7 +85,8 @@ public SqlEngine createEngine(
queryJsonMapper,
injector,
new MSQTestTaskActionClient(queryJsonMapper),
- workerMemoryParameters
+ workerMemoryParameters,
+ ImmutableList.of()
);
return new MSQTaskSqlEngine(indexingServiceClient, queryJsonMapper);
}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteUnionQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteUnionQueryMSQTest.java
new file mode 100644
index 000000000000..6ec17687c45e
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteUnionQueryMSQTest.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.msq.test;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.inject.Injector;
+import com.google.inject.Module;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.guice.DruidInjectorBuilder;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.msq.exec.WorkerMemoryParameters;
+import org.apache.druid.msq.sql.MSQTaskSqlEngine;
+import org.apache.druid.query.QueryDataSource;
+import org.apache.druid.query.TableDataSource;
+import org.apache.druid.query.UnionDataSource;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.TestGroupByBuffers;
+import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
+import org.apache.druid.sql.calcite.CalciteUnionQueryTest;
+import org.apache.druid.sql.calcite.QueryTestBuilder;
+import org.apache.druid.sql.calcite.filtration.Filtration;
+import org.apache.druid.sql.calcite.run.SqlEngine;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * Runs {@link CalciteUnionQueryTest} but with MSQ engine
+ */
+public class CalciteUnionQueryMSQTest extends CalciteUnionQueryTest
+{
+ private TestGroupByBuffers groupByBuffers;
+
+ @Before
+ public void setup2()
+ {
+ groupByBuffers = TestGroupByBuffers.createDefault();
+ }
+
+ @After
+ public void teardown2()
+ {
+ groupByBuffers.close();
+ }
+
+ @Override
+ public void configureGuice(DruidInjectorBuilder builder)
+ {
+ super.configureGuice(builder);
+ builder.addModules(CalciteMSQTestsHelper.fetchModules(temporaryFolder, groupByBuffers).toArray(new Module[0]));
+ }
+
+
+ @Override
+ public SqlEngine createEngine(
+ QueryLifecycleFactory qlf,
+ ObjectMapper queryJsonMapper,
+ Injector injector
+ )
+ {
+ final WorkerMemoryParameters workerMemoryParameters =
+ WorkerMemoryParameters.createInstance(
+ WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50,
+ 2,
+ 10,
+ 2,
+ 0,
+ 0
+ );
+ final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient(
+ queryJsonMapper,
+ injector,
+ new MSQTestTaskActionClient(queryJsonMapper),
+ workerMemoryParameters,
+ ImmutableList.of()
+ );
+ return new MSQTaskSqlEngine(indexingServiceClient, queryJsonMapper);
+ }
+
+ @Override
+ protected QueryTestBuilder testBuilder()
+ {
+ return new QueryTestBuilder(new BaseCalciteQueryTest.CalciteTestConfig(true))
+ .addCustomRunner(new ExtractResultsFactory(() -> (MSQTestOverlordServiceClient) ((MSQTaskSqlEngine) queryFramework().engine()).overlordClient()))
+ .skipVectorize(true)
+ .verifyNativeQueries(new VerifyMSQSupportedNativeQueriesPredicate())
+ .msqCompatible(msqCompatible);
+ }
+
+ /**
+ * Generates a different error hint than what is required by the native engine, since planner does try to plan "UNION"
+ * using group by, however fails due to the column name mismatch.
+ * MSQ does wnat to support any type of data source, with least restrictive column names and types, therefore it
+ * should eventually work.
+ */
+ @Test
+ @Override
+ public void testUnionIsUnplannable()
+ {
+ assertQueryIsUnplannable(
+ "SELECT dim2, dim1, m1 FROM foo2 UNION SELECT dim1, dim2, m1 FROM foo",
+ "SQL requires union between two tables and column names queried for each table are different Left: [dim2, dim1, m1], Right: [dim1, dim2, m1]."
+ );
+
+ }
+
+ @Ignore("Ignored till MSQ can plan UNION ALL with any operand")
+ @Test
+ public void testUnionOnSubqueries()
+ {
+ testQuery(
+ "SELECT\n"
+ + " SUM(cnt),\n"
+ + " COUNT(*)\n"
+ + "FROM (\n"
+ + " (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2)\n"
+ + " UNION ALL\n"
+ + " (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2)\n"
+ + ")",
+ ImmutableList.of(
+ GroupByQuery.builder()
+ .setDataSource(
+ new QueryDataSource(
+ GroupByQuery.builder()
+ .setDataSource(
+ new UnionDataSource(
+ ImmutableList.of(
+ new TableDataSource(CalciteTests.DATASOURCE1),
+ new TableDataSource(CalciteTests.DATASOURCE1)
+ )
+ )
+ )
+ .setInterval(querySegmentSpec(Filtration.eternity()))
+ .setGranularity(Granularities.ALL)
+ .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
+ .setContext(QUERY_CONTEXT_DEFAULT)
+ .build()
+ )
+ )
+ .setInterval(querySegmentSpec(Filtration.eternity()))
+ .setGranularity(Granularities.ALL)
+ .setAggregatorSpecs(aggregators(
+ new LongSumAggregatorFactory("_a0", "a0"),
+ new CountAggregatorFactory("_a1")
+ ))
+ .setContext(QUERY_CONTEXT_DEFAULT)
+ .build()
+ ),
+ NullHandling.replaceWithDefault() ?
+ ImmutableList.of(
+ new Object[]{12L, 3L}
+ ) :
+ ImmutableList.of(
+ new Object[]{12L, 4L}
+ )
+ );
+ }
+
+}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java
index 1146bb1c9d16..31ece253ebd4 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java
@@ -36,6 +36,7 @@
import com.google.inject.TypeLiteral;
import com.google.inject.util.Modules;
import com.google.inject.util.Providers;
+import org.apache.druid.client.ImmutableSegmentLoadInfo;
import org.apache.druid.collections.ReferenceCountingResourceHolder;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.common.config.NullHandling;
@@ -83,6 +84,8 @@
import org.apache.druid.msq.counters.QueryCounterSnapshot;
import org.apache.druid.msq.exec.ClusterStatisticsMergeMode;
import org.apache.druid.msq.exec.Controller;
+import org.apache.druid.msq.exec.LoadedSegmentDataProvider;
+import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory;
import org.apache.druid.msq.exec.WorkerMemoryParameters;
import org.apache.druid.msq.guice.MSQDurableStorageModule;
import org.apache.druid.msq.guice.MSQExternalDataSourceModule;
@@ -293,6 +296,11 @@ public class MSQTestBase extends BaseCalciteQueryTest
protected AuthorizerMapper authorizerMapper;
private IndexIO indexIO;
+ // Contains the metadata of loaded segments
+ protected List loadedSegmentsMetadata = new ArrayList<>();
+ // Mocks the return of data from data servers
+ protected LoadedSegmentDataProvider loadedSegmentDataProvider = mock(LoadedSegmentDataProvider.class);
+
private MSQTestSegmentManager segmentManager;
private SegmentCacheManager segmentCacheManager;
@Rule
@@ -416,7 +424,8 @@ public String getFormatString()
binder.bind(QueryProcessingPool.class)
.toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool")));
binder.bind(DataSegmentProvider.class)
- .toInstance((dataSegment, channelCounters, isReindex) -> getSupplierForSegment(dataSegment));
+ .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId));
+ binder.bind(LoadedSegmentDataProviderFactory.class).toInstance(getTestLoadedSegmentDataProviderFactory());
binder.bind(IndexIO.class).toInstance(indexIO);
binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker());
@@ -497,7 +506,8 @@ public String getFormatString()
objectMapper,
injector,
testTaskActionClient,
- workerMemoryParameters
+ workerMemoryParameters,
+ loadedSegmentsMetadata
);
CatalogResolver catalogResolver = createMockCatalogResolver();
final InProcessViewManager viewManager = new InProcessViewManager(SqlTestFramework.DRUID_VIEW_MACRO_FACTORY);
@@ -570,6 +580,15 @@ protected long[] createExpectedFrameArray(int length, int value)
return array;
}
+ private LoadedSegmentDataProviderFactory getTestLoadedSegmentDataProviderFactory()
+ {
+ LoadedSegmentDataProviderFactory mockFactory = Mockito.mock(LoadedSegmentDataProviderFactory.class);
+ doReturn(loadedSegmentDataProvider)
+ .when(mockFactory)
+ .createLoadedSegmentDataProvider(anyString(), any());
+ return mockFactory;
+ }
+
@Nonnull
private Supplier> getSupplierForSegment(SegmentId segmentId)
{
@@ -1393,9 +1412,11 @@ public Pair, List>>
public void verifyResults()
{
- Preconditions.checkArgument(expectedResultRows != null, "Result rows cannot be null");
- Preconditions.checkArgument(expectedRowSignature != null, "Row signature cannot be null");
- Preconditions.checkArgument(expectedMSQSpec != null, "MultiStageQuery Query spec cannot be null ");
+ if (expectedMSQFault == null) {
+ Preconditions.checkArgument(expectedResultRows != null, "Result rows cannot be null");
+ Preconditions.checkArgument(expectedRowSignature != null, "Row signature cannot be null");
+ Preconditions.checkArgument(expectedMSQSpec != null, "MultiStageQuery Query spec cannot be null ");
+ }
Pair, List>> specAndResults = runQueryWithResult();
if (specAndResults == null) { // A fault was expected and the assertion has been done in the runQueryWithResult
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java
index 027d2a913b21..c62be112eed4 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java
@@ -26,6 +26,7 @@
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.inject.Injector;
+import org.apache.druid.client.ImmutableSegmentLoadInfo;
import org.apache.druid.client.coordinator.CoordinatorClient;
import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskState;
@@ -56,6 +57,7 @@
import javax.annotation.Nullable;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
@@ -94,7 +96,8 @@ public MSQTestControllerContext(
ObjectMapper mapper,
Injector injector,
TaskActionClient taskActionClient,
- WorkerMemoryParameters workerMemoryParameters
+ WorkerMemoryParameters workerMemoryParameters,
+ List loadedSegments
)
{
this.mapper = mapper;
@@ -115,6 +118,18 @@ public MSQTestControllerContext(
.collect(Collectors.toList())
)
);
+
+ Mockito.when(coordinatorClient.fetchServerViewSegments(
+ ArgumentMatchers.anyString(),
+ ArgumentMatchers.any()
+ )
+ ).thenAnswer(invocation -> loadedSegments.stream()
+ .filter(immutableSegmentLoadInfo ->
+ immutableSegmentLoadInfo.getSegment()
+ .getDataSource()
+ .equals(invocation.getArguments()[0]))
+ .collect(Collectors.toList())
+ );
this.workerMemoryParameters = workerMemoryParameters;
}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java
index 1b49982cad46..c5f601d875ef 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java
@@ -25,6 +25,7 @@
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import com.google.inject.Injector;
+import org.apache.druid.client.ImmutableSegmentLoadInfo;
import org.apache.druid.client.indexing.NoopOverlordClient;
import org.apache.druid.client.indexing.TaskPayloadResponse;
import org.apache.druid.client.indexing.TaskStatusResponse;
@@ -43,6 +44,7 @@
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
public class MSQTestOverlordServiceClient extends NoopOverlordClient
@@ -51,10 +53,11 @@ public class MSQTestOverlordServiceClient extends NoopOverlordClient
private final ObjectMapper objectMapper;
private final TaskActionClient taskActionClient;
private final WorkerMemoryParameters workerMemoryParameters;
- private Map inMemoryControllers = new HashMap<>();
- private Map> reports = new HashMap<>();
- private Map inMemoryControllerTask = new HashMap<>();
- private Map inMemoryTaskStatus = new HashMap<>();
+ private final List loadedSegmentMetadata;
+ private final Map inMemoryControllers = new HashMap<>();
+ private final Map> reports = new HashMap<>();
+ private final Map inMemoryControllerTask = new HashMap<>();
+ private final Map inMemoryTaskStatus = new HashMap<>();
public static final DateTime CREATED_TIME = DateTimes.of("2023-05-31T12:00Z");
public static final DateTime QUEUE_INSERTION_TIME = DateTimes.of("2023-05-31T12:01Z");
@@ -65,13 +68,15 @@ public MSQTestOverlordServiceClient(
ObjectMapper objectMapper,
Injector injector,
TaskActionClient taskActionClient,
- WorkerMemoryParameters workerMemoryParameters
+ WorkerMemoryParameters workerMemoryParameters,
+ List loadedSegmentMetadata
)
{
this.objectMapper = objectMapper;
this.injector = injector;
this.taskActionClient = taskActionClient;
this.workerMemoryParameters = workerMemoryParameters;
+ this.loadedSegmentMetadata = loadedSegmentMetadata;
}
@Override
@@ -84,7 +89,8 @@ public ListenableFuture runTask(String taskId, Object taskObject)
objectMapper,
injector,
taskActionClient,
- workerMemoryParameters
+ workerMemoryParameters,
+ loadedSegmentMetadata
);
MSQControllerTask cTask = objectMapper.convertValue(taskObject, MSQControllerTask.class);
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java
index a478d1c3c171..51d83397ccae 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java
@@ -29,6 +29,7 @@
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.msq.exec.Controller;
import org.apache.druid.msq.exec.ControllerClient;
+import org.apache.druid.msq.exec.LoadedSegmentDataProviderFactory;
import org.apache.druid.msq.exec.Worker;
import org.apache.druid.msq.exec.WorkerClient;
import org.apache.druid.msq.exec.WorkerContext;
@@ -154,10 +155,12 @@ public void setObjectMapper(ObjectMapper objectMapper)
injector,
indexIO,
null,
+ null,
null
),
indexIO,
injector.getInstance(DataSegmentProvider.class),
+ injector.getInstance(LoadedSegmentDataProviderFactory.class),
workerMemoryParameters
);
}
@@ -179,4 +182,10 @@ public Bouncer processorBouncer()
{
return injector.getInstance(Bouncer.class);
}
+
+ @Override
+ public LoadedSegmentDataProviderFactory loadedSegmentDataProviderFactory()
+ {
+ return injector.getInstance(LoadedSegmentDataProviderFactory.class);
+ }
}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java
index 830b414daedb..5bfb4d2eb279 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java
@@ -39,6 +39,7 @@
import java.util.List;
import java.util.Map;
+import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_ARRAY_INGEST_MODE;
import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE;
import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_FAULT_TOLERANCE;
import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS;
@@ -54,46 +55,46 @@
public class MultiStageQueryContextTest
{
@Test
- public void isDurableShuffleStorageEnabled_noParameterSetReturnsDefaultValue()
+ public void isDurableShuffleStorageEnabled_unset_returnsDefaultValue()
{
Assert.assertFalse(MultiStageQueryContext.isDurableStorageEnabled(QueryContext.empty()));
}
@Test
- public void isDurableShuffleStorageEnabled_parameterSetReturnsCorrectValue()
+ public void isDurableShuffleStorageEnabled_set_returnsCorrectValue()
{
Map propertyMap = ImmutableMap.of(CTX_DURABLE_SHUFFLE_STORAGE, "true");
Assert.assertTrue(MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(propertyMap)));
}
@Test
- public void isFaultToleranceEnabled_noParameterSetReturnsDefaultValue()
+ public void isFaultToleranceEnabled_unset_returnsDefaultValue()
{
Assert.assertFalse(MultiStageQueryContext.isFaultToleranceEnabled(QueryContext.empty()));
}
@Test
- public void isFaultToleranceEnabled_parameterSetReturnsCorrectValue()
+ public void isFaultToleranceEnabled_set_returnsCorrectValue()
{
Map propertyMap = ImmutableMap.of(CTX_FAULT_TOLERANCE, "true");
Assert.assertTrue(MultiStageQueryContext.isFaultToleranceEnabled(QueryContext.of(propertyMap)));
}
@Test
- public void isFinalizeAggregations_noParameterSetReturnsDefaultValue()
+ public void isFinalizeAggregations_unset_returnsDefaultValue()
{
Assert.assertTrue(MultiStageQueryContext.isFinalizeAggregations(QueryContext.empty()));
}
@Test
- public void isFinalizeAggregations_parameterSetReturnsCorrectValue()
+ public void isFinalizeAggregations_set_returnsCorrectValue()
{
Map propertyMap = ImmutableMap.of(CTX_FINALIZE_AGGREGATIONS, "false");
Assert.assertFalse(MultiStageQueryContext.isFinalizeAggregations(QueryContext.of(propertyMap)));
}
@Test
- public void getAssignmentStrategy_noParameterSetReturnsDefaultValue()
+ public void getAssignmentStrategy_unset_returnsDefaultValue()
{
Assert.assertEquals(
WorkerAssignmentStrategy.MAX,
@@ -102,7 +103,7 @@ public void getAssignmentStrategy_noParameterSetReturnsDefaultValue()
}
@Test
- public void testGetMaxInputBytesPerWorker()
+ public void getMaxInputBytesPerWorker_set_returnsCorrectValue()
{
Map propertyMap = ImmutableMap.of(MultiStageQueryContext.CTX_MAX_INPUT_BYTES_PER_WORKER, 1024);
@@ -112,7 +113,7 @@ public void testGetMaxInputBytesPerWorker()
}
@Test
- public void getAssignmentStrategy_parameterSetReturnsCorrectValue()
+ public void getAssignmentStrategy_set_returnsCorrectValue()
{
Map propertyMap = ImmutableMap.of(CTX_TASK_ASSIGNMENT_STRATEGY, "AUTO");
Assert.assertEquals(
@@ -122,27 +123,20 @@ public void getAssignmentStrategy_parameterSetReturnsCorrectValue()
}
@Test
- public void getMaxNumTasks_noParameterSetReturnsDefaultValue()
+ public void getMaxNumTasks_unset_returnsDefaultValue()
{
Assert.assertEquals(DEFAULT_MAX_NUM_TASKS, MultiStageQueryContext.getMaxNumTasks(QueryContext.empty()));
}
@Test
- public void getMaxNumTasks_parameterSetReturnsCorrectValue()
+ public void getMaxNumTasks_set_returnsCorrectValue()
{
Map propertyMap = ImmutableMap.of(CTX_MAX_NUM_TASKS, 101);
Assert.assertEquals(101, MultiStageQueryContext.getMaxNumTasks(QueryContext.of(propertyMap)));
}
@Test
- public void getMaxNumTasks_legacyParameterSetReturnsCorrectValue()
- {
- Map propertyMap = ImmutableMap.of(CTX_MAX_NUM_TASKS, 101);
- Assert.assertEquals(101, MultiStageQueryContext.getMaxNumTasks(QueryContext.of(propertyMap)));
- }
-
- @Test
- public void getRowsPerSegment_noParameterSetReturnsDefaultValue()
+ public void getRowsPerSegment_unset_returnsDefaultValue()
{
Assert.assertEquals(
MultiStageQueryContext.DEFAULT_ROWS_PER_SEGMENT,
@@ -151,14 +145,14 @@ public void getRowsPerSegment_noParameterSetReturnsDefaultValue()
}
@Test
- public void getRowsPerSegment_parameterSetReturnsCorrectValue()
+ public void getRowsPerSegment_set_returnsCorrectValue()
{
Map propertyMap = ImmutableMap.of(CTX_ROWS_PER_SEGMENT, 10);
Assert.assertEquals(10, MultiStageQueryContext.getRowsPerSegment(QueryContext.of(propertyMap)));
}
@Test
- public void getRowsInMemory_noParameterSetReturnsDefaultValue()
+ public void getRowsInMemory_unset_returnsDefaultValue()
{
Assert.assertEquals(
MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY,
@@ -167,12 +161,91 @@ public void getRowsInMemory_noParameterSetReturnsDefaultValue()
}
@Test
- public void getRowsInMemory_parameterSetReturnsCorrectValue()
+ public void getRowsInMemory_set_returnsCorrectValue()
{
Map propertyMap = ImmutableMap.of(CTX_ROWS_IN_MEMORY, 10);
Assert.assertEquals(10, MultiStageQueryContext.getRowsInMemory(QueryContext.of(propertyMap)));
}
+ @Test
+ public void getSortOrder_unset_returnsDefaultValue()
+ {
+ Assert.assertEquals(Collections.emptyList(), MultiStageQueryContext.getSortOrder(QueryContext.empty()));
+ }
+
+ @Test
+ public void getSortOrder_set_returnsCorrectValue()
+ {
+ Map propertyMap = ImmutableMap.of(CTX_SORT_ORDER, "a, b,\"c,d\"");
+ Assert.assertEquals(
+ ImmutableList.of("a", "b", "c,d"),
+ MultiStageQueryContext.getSortOrder(QueryContext.of(propertyMap))
+ );
+ }
+
+ @Test
+ public void getMSQMode_unset_returnsDefaultValue()
+ {
+ Assert.assertEquals("strict", MultiStageQueryContext.getMSQMode(QueryContext.empty()));
+ }
+
+ @Test
+ public void getMSQMode_set_returnsCorrectValue()
+ {
+ Map propertyMap = ImmutableMap.of(CTX_MSQ_MODE, "nonStrict");
+ Assert.assertEquals("nonStrict", MultiStageQueryContext.getMSQMode(QueryContext.of(propertyMap)));
+ }
+
+ @Test
+ public void getSelectDestination_unset_returnsDefaultValue()
+ {
+ Assert.assertEquals(MSQSelectDestination.TASKREPORT, MultiStageQueryContext.getSelectDestination(QueryContext.empty()));
+ }
+
+ @Test
+ public void useAutoColumnSchemes_unset_returnsDefaultValue()
+ {
+ Assert.assertFalse(MultiStageQueryContext.useAutoColumnSchemas(QueryContext.empty()));
+ }
+
+ @Test
+ public void useAutoColumnSchemes_set_returnsCorrectValue()
+ {
+ Map propertyMap = ImmutableMap.of(CTX_USE_AUTO_SCHEMAS, true);
+ Assert.assertTrue(MultiStageQueryContext.useAutoColumnSchemas(QueryContext.of(propertyMap)));
+ }
+
+ @Test
+ public void arrayIngestMode_unset_returnsDefaultValue()
+ {
+ Assert.assertEquals(ArrayIngestMode.MVD, MultiStageQueryContext.getArrayIngestMode(QueryContext.empty()));
+ }
+
+ @Test
+ public void arrayIngestMode_set_returnsCorrectValue()
+ {
+ Assert.assertEquals(
+ ArrayIngestMode.NONE,
+ MultiStageQueryContext.getArrayIngestMode(QueryContext.of(ImmutableMap.of(CTX_ARRAY_INGEST_MODE, "none")))
+ );
+
+ Assert.assertEquals(
+ ArrayIngestMode.MVD,
+ MultiStageQueryContext.getArrayIngestMode(QueryContext.of(ImmutableMap.of(CTX_ARRAY_INGEST_MODE, "mvd")))
+ );
+
+ Assert.assertEquals(
+ ArrayIngestMode.ARRAY,
+ MultiStageQueryContext.getArrayIngestMode(QueryContext.of(ImmutableMap.of(CTX_ARRAY_INGEST_MODE, "array")))
+ );
+
+ Assert.assertThrows(
+ BadQueryContextException.class,
+ () ->
+ MultiStageQueryContext.getArrayIngestMode(QueryContext.of(ImmutableMap.of(CTX_ARRAY_INGEST_MODE, "dummy")))
+ );
+ }
+
@Test
public void testDecodeSortOrder()
{
@@ -221,48 +294,6 @@ public void testGetIndexSpec()
);
}
- @Test
- public void getSortOrderNoParameterSetReturnsDefaultValue()
- {
- Assert.assertEquals(Collections.emptyList(), MultiStageQueryContext.getSortOrder(QueryContext.empty()));
- }
-
- @Test
- public void getSortOrderParameterSetReturnsCorrectValue()
- {
- Map propertyMap = ImmutableMap.of(CTX_SORT_ORDER, "a, b,\"c,d\"");
- Assert.assertEquals(
- ImmutableList.of("a", "b", "c,d"),
- MultiStageQueryContext.getSortOrder(QueryContext.of(propertyMap))
- );
- }
-
- @Test
- public void getMSQModeNoParameterSetReturnsDefaultValue()
- {
- Assert.assertEquals("strict", MultiStageQueryContext.getMSQMode(QueryContext.empty()));
- }
-
- @Test
- public void getMSQModeParameterSetReturnsCorrectValue()
- {
- Map propertyMap = ImmutableMap.of(CTX_MSQ_MODE, "nonStrict");
- Assert.assertEquals("nonStrict", MultiStageQueryContext.getMSQMode(QueryContext.of(propertyMap)));
- }
-
- @Test
- public void limitSelectResultReturnsDefaultValue()
- {
- Assert.assertEquals(MSQSelectDestination.TASKREPORT, MultiStageQueryContext.getSelectDestination(QueryContext.empty()));
- }
-
- @Test
- public void testUseAutoSchemas()
- {
- Map propertyMap = ImmutableMap.of(CTX_USE_AUTO_SCHEMAS, true);
- Assert.assertTrue(MultiStageQueryContext.useAutoColumnSchemas(QueryContext.of(propertyMap)));
- }
-
private static List decodeSortOrder(@Nullable final String input)
{
return MultiStageQueryContext.decodeSortOrder(input);
diff --git a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java
index 0b1562eb83d1..b2ed565d6276 100644
--- a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java
+++ b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java
@@ -21,15 +21,14 @@
import com.google.common.collect.ImmutableList;
import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregatorFactory;
@@ -39,15 +38,14 @@
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.column.ColumnType;
-import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
import org.apache.druid.sql.calcite.aggregation.Aggregations;
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
-import org.apache.druid.sql.calcite.expression.Expressions;
import org.apache.druid.sql.calcite.expression.OperatorConversions;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.rel.InputAccessor;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import org.apache.druid.sql.calcite.table.RowSignatures;
@@ -60,41 +58,35 @@ public abstract class BaseVarianceSqlAggregator implements SqlAggregator
private static final String STDDEV_NAME = "STDDEV";
private static final SqlAggFunction VARIANCE_SQL_AGG_FUNC_INSTANCE =
- buildSqlAvgAggFunction(VARIANCE_NAME);
+ buildSqlVarianceAggFunction(VARIANCE_NAME);
private static final SqlAggFunction VARIANCE_POP_SQL_AGG_FUNC_INSTANCE =
- buildSqlAvgAggFunction(SqlKind.VAR_POP.name());
+ buildSqlVarianceAggFunction(SqlKind.VAR_POP.name());
private static final SqlAggFunction VARIANCE_SAMP_SQL_AGG_FUNC_INSTANCE =
- buildSqlAvgAggFunction(SqlKind.VAR_SAMP.name());
+ buildSqlVarianceAggFunction(SqlKind.VAR_SAMP.name());
private static final SqlAggFunction STDDEV_SQL_AGG_FUNC_INSTANCE =
- buildSqlAvgAggFunction(STDDEV_NAME);
+ buildSqlVarianceAggFunction(STDDEV_NAME);
private static final SqlAggFunction STDDEV_POP_SQL_AGG_FUNC_INSTANCE =
- buildSqlAvgAggFunction(SqlKind.STDDEV_POP.name());
+ buildSqlVarianceAggFunction(SqlKind.STDDEV_POP.name());
private static final SqlAggFunction STDDEV_SAMP_SQL_AGG_FUNC_INSTANCE =
- buildSqlAvgAggFunction(SqlKind.STDDEV_SAMP.name());
+ buildSqlVarianceAggFunction(SqlKind.STDDEV_SAMP.name());
@Nullable
@Override
public Aggregation toDruidAggregation(
PlannerContext plannerContext,
- RowSignature rowSignature,
VirtualColumnRegistry virtualColumnRegistry,
- RexBuilder rexBuilder,
String name,
AggregateCall aggregateCall,
- Project project,
+ InputAccessor inputAccessor,
List existingAggregations,
boolean finalizeAggregations
)
{
- final RexNode inputOperand = Expressions.fromFieldAccess(
- rexBuilder.getTypeFactory(),
- rowSignature,
- project,
- aggregateCall.getArgList().get(0)
- );
+ final RexNode inputOperand = inputAccessor.getField(aggregateCall.getArgList().get(0));
+
final DruidExpression input = Aggregations.toDruidExpressionForNumericAggregator(
plannerContext,
- rowSignature,
+ inputAccessor.getInputRowSignature(),
inputOperand
);
if (input == null) {
@@ -160,14 +152,15 @@ public Aggregation toDruidAggregation(
}
/**
- * Creates a {@link SqlAggFunction} that is the same as {@link org.apache.calcite.sql.fun.SqlAvgAggFunction}
- * but with an operand type that accepts variance aggregator objects in addition to numeric inputs.
+ * Creates a {@link SqlAggFunction}
+ *
+ * It accepts variance aggregator objects in addition to numeric inputs.
*/
- private static SqlAggFunction buildSqlAvgAggFunction(String name)
+ private static SqlAggFunction buildSqlVarianceAggFunction(String name)
{
return OperatorConversions
.aggregatorBuilder(name)
- .returnTypeInference(ReturnTypes.AVG_AGG_FUNCTION)
+ .returnTypeInference(ReturnTypes.explicit(SqlTypeName.DOUBLE))
.operandTypeChecker(
OperandTypes.or(
OperandTypes.NUMERIC,
diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java
index fe68b2737ef3..e45a93784967 100644
--- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java
+++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java
@@ -171,8 +171,8 @@ public void testVarPop()
final List expectedResults = ImmutableList.of(
new Object[]{
holder1.getVariance(true),
- holder2.getVariance(true).doubleValue(),
- holder3.getVariance(true).longValue()
+ holder2.getVariance(true),
+ holder3.getVariance(true)
}
);
testQuery(
@@ -219,7 +219,7 @@ public void testVarSamp()
new Object[] {
holder1.getVariance(false),
holder2.getVariance(false).doubleValue(),
- holder3.getVariance(false).longValue(),
+ holder3.getVariance(false),
}
);
testQuery(
@@ -266,7 +266,7 @@ public void testStdDevPop()
new Object[] {
Math.sqrt(holder1.getVariance(true)),
Math.sqrt(holder2.getVariance(true)),
- (long) Math.sqrt(holder3.getVariance(true)),
+ Math.sqrt(holder3.getVariance(true)),
}
);
@@ -321,7 +321,7 @@ public void testStdDevSamp()
new Object[]{
Math.sqrt(holder1.getVariance(false)),
Math.sqrt(holder2.getVariance(false)),
- (long) Math.sqrt(holder3.getVariance(false)),
+ Math.sqrt(holder3.getVariance(false)),
}
);
@@ -374,7 +374,7 @@ public void testStdDevWithVirtualColumns()
new Object[]{
Math.sqrt(holder1.getVariance(false)),
Math.sqrt(holder2.getVariance(false)),
- (long) Math.sqrt(holder3.getVariance(false)),
+ Math.sqrt(holder3.getVariance(false)),
}
);
@@ -543,7 +543,7 @@ public void testEmptyTimeseriesResults()
),
ImmutableList.of(
NullHandling.replaceWithDefault()
- ? new Object[]{0.0, 0.0, 0.0, 0.0, 0L, 0L, 0L, 0L}
+ ? new Object[]{0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0}
: new Object[]{null, null, null, null, null, null, null, null}
)
);
@@ -623,7 +623,7 @@ public void testGroupByAggregatorDefaultValues()
),
ImmutableList.of(
NullHandling.replaceWithDefault()
- ? new Object[]{"a", 0.0, 0.0, 0.0, 0.0, 0L, 0L, 0L, 0L}
+ ? new Object[]{"a", 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0}
: new Object[]{"a", null, null, null, null, null, null, null, null}
)
);
@@ -688,9 +688,9 @@ public void assertResultsEquals(String sql, List expectedResults, List
Assert.assertEquals(expectedResult.length, result.length);
for (int j = 0; j < expectedResult.length; j++) {
if (expectedResult[j] instanceof Float) {
- Assert.assertEquals((Float) expectedResult[j], (Float) result[j], 1e-10);
+ Assert.assertEquals((Float) expectedResult[j], (Float) result[j], 1e-5);
} else if (expectedResult[j] instanceof Double) {
- Assert.assertEquals((Double) expectedResult[j], (Double) result[j], 1e-10);
+ Assert.assertEquals((Double) expectedResult[j], (Double) result[j], 1e-5);
} else {
Assert.assertEquals(expectedResult[j], result[j]);
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java
index cc760894603e..1726a3e68003 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java
@@ -222,18 +222,30 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception
toolbox.getTaskActionClient().submit(new SegmentNukeAction(new HashSet<>(unusedSegments)));
- // Fetch the load specs of all segments overlapping with the given interval
- final Set> usedSegmentLoadSpecs = toolbox
- .getTaskActionClient()
- .submit(new RetrieveUsedSegmentsAction(getDataSource(), getInterval(), null, Segments.INCLUDING_OVERSHADOWED))
- .stream()
- .map(DataSegment::getLoadSpec)
- .collect(Collectors.toSet());
+ final Set unusedSegmentIntervals = unusedSegments.stream()
+ .map(DataSegment::getInterval)
+ .collect(Collectors.toSet());
+ final Set> usedSegmentLoadSpecs = new HashSet<>();
+ if (!unusedSegmentIntervals.isEmpty()) {
+ RetrieveUsedSegmentsAction retrieveUsedSegmentsAction = new RetrieveUsedSegmentsAction(
+ getDataSource(),
+ null,
+ unusedSegmentIntervals,
+ Segments.INCLUDING_OVERSHADOWED
+ );
+ // Fetch the load specs of all segments overlapping with the unused segment intervals
+ usedSegmentLoadSpecs.addAll(toolbox.getTaskActionClient().submit(retrieveUsedSegmentsAction)
+ .stream()
+ .map(DataSegment::getLoadSpec)
+ .collect(Collectors.toSet())
+ );
+ }
// Kill segments from the deep storage only if their load specs are not being used by any used segments
final List segmentsToBeKilled = unusedSegments
.stream()
- .filter(unusedSegment -> !usedSegmentLoadSpecs.contains(unusedSegment.getLoadSpec()))
+ .filter(unusedSegment -> unusedSegment.getLoadSpec() == null
+ || !usedSegmentLoadSpecs.contains(unusedSegment.getLoadSpec()))
.collect(Collectors.toList());
toolbox.getDataSegmentKiller().kill(segmentsToBeKilled);
diff --git a/processing/src/main/java/org/apache/druid/collections/BlockingPool.java b/processing/src/main/java/org/apache/druid/collections/BlockingPool.java
index c17329917cd2..4fb3ff66d8bf 100644
--- a/processing/src/main/java/org/apache/druid/collections/BlockingPool.java
+++ b/processing/src/main/java/org/apache/druid/collections/BlockingPool.java
@@ -31,7 +31,6 @@ public interface BlockingPool
*
* @param elementNum number of resources to take
* @param timeoutMs maximum time to wait for resources, in milliseconds.
- *
* @return a list of resource holders. An empty list is returned if {@code elementNum} resources aren't available.
*/
List> takeBatch(int elementNum, long timeoutMs);
@@ -40,8 +39,14 @@ public interface BlockingPool
* Take resources from the pool, waiting if necessary until the elements of the given number become available.
*
* @param elementNum number of resources to take
- *
* @return a list of resource holders. An empty list is returned if {@code elementNum} resources aren't available.
*/
List> takeBatch(int elementNum);
+
+ /**
+ * Returns the count of the requests waiting to acquire a batch of resources.
+ *
+ * @return count of pending requests
+ */
+ long getPendingRequests();
}
diff --git a/processing/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java b/processing/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java
index 1021974b1b4e..e41a9e5d75d4 100644
--- a/processing/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java
+++ b/processing/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java
@@ -30,6 +30,7 @@
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
@@ -48,6 +49,8 @@ public class DefaultBlockingPool implements BlockingPool
private final Condition notEnough;
private final int maxSize;
+ private final AtomicLong pendingRequests;
+
public DefaultBlockingPool(
Supplier generator,
int limit
@@ -62,6 +65,7 @@ public DefaultBlockingPool(
this.lock = new ReentrantLock();
this.notEnough = lock.newCondition();
+ this.pendingRequests = new AtomicLong();
}
@Override
@@ -91,12 +95,16 @@ public List> takeBatch(final int elementNum,
Preconditions.checkArgument(timeoutMs >= 0, "timeoutMs must be a non-negative value, but was [%s]", timeoutMs);
checkInitialized();
try {
+ pendingRequests.incrementAndGet();
final List objects = timeoutMs > 0 ? pollObjects(elementNum, timeoutMs) : pollObjects(elementNum);
return objects.stream().map(this::wrapObject).collect(Collectors.toList());
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
+ finally {
+ pendingRequests.decrementAndGet();
+ }
}
@Override
@@ -104,11 +112,21 @@ public List> takeBatch(final int elementNum)
{
checkInitialized();
try {
+ pendingRequests.incrementAndGet();
return takeObjects(elementNum).stream().map(this::wrapObject).collect(Collectors.toList());
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
+ finally {
+ pendingRequests.incrementAndGet();
+ }
+ }
+
+ @Override
+ public long getPendingRequests()
+ {
+ return pendingRequests.get();
}
private List pollObjects(int elementNum) throws InterruptedException
diff --git a/processing/src/main/java/org/apache/druid/collections/DummyBlockingPool.java b/processing/src/main/java/org/apache/druid/collections/DummyBlockingPool.java
index dcd6cea07aa7..2553f9ab425f 100644
--- a/processing/src/main/java/org/apache/druid/collections/DummyBlockingPool.java
+++ b/processing/src/main/java/org/apache/druid/collections/DummyBlockingPool.java
@@ -55,4 +55,10 @@ public List> takeBatch(int elementNum)
{
throw new UnsupportedOperationException();
}
+
+ @Override
+ public long getPendingRequests()
+ {
+ return 0;
+ }
}
diff --git a/processing/src/main/java/org/apache/druid/frame/file/FrameFileHttpResponseHandler.java b/processing/src/main/java/org/apache/druid/frame/file/FrameFileHttpResponseHandler.java
index 0f70fb3d6983..661ba351dfc2 100644
--- a/processing/src/main/java/org/apache/druid/frame/file/FrameFileHttpResponseHandler.java
+++ b/processing/src/main/java/org/apache/druid/frame/file/FrameFileHttpResponseHandler.java
@@ -21,6 +21,7 @@
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.error.DruidException;
import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.http.client.response.ClientResponse;
@@ -49,11 +50,20 @@ public class FrameFileHttpResponseHandler implements HttpResponseHandler response(
return ClientResponse.finished(clientResponseObj);
}
- final byte[] chunk = new byte[content.readableBytes()];
- content.getBytes(content.readerIndex(), chunk);
+ final byte[] chunk;
+ final int chunkSize = content.readableBytes();
- try {
- final ListenableFuture> backpressureFuture = channel.addChunk(chunk);
+ // Potentially skip some of this chunk, if the relevant bytes have already been read by the handler. This can
+ // happen if a request reads some data, then fails with a retryable I/O error, and then is retried. The retry
+ // will re-read some data that has already been added to the channel, so we need to skip it.
+ final long readByThisHandler = channel.getBytesAdded() - startOffset;
+ final long readByThisRequest = clientResponseObj.getBytesRead(); // Prior to the current chunk
+ final long toSkip = readByThisHandler - readByThisRequest;
- if (backpressureFuture != null) {
- clientResponseObj.setBackpressureFuture(backpressureFuture);
- }
+ if (toSkip < 0) {
+ throw DruidException.defensive("Expected toSkip[%d] to be nonnegative", toSkip);
+ } else if (toSkip < chunkSize) { // When toSkip >= chunkSize, we skip the entire chunk and do not toucn the channel
+ chunk = new byte[chunkSize - (int) toSkip];
+ content.getBytes(content.readerIndex() + (int) toSkip, chunk);
- clientResponseObj.addBytesRead(chunk.length);
- }
- catch (Exception e) {
- clientResponseObj.exceptionCaught(e);
+ try {
+ final ListenableFuture> backpressureFuture = channel.addChunk(chunk);
+
+ if (backpressureFuture != null) {
+ clientResponseObj.setBackpressureFuture(backpressureFuture);
+ }
+ }
+ catch (Exception e) {
+ clientResponseObj.exceptionCaught(e);
+ }
}
+ // Call addBytesRead even if we skipped some or all of the chunk, because that lets us know when to stop skipping.
+ clientResponseObj.addBytesRead(chunkSize);
return ClientResponse.unfinished(clientResponseObj);
}
}
diff --git a/processing/src/main/java/org/apache/druid/frame/file/FrameFilePartialFetch.java b/processing/src/main/java/org/apache/druid/frame/file/FrameFilePartialFetch.java
index 8c2056dcbe43..9e6b84c6bbf7 100644
--- a/processing/src/main/java/org/apache/druid/frame/file/FrameFilePartialFetch.java
+++ b/processing/src/main/java/org/apache/druid/frame/file/FrameFilePartialFetch.java
@@ -74,6 +74,14 @@ public boolean isExceptionCaught()
return exceptionCaught != null;
}
+ /**
+ * Number of bytes read so far by this request.
+ */
+ public long getBytesRead()
+ {
+ return bytesRead;
+ }
+
/**
* Future that resolves when it is a good time to request the next chunk of the frame file.
*
@@ -105,6 +113,9 @@ void exceptionCaught(final Throwable t)
}
}
+ /**
+ * Increment the value returned by {@link #getBytesRead()}. Called whenever a chunk of data is read from the response.
+ */
void addBytesRead(final long n)
{
bytesRead += n;
diff --git a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java
index b4d06edc77cf..4bf1cb92a610 100644
--- a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java
+++ b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java
@@ -24,6 +24,7 @@
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.RowAdapter;
import org.apache.druid.segment.RowBasedCursor;
@@ -32,6 +33,7 @@
import org.apache.druid.segment.column.RowSignature;
import java.io.Closeable;
+import java.util.Iterator;
/**
* Helper methods to create cursor from iterable of rows
@@ -82,4 +84,35 @@ public static Pair getCursorFromSequence(Sequence r
return Pair.of(baseCursor, rowWalker);
}
+
+ public static Pair getCursorFromYielder(Yielder yielderParam, RowSignature rowSignature)
+ {
+ return getCursorFromIterable(
+ new Iterable()
+ {
+ Yielder yielder = yielderParam;
+ @Override
+ public Iterator iterator()
+ {
+ return new Iterator()
+ {
+ @Override
+ public boolean hasNext()
+ {
+ return !yielder.isDone();
+ }
+
+ @Override
+ public Object[] next()
+ {
+ Object[] retVal = yielder.get();
+ yielder = yielder.next(null);
+ return retVal;
+ }
+ };
+ }
+ },
+ rowSignature
+ );
+ }
}
diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java
index 3f538f5ad5aa..27a0113d76f1 100644
--- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java
+++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java
@@ -23,11 +23,12 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
+import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.segment.SegmentReference;
+import org.apache.druid.utils.CollectionUtils;
import java.util.Collections;
import java.util.List;
@@ -36,13 +37,24 @@
import java.util.function.Function;
import java.util.stream.Collectors;
+/**
+ * Reperesents a UNION ALL of two or more datasources.
+ *
+ * Native engine can only work with table datasources that are scans or simple mappings (column rename without any
+ * expression applied on top). Therefore, it uses methods like {@link #getTableNames()} and
+ * {@link #getDataSourcesAsTableDataSources()} to assert that the children were TableDataSources.
+ *
+ * MSQ should be able to plan and work with arbitrary datasources. It also needs to replace the datasource with the
+ * InputNumberDataSource while preparing the query plan.
+ */
public class UnionDataSource implements DataSource
{
- @JsonProperty
- private final List dataSources;
+
+ @JsonProperty("dataSources")
+ private final List dataSources;
@JsonCreator
- public UnionDataSource(@JsonProperty("dataSources") List dataSources)
+ public UnionDataSource(@JsonProperty("dataSources") List dataSources)
{
if (dataSources == null || dataSources.isEmpty()) {
throw new ISE("'dataSources' must be non-null and non-empty for 'union'");
@@ -51,18 +63,45 @@ public UnionDataSource(@JsonProperty("dataSources") List dataSo
this.dataSources = dataSources;
}
+ public List getDataSources()
+ {
+ return dataSources;
+ }
+
+
+ /**
+ * Asserts that the children of the union are all table data sources before returning the table names
+ */
@Override
public Set getTableNames()
{
- return dataSources.stream()
- .map(input -> Iterables.getOnlyElement(input.getTableNames()))
- .collect(Collectors.toSet());
+ return dataSources
+ .stream()
+ .map(input -> {
+ if (!(input instanceof TableDataSource)) {
+ throw DruidException.defensive("should be table");
+ }
+ return CollectionUtils.getOnlyElement(
+ input.getTableNames(),
+ xs -> DruidException.defensive("Expected only single table name in TableDataSource")
+ );
+ })
+ .collect(Collectors.toSet());
}
- @JsonProperty
- public List getDataSources()
+ /**
+ * Asserts that the children of the union are all table data sources
+ */
+ public List getDataSourcesAsTableDataSources()
{
- return dataSources;
+ return dataSources.stream()
+ .map(input -> {
+ if (!(input instanceof TableDataSource)) {
+ throw DruidException.defensive("should be table");
+ }
+ return (TableDataSource) input;
+ })
+ .collect(Collectors.toList());
}
@Override
@@ -78,13 +117,7 @@ public DataSource withChildren(List children)
throw new IAE("Expected [%d] children, got [%d]", dataSources.size(), children.size());
}
- if (!children.stream().allMatch(dataSource -> dataSource instanceof TableDataSource)) {
- throw new IAE("All children must be tables");
- }
-
- return new UnionDataSource(
- children.stream().map(dataSource -> (TableDataSource) dataSource).collect(Collectors.toList())
- );
+ return new UnionDataSource(children);
}
@Override
@@ -149,11 +182,7 @@ public boolean equals(Object o)
UnionDataSource that = (UnionDataSource) o;
- if (!dataSources.equals(that.dataSources)) {
- return false;
- }
-
- return true;
+ return dataSources.equals(that.dataSources);
}
@Override
diff --git a/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java b/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java
index aeb3897e644b..5459e1d8c22e 100644
--- a/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java
@@ -57,16 +57,16 @@ public Sequence run(final QueryPlus queryPlus, final ResponseContext respo
final UnionDataSource unionDataSource = analysis.getBaseUnionDataSource().get();
- if (unionDataSource.getDataSources().isEmpty()) {
+ if (unionDataSource.getDataSourcesAsTableDataSources().isEmpty()) {
// Shouldn't happen, because UnionDataSource doesn't allow empty unions.
throw new ISE("Unexpectedly received empty union");
- } else if (unionDataSource.getDataSources().size() == 1) {
+ } else if (unionDataSource.getDataSourcesAsTableDataSources().size() == 1) {
// Single table. Run as a normal query.
return baseRunner.run(
queryPlus.withQuery(
Queries.withBaseDataSource(
query,
- Iterables.getOnlyElement(unionDataSource.getDataSources())
+ Iterables.getOnlyElement(unionDataSource.getDataSourcesAsTableDataSources())
)
),
responseContext
@@ -77,8 +77,8 @@ public Sequence run(final QueryPlus queryPlus, final ResponseContext respo
query.getResultOrdering(),
Sequences.simple(
Lists.transform(
- IntStream.range(0, unionDataSource.getDataSources().size())
- .mapToObj(i -> new Pair<>(unionDataSource.getDataSources().get(i), i + 1))
+ IntStream.range(0, unionDataSource.getDataSourcesAsTableDataSources().size())
+ .mapToObj(i -> new Pair<>(unionDataSource.getDataSourcesAsTableDataSources().get(i), i + 1))
.collect(Collectors.toList()),
(Function, Sequence>) singleSourceWithIndex ->
baseRunner.run(
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java
index b79c4358a3de..b242ff98555a 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java
@@ -177,40 +177,8 @@ public BinaryOperator createMergeFn(Query queryParam)
return new GroupByBinaryFnV2((GroupByQuery) queryParam);
}
- /**
- * Runs a provided {@link QueryRunner} on a provided {@link GroupByQuery}, which is assumed to return rows that are
- * properly sorted (by timestamp and dimensions) but not necessarily fully merged (that is, there may be adjacent
- * rows with the same timestamp and dimensions) and without PostAggregators computed. This method will fully merge
- * the rows, apply PostAggregators, and return the resulting {@link Sequence}.
- *
- * The query will be modified before passing it down to the base runner. For example, "having" clauses will be
- * removed and various context parameters will be adjusted.
- *
- * Despite the similar name, this method is much reduced in scope compared to
- * {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}. That method does delegate to this one at some points,
- * but has a truckload of other responsibility, including computing outer query results (if there are subqueries),
- * computing subtotals (like GROUPING SETS), and computing the havingSpec and limitSpec.
- *
- * @param baseRunner base query runner
- * @param query the groupBy query to run inside the base query runner
- * @param responseContext the response context to pass to the base query runner
- *
- * @return merged result sequence
- */
- public Sequence mergeResults(
- final QueryRunner baseRunner,
- final GroupByQuery query,
- final ResponseContext responseContext
- )
+ public GroupByQuery prepareGroupByQuery(GroupByQuery query)
{
- // Merge streams using ResultMergeQueryRunner, then apply postaggregators, then apply limit (which may
- // involve materialization)
- final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner<>(
- baseRunner,
- this::createResultComparator,
- this::createMergeFn
- );
-
// Set up downstream context.
final ImmutableMap.Builder context = ImmutableMap.builder();
context.put(QueryContexts.FINALIZE_KEY, false);
@@ -224,7 +192,6 @@ public Sequence mergeResults(
final boolean hasTimestampResultField = (timestampResultField != null && !timestampResultField.isEmpty())
&& queryContext.getBoolean(CTX_KEY_OUTERMOST, true)
&& !query.isApplyLimitPushDown();
- int timestampResultFieldIndex = 0;
if (hasTimestampResultField) {
// sql like "group by city_id,time_floor(__time to day)",
// the original translated query is granularity=all and dimensions:[d0, d1]
@@ -257,7 +224,7 @@ public Sequence mergeResults(
granularity = timestampResultFieldGranularity;
// when timestampResultField is the last dimension, should set sortByDimsFirst=true,
// otherwise the downstream is sorted by row's timestamp first which makes the final ordering not as expected
- timestampResultFieldIndex = queryContext.getInt(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_INDEX);
+ int timestampResultFieldIndex = queryContext.getInt(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_INDEX, 0);
if (!query.getContextSortByDimsFirst() && timestampResultFieldIndex == query.getDimensions().size() - 1) {
context.put(GroupByQuery.CTX_KEY_SORT_BY_DIMS_FIRST, true);
}
@@ -269,7 +236,6 @@ public Sequence mergeResults(
// when hasTimestampResultField=true and timestampResultField is neither first nor last dimension,
// the DefaultLimitSpec will always do the reordering
}
- final int timestampResultFieldIndexInOriginalDimensions = timestampResultFieldIndex;
if (query.getUniversalTimestamp() != null && !hasTimestampResultField) {
// universalTimestamp works only when granularity is all
// hasTimestampResultField works only when granularity is all
@@ -283,7 +249,7 @@ public Sequence mergeResults(
// Always request array result rows when passing the query downstream.
context.put(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true);
- final GroupByQuery newQuery = new GroupByQuery(
+ return new GroupByQuery(
query.getDataSource(),
query.getQuerySegmentSpec(),
query.getVirtualColumns(),
@@ -305,6 +271,49 @@ public Sequence mergeResults(
).withOverriddenContext(
context.build()
);
+ }
+
+ /**
+ * Runs a provided {@link QueryRunner} on a provided {@link GroupByQuery}, which is assumed to return rows that are
+ * properly sorted (by timestamp and dimensions) but not necessarily fully merged (that is, there may be adjacent
+ * rows with the same timestamp and dimensions) and without PostAggregators computed. This method will fully merge
+ * the rows, apply PostAggregators, and return the resulting {@link Sequence}.
+ *
+ * The query will be modified using {@link #prepareGroupByQuery(GroupByQuery)} before passing it down to the base
+ * runner. For example, "having" clauses will be removed and various context parameters will be adjusted.
+ *
+ * Despite the similar name, this method is much reduced in scope compared to
+ * {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}. That method does delegate to this one at some points,
+ * but has a truckload of other responsibility, including computing outer query results (if there are subqueries),
+ * computing subtotals (like GROUPING SETS), and computing the havingSpec and limitSpec.
+ *
+ * @param baseRunner base query runner
+ * @param query the groupBy query to run inside the base query runner
+ * @param responseContext the response context to pass to the base query runner
+ *
+ * @return merged result sequence
+ */
+ public Sequence mergeResults(
+ final QueryRunner baseRunner,
+ final GroupByQuery query,
+ final ResponseContext responseContext
+ )
+ {
+ // Merge streams using ResultMergeQueryRunner, then apply postaggregators, then apply limit (which may
+ // involve materialization)
+ final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner<>(
+ baseRunner,
+ this::createResultComparator,
+ this::createMergeFn
+ );
+
+ final QueryContext queryContext = query.context();
+ final String timestampResultField = queryContext.getString(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD);
+ final boolean hasTimestampResultField = (timestampResultField != null && !timestampResultField.isEmpty())
+ && queryContext.getBoolean(CTX_KEY_OUTERMOST, true)
+ && !query.isApplyLimitPushDown();
+ final int timestampResultFieldIndexInOriginalDimensions = hasTimestampResultField ? queryContext.getInt(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_INDEX) : 0;
+ final GroupByQuery newQuery = prepareGroupByQuery(query);
final Sequence mergedResults = mergingQueryRunner.run(QueryPlus.wrap(newQuery), responseContext);
diff --git a/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java
index 94f13eb58c74..60c5891afafc 100644
--- a/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java
@@ -74,7 +74,7 @@ public Sequence run(final QueryPlus input, final ResponseContext responseC
final Thread currThread = setName ? Thread.currentThread() : null;
final String currThreadName = setName ? currThread.getName() : null;
- final String newName = setName ? query.getType() + "_" + query.getDataSource() + "_" + query.getIntervals() : null;
+ final String newName = setName ? "processing_" + query.getId() : null;
final Sequence baseSequence;
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 02f8c0064aa2..e9839a37818c 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java
@@ -322,19 +322,17 @@ to generate filters to be passed to base cursor (filtersPushedDownToBaseCursor)
// outside filter contains unnested column
// requires check for OR and And filters, disqualify rewrite for non-unnest filters
if (queryFilter instanceof BooleanFilter) {
- boolean isTopLevelAndFilter = queryFilter instanceof AndFilter;
List preFilterList = recursiveRewriteOnUnnestFilters(
(BooleanFilter) queryFilter,
inputColumn,
inputColumnCapabilites,
- filterSplitter,
- isTopLevelAndFilter
+ filterSplitter
);
// If rewite on entire query filter is successful then add entire filter to preFilter else skip and only add to post filter.
- if (filterSplitter.getPreFilterCount() == filterSplitter.getOriginalFilterCount()) {
+ if (!preFilterList.isEmpty()) {
if (queryFilter instanceof AndFilter) {
filterSplitter.addPreFilter(new AndFilter(preFilterList));
- } else if (queryFilter instanceof OrFilter) {
+ } else if (queryFilter instanceof OrFilter && filterSplitter.getPreFilterCount() == filterSplitter.getOriginalFilterCount()) {
filterSplitter.addPreFilter(new OrFilter(preFilterList));
}
}
@@ -470,8 +468,7 @@ private List recursiveRewriteOnUnnestFilters(
BooleanFilter queryFilter,
final String inputColumn,
final ColumnCapabilities inputColumnCapabilites,
- final FilterSplitter filterSplitter,
- final boolean isTopLevelAndFilter
+ final FilterSplitter filterSplitter
)
{
final List preFilterList = new ArrayList<>();
@@ -482,25 +479,26 @@ private List recursiveRewriteOnUnnestFilters(
(BooleanFilter) filter,
inputColumn,
inputColumnCapabilites,
- filterSplitter,
- isTopLevelAndFilter
+ filterSplitter
);
if (!andChildFilters.isEmpty()) {
preFilterList.add(new AndFilter(andChildFilters));
}
} else if (filter instanceof OrFilter) {
- // in case of Or Fiters, we set isTopLevelAndFilter to false that prevents pushing down any child filters to base
List orChildFilters = recursiveRewriteOnUnnestFilters(
(BooleanFilter) filter,
inputColumn,
inputColumnCapabilites,
- filterSplitter,
- false
+ filterSplitter
);
- preFilterList.add(new OrFilter(orChildFilters));
+ if (orChildFilters.size() == ((OrFilter) filter).getFilters().size()) {
+ preFilterList.add(new OrFilter(orChildFilters));
+ }
} else if (filter instanceof NotFilter) {
+ // nothing to do here...
continue;
} else {
+ // can we rewrite
final Filter newFilter = rewriteFilterOnUnnestColumnIfPossible(
filter,
inputColumn,
@@ -511,13 +509,6 @@ private List recursiveRewriteOnUnnestFilters(
preFilterList.add(newFilter);
filterSplitter.addToPreFilterCount(1);
}
- /*
- Push down the filters to base only if top level is And Filter
- we can not push down if top level filter is OR or unnestColumn is derived expression like arrays
- */
- if (isTopLevelAndFilter && getUnnestInputIfDirectAccess(unnestColumn) != null) {
- filterSplitter.addPreFilter(newFilter != null ? newFilter : filter);
- }
filterSplitter.addToOriginalFilterCount(1);
}
} else {
diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java
index 4eeaaddbe892..06c160e68409 100644
--- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java
@@ -346,6 +346,86 @@ public void testCaughtExceptionDuringChunkedResponse() throws Exception
);
}
+ @Test
+ public void testCaughtExceptionDuringChunkedResponseRetryWithSameHandler() throws Exception
+ {
+ // Split file into 12 chunks after the first 100 bytes.
+ final int firstPart = 100;
+ final int chunkSize = Ints.checkedCast(LongMath.divide(file.length() - firstPart, 12, RoundingMode.CEILING));
+ final byte[] allBytes = Files.readAllBytes(file.toPath());
+
+ // Add firstPart and be done.
+ ClientResponse response = handler.done(
+ handler.handleResponse(
+ makeResponse(HttpResponseStatus.OK, byteSlice(allBytes, 0, firstPart)),
+ null
+ )
+ );
+
+ Assert.assertEquals(firstPart, channel.getBytesAdded());
+ Assert.assertTrue(response.isFinished());
+
+ // Add first quarter after firstPart using a new handler.
+ handler = new FrameFileHttpResponseHandler(channel);
+ response = handler.handleResponse(
+ makeResponse(HttpResponseStatus.OK, byteSlice(allBytes, firstPart, chunkSize * 3)),
+ null
+ );
+
+ // Set an exception.
+ handler.exceptionCaught(response, new ISE("Oh no!"));
+
+ // Add another chunk after the exception is caught (this can happen in real life!). We expect it to be ignored.
+ response = handler.handleChunk(
+ response,
+ makeChunk(byteSlice(allBytes, firstPart + chunkSize * 3, chunkSize * 3)),
+ 2
+ );
+
+ // Verify that the exception handler was called.
+ Assert.assertTrue(response.getObj().isExceptionCaught());
+ final Throwable e = response.getObj().getExceptionCaught();
+ MatcherAssert.assertThat(e, CoreMatchers.instanceOf(IllegalStateException.class));
+ MatcherAssert.assertThat(e, ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("Oh no!")));
+
+ // Retry connection with the same handler and same initial offset firstPart (don't recreate handler), but now use
+ // thirds instead of quarters as chunks. (ServiceClientImpl would retry from the same offset with the same handler
+ // if the exception is retryable.)
+ response = handler.handleResponse(
+ makeResponse(HttpResponseStatus.OK, byteSlice(allBytes, firstPart, chunkSize * 4)),
+ null
+ );
+
+ Assert.assertEquals(firstPart + chunkSize * 4L, channel.getBytesAdded());
+ Assert.assertFalse(response.isFinished());
+
+ // Send the rest of the data.
+ response = handler.handleChunk(
+ response,
+ makeChunk(byteSlice(allBytes, firstPart + chunkSize * 4, chunkSize * 4)),
+ 1
+ );
+ Assert.assertEquals(firstPart + chunkSize * 8L, channel.getBytesAdded());
+
+ response = handler.handleChunk(
+ response,
+ makeChunk(byteSlice(allBytes, firstPart + chunkSize * 8, chunkSize * 4)),
+ 2
+ );
+ response = handler.done(response);
+
+ Assert.assertTrue(response.isFinished());
+ Assert.assertFalse(response.getObj().isExceptionCaught());
+
+ // Verify channel.
+ Assert.assertEquals(allBytes.length, channel.getBytesAdded());
+ channel.doneWriting();
+ FrameTestUtil.assertRowsEqual(
+ FrameTestUtil.readRowsFromAdapter(adapter, null, false),
+ FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(adapter.getRowSignature()))
+ );
+ }
+
private static HttpResponse makeResponse(final HttpResponseStatus status, final byte[] content)
{
final ByteBufferBackedChannelBuffer channelBuffer = new ByteBufferBackedChannelBuffer(ByteBuffer.wrap(content));
diff --git a/processing/src/test/java/org/apache/druid/query/DataSourceTest.java b/processing/src/test/java/org/apache/druid/query/DataSourceTest.java
index 7c7f50f281bb..e7850953a609 100644
--- a/processing/src/test/java/org/apache/druid/query/DataSourceTest.java
+++ b/processing/src/test/java/org/apache/druid/query/DataSourceTest.java
@@ -89,7 +89,7 @@ public void testUnionDataSource() throws Exception
Assert.assertTrue(dataSource instanceof UnionDataSource);
Assert.assertEquals(
Lists.newArrayList(new TableDataSource("ds1"), new TableDataSource("ds2")),
- Lists.newArrayList(((UnionDataSource) dataSource).getDataSources())
+ Lists.newArrayList(((UnionDataSource) dataSource).getDataSourcesAsTableDataSources())
);
Assert.assertEquals(
ImmutableSet.of("ds1", "ds2"),
diff --git a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java
index 45f14b80976c..7628c3289dd1 100644
--- a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java
+++ b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java
@@ -21,6 +21,7 @@
import com.google.common.collect.ImmutableList;
import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.guava.Yielders;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.column.ColumnType;
@@ -60,6 +61,13 @@ public void getCursorFromSequence()
testCursorMatchesRowSequence(cursor, rowSignature, rows);
}
+ @Test
+ public void getCursorFromYielder()
+ {
+ Cursor cursor = IterableRowsCursorHelper.getCursorFromYielder(Yielders.each(Sequences.simple(rows)), rowSignature).lhs;
+ testCursorMatchesRowSequence(cursor, rowSignature, rows);
+ }
+
private void testCursorMatchesRowSequence(
Cursor cursor,
RowSignature expectedRowSignature,
diff --git a/processing/src/test/java/org/apache/druid/query/QueryContextTest.java b/processing/src/test/java/org/apache/druid/query/QueryContextTest.java
index 54acab0a3f87..71b477d16c37 100644
--- a/processing/src/test/java/org/apache/druid/query/QueryContextTest.java
+++ b/processing/src/test/java/org/apache/druid/query/QueryContextTest.java
@@ -339,7 +339,7 @@ public void testGetMaxSubqueryBytes()
assertEquals("auto", context2.getMaxSubqueryMemoryBytes(null));
final QueryContext context3 = new QueryContext(ImmutableMap.of());
- assertEquals("unlimited", context3.getMaxSubqueryMemoryBytes("unlimited"));
+ assertEquals("disabled", context3.getMaxSubqueryMemoryBytes("disabled"));
}
@Test
diff --git a/processing/src/test/java/org/apache/druid/query/TestBufferPool.java b/processing/src/test/java/org/apache/druid/query/TestBufferPool.java
index 10690d31be13..a650437f83f0 100644
--- a/processing/src/test/java/org/apache/druid/query/TestBufferPool.java
+++ b/processing/src/test/java/org/apache/druid/query/TestBufferPool.java
@@ -132,6 +132,12 @@ public List> takeBatch(int elementNu
}
}
+ @Override
+ public long getPendingRequests()
+ {
+ return 0;
+ }
+
public long getOutstandingObjectCount()
{
return takenFromMap.size();
diff --git a/processing/src/test/java/org/apache/druid/query/UnionDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/UnionDataSourceTest.java
index f408e71abf23..12522df08df3 100644
--- a/processing/src/test/java/org/apache/druid/query/UnionDataSourceTest.java
+++ b/processing/src/test/java/org/apache/druid/query/UnionDataSourceTest.java
@@ -123,7 +123,7 @@ public void test_withChildren_empty()
@Test
public void test_withChildren_sameNumber()
{
- final List newDataSources = ImmutableList.of(
+ final List