mirror of https://github.com/apache/druid.git
Various changes and fixes to UNNEST. (#13892)
* Various changes and fixes to UNNEST. Native changes: 1) UnnestDataSource: Replace "column" and "outputName" with "virtualColumn". This enables pushing expressions into the datasource. This in turn allows us to do the next thing... 2) UnnestStorageAdapter: Logically apply query-level filters and virtual columns after the unnest operation. (Physically, filters are pulled up, when possible.) This is beneficial because it allows filters and virtual columns to reference the unnested column, and because it is consistent with how the join datasource works. 3) Various documentation updates, including declaring "unnest" as an experimental feature for now. SQL changes: 1) Rename DruidUnnestRel (& Rule) to DruidUnnestRel (& Rule). The rel is simplified: it only handles the UNNEST part of a correlated join. Constant UNNESTs are handled with regular inline rels. 2) Rework DruidCorrelateUnnestRule to focus on pulling Projects from the left side up above the Correlate. New test testUnnestTwice verifies that this works even when two UNNESTs are stacked on the same table. 3) Include ProjectCorrelateTransposeRule from Calcite to encourage pushing mappings down below the left-hand side of the Correlate. 4) Add a new CorrelateFilterLTransposeRule and CorrelateFilterRTransposeRule to handle pulling Filters up above the Correlate. New tests testUnnestWithFiltersOutside and testUnnestTwiceWithFilters verify this behavior. 5) Require a context feature flag for SQL UNNEST, since it's undocumented. As part of this, also cleaned up how we handle feature flags in SQL. They're now hooked into EngineFeatures, which is useful because not all engines support all features.
This commit is contained in:
parent
64b67c22c4
commit
4b1ffbc452
|
@ -54,7 +54,7 @@ For logical operators, a number is true if and only if it is positive (0 or nega
|
|||
|
||||
[Multi-value string dimensions](../querying/multi-value-dimensions.md) are supported and may be treated as either scalar or array typed values, as follows:
|
||||
* When treated as a scalar type, the expression is automatically transformed so that the scalar operation is applied across all values of the multi-valued type, mimicking Druid's native behavior.
|
||||
* Druid coerces values that result in arrays back into the native Druid string type for grouping and aggregation. Grouping on multi-value string dimensions in Druid groups by the individual values, not the 'array'. This behavior produces results similar to the `UNNEST` operator available in many SQL dialects. Alternatively, you can use the `array_to_string` function to perform the aggregation on a _stringified_ version of the complete array and therefore preserve the complete row. To transform the stringified dimension back into the true native array type, use `string_to_array` in an expression post-aggregator.
|
||||
* Druid coerces values that result in arrays back into the native Druid string type for grouping and aggregation. Grouping on multi-value string dimensions in Druid groups by the individual values, not the 'array'. This behavior produces results similar to an implicit SQL `UNNEST` operation. Alternatively, you can use the `array_to_string` function to perform the aggregation on a _stringified_ version of the complete array and therefore preserve the complete row. To transform the stringified dimension back into the true native array type, use `string_to_array` in an expression post-aggregator.
|
||||
|
||||
|
||||
The following built-in functions are available.
|
||||
|
|
|
@ -371,9 +371,10 @@ future versions:
|
|||
|
||||
### `unnest`
|
||||
|
||||
> The unnest datasource is currently only available as part of a native query.
|
||||
> The unnest datasource is [experimental](../development/experimental.md). Its API and behavior are subject
|
||||
> to change in future releases. It is not recommended to use this feature in production at this time.
|
||||
|
||||
Use the `unnest` datasource to unnest a column with multiple values in an array.
|
||||
Use the `unnest` datasource to unnest a column with multiple values in an array.
|
||||
For example, you have a source column that looks like this:
|
||||
|
||||
| Nested |
|
||||
|
@ -407,7 +408,10 @@ The `unnest` datasource uses the following syntax:
|
|||
"type": "table",
|
||||
"name": "nested_data"
|
||||
},
|
||||
"column": "nested_source_column",
|
||||
"virtualColumn": {
|
||||
"type": "expression",
|
||||
"expression": "\"column_reference\""
|
||||
},
|
||||
"outputName": "unnested_target_column",
|
||||
"allowList": []
|
||||
},
|
||||
|
@ -416,9 +420,7 @@ The `unnest` datasource uses the following syntax:
|
|||
* `dataSource.type`: Set this to `unnest`.
|
||||
* `dataSource.base`: Defines the datasource you want to unnest.
|
||||
* `dataSource.base.type`: The type of datasource you want to unnest, such as a table.
|
||||
* `dataSource.base.name`: The name of the datasource you want to unnest.
|
||||
* `dataSource.column`: The name of the source column that contains the nested values.
|
||||
* `dataSource.outputName`: The name you want to assign to the column that will contain the unnested values. You can replace the source column with the unnested column by specifying the source column's name or a new column by specifying a different name. Outputting it to a new column can help you verify that you get the results that you expect but isn't required.
|
||||
* `dataSource.virtualColumn`: [Virtual column](virtual-columns.md) that references the nested values. The output name of this column is reused as the name of the column that contains unnested values. You can replace the source column with the unnested column by specifying the source column's name or a new column by specifying a different name. Outputting it to a new column can help you verify that you get the results that you expect but isn't required.
|
||||
* `dataSource.allowList`: Optional. The subset of values you want to unnest.
|
||||
|
||||
To learn more about how to use the `unnest` datasource, see the [unnest tutorial](../tutorials/tutorial-unnest-datasource.md).
|
||||
|
|
|
@ -140,12 +140,11 @@ This "selector" filter would match row4 of the dataset above:
|
|||
### Grouping
|
||||
|
||||
topN and groupBy queries can group on multi-value dimensions. When grouping on a multi-value dimension, _all_ values
|
||||
from matching rows will be used to generate one group per value. This can be thought of as the equivalent to the
|
||||
`UNNEST` operator used on an `ARRAY` type that many SQL dialects support. This means it's possible for a query to return
|
||||
more groups than there are rows. For example, a topN on the dimension `tags` with filter `"t1" AND "t3"` would match
|
||||
only row1, and generate a result with three groups: `t1`, `t2`, and `t3`. If you only need to include values that match
|
||||
your filter, you can use a [filtered dimensionSpec](dimensionspecs.md#filtered-dimensionspecs). This can also
|
||||
improve performance.
|
||||
from matching rows will be used to generate one group per value. This behaves similarly to an implicit SQL `UNNEST`
|
||||
operation. This means it's possible for a query to return more groups than there are rows. For example, a topN on the
|
||||
dimension `tags` with filter `"t1" AND "t3"` would match only row1, and generate a result with three groups:
|
||||
`t1`, `t2`, and `t3`. If you only need to include values that match your filter, you can use a
|
||||
[filtered dimensionSpec](dimensionspecs.md#filtered-dimensionspecs). This can also improve performance.
|
||||
|
||||
## Example: GroupBy query with no filtering
|
||||
|
||||
|
|
|
@ -78,8 +78,8 @@ applied to all values for each row individually. Multi-value string dimensions c
|
|||
[multi-value string functions](sql-multivalue-string-functions.md), which can perform powerful array-aware operations.
|
||||
|
||||
Grouping by a multi-value expression observes the native Druid multi-value aggregation behavior, which is similar to
|
||||
the `UNNEST` functionality available in some other SQL dialects. Refer to the documentation on
|
||||
[multi-value string dimensions](multi-value-dimensions.md) for additional details.
|
||||
an implicit SQL `UNNEST`. Refer to the documentation on [multi-value string dimensions](multi-value-dimensions.md)
|
||||
for additional details.
|
||||
|
||||
> Because multi-value dimensions are treated by the SQL planner as `VARCHAR`, there are some inconsistencies between how
|
||||
> they are handled in Druid SQL and in native queries. For example, expressions involving multi-value dimensions may be
|
||||
|
|
|
@ -25,7 +25,8 @@ title: "Tutorial: Unnest data in a column"
|
|||
|
||||
> If you're looking for information about how to unnest `COMPLEX<json>` columns, see [Nested columns](../querying/nested-columns.md).
|
||||
|
||||
> The unnest datasource is currently only available as part of a native query.
|
||||
> The unnest datasource is [experimental](../development/experimental.md). Its API and behavior are subject
|
||||
> to change in future releases. It is not recommended to use this feature in production at this time.
|
||||
|
||||
This tutorial demonstrates how to use the unnest datasource to unnest a column that has data stored in arrays. For example, if you have a column named `dim3` with values like `[a,b]` or `[c,d,f]`, the unnest datasource can output the data to a new column with individual rows that contain single values like `a` and `b`. When doing this, be mindful of the following:
|
||||
|
||||
|
@ -161,9 +162,11 @@ The following native Scan query returns the rows of the datasource and unnests t
|
|||
"type": "table",
|
||||
"name": "nested_data"
|
||||
},
|
||||
"column": "dim3",
|
||||
"outputName": "unnest-dim3",
|
||||
"allowList": []
|
||||
"virtualColumn": {
|
||||
"type": "expression",
|
||||
"name": "unnest-dim3",
|
||||
"expression": "\"dim3\""
|
||||
}
|
||||
},
|
||||
"intervals": {
|
||||
"type": "intervals",
|
||||
|
@ -222,9 +225,11 @@ The following query returns an unnested version of the column `dim3` as the colu
|
|||
"dataSource": {
|
||||
"type": "unnest",
|
||||
"base": "nested_data",
|
||||
"column": "dim3",
|
||||
"outputName": "unnest-dim3",
|
||||
"allowList": []
|
||||
"virtualColumn": {
|
||||
"type": "expression",
|
||||
"name": "unnest-dim3",
|
||||
"expression": "\"dim3\""
|
||||
}
|
||||
},
|
||||
"intervals": ["-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"],
|
||||
"granularity": "all",
|
||||
|
@ -264,8 +269,11 @@ The example topN query unnests `dim3` into the column `unnest-dim3`. The query u
|
|||
"type": "table",
|
||||
"name": "nested_data"
|
||||
},
|
||||
"column": "dim3",
|
||||
"outputName": "unnest-dim3",
|
||||
"virtualColumn": {
|
||||
"type": "expression",
|
||||
"name": "unnest-dim3",
|
||||
"expression": "\"dim3\""
|
||||
},
|
||||
"allowList": null
|
||||
},
|
||||
"dimension": {
|
||||
|
@ -369,9 +377,11 @@ This query joins the `nested_data` table with itself and outputs the unnested da
|
|||
"condition": "(\"m1\" == \"j0.v0\")",
|
||||
"joinType": "INNER"
|
||||
},
|
||||
"column": "dim3",
|
||||
"outputName": "unnest-dim3",
|
||||
"allowList": []
|
||||
"virtualColumn": {
|
||||
"type": "expression",
|
||||
"name": "unnest-dim3",
|
||||
"expression": "\"dim3\""
|
||||
}
|
||||
},
|
||||
"intervals": {
|
||||
"type": "intervals",
|
||||
|
@ -520,13 +530,15 @@ When you run the query, pay special attention to how the total number of rows ha
|
|||
"type": "table",
|
||||
"name": "nested_data2"
|
||||
},
|
||||
"column": "dim3",
|
||||
"outputName": "unnest-dim3",
|
||||
"virtualColumn": {
|
||||
"type": "expression",
|
||||
"name": "unnest-dim3",
|
||||
"expression": "\"dim3\""
|
||||
},
|
||||
"allowList": []
|
||||
},
|
||||
"column": "dim2",
|
||||
"outputName": "unnest-dim2",
|
||||
"allowList": []
|
||||
"outputName": "unnest-dim2"
|
||||
},
|
||||
"intervals": {
|
||||
"type": "intervals",
|
||||
|
@ -583,9 +595,11 @@ You can also use the `unnest` datasource to unnest an inline datasource. The fol
|
|||
]
|
||||
]
|
||||
},
|
||||
"column": "inline_data",
|
||||
"outputName": "output",
|
||||
"allowList": []
|
||||
"virtualColumn": {
|
||||
"type": "expression",
|
||||
"name": "output",
|
||||
"expression": "\"inline_data\""
|
||||
}
|
||||
},
|
||||
"intervals": {
|
||||
"type": "intervals",
|
||||
|
@ -625,8 +639,11 @@ The following Scan query uses the `nested_data2` table you created in [Load data
|
|||
"type": "table",
|
||||
"name": "nested_data2"
|
||||
},
|
||||
"column": "v0",
|
||||
"outputName": "unnest-v0"
|
||||
"virtualColumn": {
|
||||
"type": "expression",
|
||||
"name": "unnest-v0",
|
||||
"expression": "\"v0\""
|
||||
}
|
||||
}
|
||||
"intervals": {
|
||||
"type": "intervals",
|
||||
|
|
|
@ -104,7 +104,7 @@ public class MSQTaskSqlEngine implements SqlEngine
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean feature(EngineFeature feature, PlannerContext plannerContext)
|
||||
public boolean featureAvailable(EngineFeature feature, PlannerContext plannerContext)
|
||||
{
|
||||
switch (feature) {
|
||||
case ALLOW_BINDABLE_PLAN:
|
||||
|
@ -112,6 +112,9 @@ public class MSQTaskSqlEngine implements SqlEngine
|
|||
case TIMESERIES_QUERY:
|
||||
case TOPN_QUERY:
|
||||
case TIME_BOUNDARY_QUERY:
|
||||
case GROUPING_SETS:
|
||||
case WINDOW_FUNCTIONS:
|
||||
case UNNEST:
|
||||
return false;
|
||||
case CAN_SELECT:
|
||||
case CAN_INSERT:
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.IAE;
|
|||
import org.apache.druid.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.UnnestSegmentReference;
|
||||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.utils.JvmUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -48,32 +49,28 @@ import java.util.function.Function;
|
|||
public class UnnestDataSource implements DataSource
|
||||
{
|
||||
private final DataSource base;
|
||||
private final String column;
|
||||
private final String outputName;
|
||||
private final VirtualColumn virtualColumn;
|
||||
private final LinkedHashSet<String> allowList;
|
||||
|
||||
private UnnestDataSource(
|
||||
DataSource dataSource,
|
||||
String columnName,
|
||||
String outputName,
|
||||
VirtualColumn virtualColumn,
|
||||
LinkedHashSet<String> allowList
|
||||
)
|
||||
{
|
||||
this.base = dataSource;
|
||||
this.column = columnName;
|
||||
this.outputName = outputName;
|
||||
this.virtualColumn = virtualColumn;
|
||||
this.allowList = allowList;
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public static UnnestDataSource create(
|
||||
@JsonProperty("base") DataSource base,
|
||||
@JsonProperty("column") String columnName,
|
||||
@JsonProperty("outputName") String outputName,
|
||||
@JsonProperty("virtualColumn") VirtualColumn virtualColumn,
|
||||
@Nullable @JsonProperty("allowList") LinkedHashSet<String> allowList
|
||||
)
|
||||
{
|
||||
return new UnnestDataSource(base, columnName, outputName, allowList);
|
||||
return new UnnestDataSource(base, virtualColumn, allowList);
|
||||
}
|
||||
|
||||
@JsonProperty("base")
|
||||
|
@ -82,16 +79,10 @@ public class UnnestDataSource implements DataSource
|
|||
return base;
|
||||
}
|
||||
|
||||
@JsonProperty("column")
|
||||
public String getColumn()
|
||||
@JsonProperty("virtualColumn")
|
||||
public VirtualColumn getVirtualColumn()
|
||||
{
|
||||
return column;
|
||||
}
|
||||
|
||||
@JsonProperty("outputName")
|
||||
public String getOutputName()
|
||||
{
|
||||
return outputName;
|
||||
return virtualColumn;
|
||||
}
|
||||
|
||||
@JsonProperty("allowList")
|
||||
|
@ -118,7 +109,7 @@ public class UnnestDataSource implements DataSource
|
|||
if (children.size() != 1) {
|
||||
throw new IAE("Expected [1] child, got [%d]", children.size());
|
||||
}
|
||||
return new UnnestDataSource(children.get(0), column, outputName, allowList);
|
||||
return new UnnestDataSource(children.get(0), virtualColumn, allowList);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -151,22 +142,13 @@ public class UnnestDataSource implements DataSource
|
|||
);
|
||||
return JvmUtils.safeAccumulateThreadCpuTime(
|
||||
cpuTimeAccumulator,
|
||||
() -> {
|
||||
if (column == null) {
|
||||
return segmentMapFn;
|
||||
} else if (column.isEmpty()) {
|
||||
return segmentMapFn;
|
||||
} else {
|
||||
return
|
||||
baseSegment ->
|
||||
new UnnestSegmentReference(
|
||||
segmentMapFn.apply(baseSegment),
|
||||
column,
|
||||
outputName,
|
||||
allowList
|
||||
);
|
||||
}
|
||||
}
|
||||
() ->
|
||||
baseSegment ->
|
||||
new UnnestSegmentReference(
|
||||
segmentMapFn.apply(baseSegment),
|
||||
virtualColumn,
|
||||
allowList
|
||||
)
|
||||
);
|
||||
|
||||
}
|
||||
|
@ -174,7 +156,7 @@ public class UnnestDataSource implements DataSource
|
|||
@Override
|
||||
public DataSource withUpdatedDataSource(DataSource newSource)
|
||||
{
|
||||
return new UnnestDataSource(newSource, column, outputName, allowList);
|
||||
return new UnnestDataSource(newSource, virtualColumn, allowList);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -205,15 +187,14 @@ public class UnnestDataSource implements DataSource
|
|||
return false;
|
||||
}
|
||||
UnnestDataSource that = (UnnestDataSource) o;
|
||||
return column.equals(that.column)
|
||||
&& outputName.equals(that.outputName)
|
||||
return virtualColumn.equals(that.virtualColumn)
|
||||
&& base.equals(that.base);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(base, column, outputName);
|
||||
return Objects.hash(base, virtualColumn);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -221,8 +202,7 @@ public class UnnestDataSource implements DataSource
|
|||
{
|
||||
return "UnnestDataSource{" +
|
||||
"base=" + base +
|
||||
", column='" + column + '\'' +
|
||||
", outputName='" + outputName + '\'' +
|
||||
", column='" + virtualColumn + '\'' +
|
||||
", allowList=" + allowList +
|
||||
'}';
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.druid.query.QueryRunner;
|
|||
import org.apache.druid.query.QueryToolChest;
|
||||
import org.apache.druid.query.aggregation.MetricManipulationFn;
|
||||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
|
@ -172,7 +173,8 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
|
|||
|
||||
final VirtualColumn virtualColumn = query.getVirtualColumns().getVirtualColumn(columnName);
|
||||
if (virtualColumn != null) {
|
||||
columnType = virtualColumn.capabilities(columnName).toColumnType();
|
||||
final ColumnCapabilities capabilities = virtualColumn.capabilities(c -> null, columnName);
|
||||
columnType = capabilities != null ? capabilities.toColumnType() : null;
|
||||
} else {
|
||||
// Unknown type. In the future, it would be nice to have a way to fill these in.
|
||||
columnType = null;
|
||||
|
|
|
@ -28,8 +28,8 @@ import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
|||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -63,7 +63,7 @@ public class UnnestColumnValueSelectorCursor implements Cursor
|
|||
private final Cursor baseCursor;
|
||||
private final ColumnSelectorFactory baseColumnSelectorFactory;
|
||||
private final ColumnValueSelector columnValueSelector;
|
||||
private final String columnName;
|
||||
private final VirtualColumn unnestColumn;
|
||||
private final String outputName;
|
||||
private final LinkedHashSet<String> allowSet;
|
||||
private int index;
|
||||
|
@ -73,16 +73,19 @@ public class UnnestColumnValueSelectorCursor implements Cursor
|
|||
|
||||
public UnnestColumnValueSelectorCursor(
|
||||
Cursor cursor,
|
||||
ColumnSelectorFactory baseColumSelectorFactory,
|
||||
String columnName,
|
||||
ColumnSelectorFactory baseColumnSelectorFactory,
|
||||
VirtualColumn unnestColumn,
|
||||
String outputColumnName,
|
||||
LinkedHashSet<String> allowSet
|
||||
)
|
||||
{
|
||||
this.baseCursor = cursor;
|
||||
this.baseColumnSelectorFactory = baseColumSelectorFactory;
|
||||
this.columnValueSelector = this.baseColumnSelectorFactory.makeColumnValueSelector(columnName);
|
||||
this.columnName = columnName;
|
||||
this.baseColumnSelectorFactory = baseColumnSelectorFactory;
|
||||
this.columnValueSelector = unnestColumn.makeColumnValueSelector(
|
||||
unnestColumn.getOutputName(),
|
||||
this.baseColumnSelectorFactory
|
||||
);
|
||||
this.unnestColumn = unnestColumn;
|
||||
this.index = 0;
|
||||
this.outputName = outputColumnName;
|
||||
this.needInitialization = true;
|
||||
|
@ -215,14 +218,21 @@ public class UnnestColumnValueSelectorCursor implements Cursor
|
|||
if (!outputName.equals(column)) {
|
||||
return baseColumnSelectorFactory.getColumnCapabilities(column);
|
||||
}
|
||||
final ColumnCapabilities capabilities = baseColumnSelectorFactory.getColumnCapabilities(columnName);
|
||||
if (capabilities.isArray()) {
|
||||
|
||||
final ColumnCapabilities capabilities = unnestColumn.capabilities(
|
||||
baseColumnSelectorFactory,
|
||||
unnestColumn.getOutputName()
|
||||
);
|
||||
|
||||
if (capabilities == null) {
|
||||
return null;
|
||||
} else if (capabilities.isArray()) {
|
||||
return ColumnCapabilitiesImpl.copyOf(capabilities).setType(capabilities.getElementType());
|
||||
}
|
||||
if (capabilities.hasMultipleValues().isTrue()) {
|
||||
} else if (capabilities.hasMultipleValues().isTrue()) {
|
||||
return ColumnCapabilitiesImpl.copyOf(capabilities).setHasMultipleValues(false);
|
||||
} else {
|
||||
return capabilities;
|
||||
}
|
||||
return baseColumnSelectorFactory.getColumnCapabilities(columnName);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -276,28 +286,18 @@ public class UnnestColumnValueSelectorCursor implements Cursor
|
|||
|
||||
/**
|
||||
* This method populates the objects when the base cursor moves to the next row
|
||||
*
|
||||
* @param firstRun flag to populate one time object references to hold values for unnest cursor
|
||||
*/
|
||||
private void getNextRow(boolean firstRun)
|
||||
private void getNextRow()
|
||||
{
|
||||
currentVal = this.columnValueSelector.getObject();
|
||||
if (currentVal == null) {
|
||||
if (!firstRun) {
|
||||
unnestListForCurrentRow = new ArrayList<>();
|
||||
}
|
||||
unnestListForCurrentRow.add(null);
|
||||
unnestListForCurrentRow = Collections.singletonList(null);
|
||||
} else if (currentVal instanceof List) {
|
||||
unnestListForCurrentRow = (List<Object>) currentVal;
|
||||
} else if (currentVal instanceof Object[]) {
|
||||
unnestListForCurrentRow = Arrays.asList((Object[]) currentVal);
|
||||
} else {
|
||||
if (currentVal instanceof List) {
|
||||
unnestListForCurrentRow = (List<Object>) currentVal;
|
||||
} else if (currentVal instanceof Object[]) {
|
||||
unnestListForCurrentRow = Arrays.asList((Object[]) currentVal);
|
||||
} else if (currentVal.getClass().equals(String.class)) {
|
||||
if (!firstRun) {
|
||||
unnestListForCurrentRow = new ArrayList<>();
|
||||
}
|
||||
unnestListForCurrentRow.add(currentVal);
|
||||
}
|
||||
unnestListForCurrentRow = Collections.singletonList(currentVal);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -309,8 +309,7 @@ public class UnnestColumnValueSelectorCursor implements Cursor
|
|||
*/
|
||||
private void initialize()
|
||||
{
|
||||
this.unnestListForCurrentRow = new ArrayList<>();
|
||||
getNextRow(needInitialization);
|
||||
getNextRow();
|
||||
if (allowSet != null) {
|
||||
if (!allowSet.isEmpty()) {
|
||||
if (!allowSet.contains((String) unnestListForCurrentRow.get(index))) {
|
||||
|
@ -333,7 +332,7 @@ public class UnnestColumnValueSelectorCursor implements Cursor
|
|||
index = 0;
|
||||
baseCursor.advance();
|
||||
if (!baseCursor.isDone()) {
|
||||
getNextRow(needInitialization);
|
||||
getNextRow();
|
||||
}
|
||||
} else {
|
||||
index++;
|
||||
|
|
|
@ -77,28 +77,32 @@ public class UnnestDimensionCursor implements Cursor
|
|||
{
|
||||
private final Cursor baseCursor;
|
||||
private final DimensionSelector dimSelector;
|
||||
private final String columnName;
|
||||
private final VirtualColumn unnestColumn;
|
||||
private final String outputName;
|
||||
private final LinkedHashSet<String> allowSet;
|
||||
private final BitSet allowedBitSet;
|
||||
private final ColumnSelectorFactory baseColumnSelectorFactory;
|
||||
private int index;
|
||||
@Nullable private IndexedInts indexedIntsForCurrentRow;
|
||||
@Nullable
|
||||
private IndexedInts indexedIntsForCurrentRow;
|
||||
private boolean needInitialization;
|
||||
private SingleIndexInts indexIntsForRow;
|
||||
|
||||
public UnnestDimensionCursor(
|
||||
Cursor cursor,
|
||||
ColumnSelectorFactory baseColumnSelectorFactory,
|
||||
String columnName,
|
||||
VirtualColumn unnestColumn,
|
||||
String outputColumnName,
|
||||
LinkedHashSet<String> allowSet
|
||||
)
|
||||
{
|
||||
this.baseCursor = cursor;
|
||||
this.baseColumnSelectorFactory = baseColumnSelectorFactory;
|
||||
this.dimSelector = this.baseColumnSelectorFactory.makeDimensionSelector(DefaultDimensionSpec.of(columnName));
|
||||
this.columnName = columnName;
|
||||
this.dimSelector = unnestColumn.makeDimensionSelector(
|
||||
DefaultDimensionSpec.of(unnestColumn.getOutputName()),
|
||||
this.baseColumnSelectorFactory
|
||||
);
|
||||
this.unnestColumn = unnestColumn;
|
||||
this.index = 0;
|
||||
this.outputName = outputColumnName;
|
||||
this.needInitialization = true;
|
||||
|
@ -254,14 +258,18 @@ public class UnnestDimensionCursor implements Cursor
|
|||
// This is fine for STRING types
|
||||
// But going forward if the dimension to be unnested is of type ARRAY,
|
||||
// this should strip down to the base type of the array
|
||||
final ColumnCapabilities capabilities = baseColumnSelectorFactory.getColumnCapabilities(columnName);
|
||||
final ColumnCapabilities capabilities = unnestColumn.capabilities(
|
||||
baseColumnSelectorFactory,
|
||||
unnestColumn.getOutputName()
|
||||
);
|
||||
|
||||
if (capabilities.isArray()) {
|
||||
return ColumnCapabilitiesImpl.copyOf(capabilities).setType(capabilities.getElementType());
|
||||
}
|
||||
if (capabilities.hasMultipleValues().isTrue()) {
|
||||
return ColumnCapabilitiesImpl.copyOf(capabilities).setHasMultipleValues(false);
|
||||
}
|
||||
return baseColumnSelectorFactory.getColumnCapabilities(columnName);
|
||||
return capabilities;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -40,15 +40,17 @@ public class UnnestSegmentReference implements SegmentReference
|
|||
private static final Logger log = new Logger(UnnestSegmentReference.class);
|
||||
|
||||
private final SegmentReference baseSegment;
|
||||
private final String dimension;
|
||||
private final String renamedOutputDimension;
|
||||
private final VirtualColumn unnestColumn;
|
||||
private final LinkedHashSet<String> allowSet;
|
||||
|
||||
public UnnestSegmentReference(SegmentReference baseSegment, String dimension, String outputName, LinkedHashSet<String> allowList)
|
||||
public UnnestSegmentReference(
|
||||
SegmentReference baseSegment,
|
||||
VirtualColumn unnestColumn,
|
||||
LinkedHashSet<String> allowList
|
||||
)
|
||||
{
|
||||
this.baseSegment = baseSegment;
|
||||
this.dimension = dimension;
|
||||
this.renamedOutputDimension = outputName;
|
||||
this.unnestColumn = unnestColumn;
|
||||
this.allowSet = allowList;
|
||||
}
|
||||
|
||||
|
@ -101,8 +103,7 @@ public class UnnestSegmentReference implements SegmentReference
|
|||
{
|
||||
return new UnnestStorageAdapter(
|
||||
baseSegment.asStorageAdapter(),
|
||||
dimension,
|
||||
renamedOutputDimension,
|
||||
unnestColumn,
|
||||
allowSet
|
||||
);
|
||||
}
|
||||
|
|
|
@ -19,24 +19,38 @@
|
|||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.query.QueryMetrics;
|
||||
import org.apache.druid.query.filter.BooleanFilter;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.InDimFilter;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.data.ListIndexed;
|
||||
import org.apache.druid.segment.filter.AndFilter;
|
||||
import org.apache.druid.segment.filter.BoundFilter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.filter.LikeFilter;
|
||||
import org.apache.druid.segment.filter.NotFilter;
|
||||
import org.apache.druid.segment.filter.SelectorFilter;
|
||||
import org.apache.druid.segment.join.PostJoinCursor;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* This class serves as the Storage Adapter for the Unnest Segment and is responsible for creating the cursors
|
||||
|
@ -46,20 +60,19 @@ import java.util.Objects;
|
|||
public class UnnestStorageAdapter implements StorageAdapter
|
||||
{
|
||||
private final StorageAdapter baseAdapter;
|
||||
private final String dimensionToUnnest;
|
||||
private final VirtualColumn unnestColumn;
|
||||
private final String outputColumnName;
|
||||
private final LinkedHashSet<String> allowSet;
|
||||
|
||||
public UnnestStorageAdapter(
|
||||
final StorageAdapter baseAdapter,
|
||||
final String dimension,
|
||||
final String outputColumnName,
|
||||
final VirtualColumn unnestColumn,
|
||||
final LinkedHashSet<String> allowSet
|
||||
)
|
||||
{
|
||||
this.baseAdapter = baseAdapter;
|
||||
this.dimensionToUnnest = dimension;
|
||||
this.outputColumnName = outputColumnName;
|
||||
this.unnestColumn = unnestColumn;
|
||||
this.outputColumnName = unnestColumn.getOutputName();
|
||||
this.allowSet = allowSet;
|
||||
}
|
||||
|
||||
|
@ -73,22 +86,20 @@ public class UnnestStorageAdapter implements StorageAdapter
|
|||
@Nullable QueryMetrics<?> queryMetrics
|
||||
)
|
||||
{
|
||||
Filter updatedFilter;
|
||||
if (allowSet != null && !allowSet.isEmpty()) {
|
||||
final InDimFilter allowListFilters;
|
||||
allowListFilters = new InDimFilter(dimensionToUnnest, allowSet);
|
||||
if (filter != null) {
|
||||
updatedFilter = new AndFilter(Arrays.asList(filter, allowListFilters));
|
||||
} else {
|
||||
updatedFilter = allowListFilters;
|
||||
}
|
||||
} else {
|
||||
updatedFilter = filter;
|
||||
}
|
||||
final Sequence<Cursor> baseCursorSequence = baseAdapter.makeCursors(
|
||||
updatedFilter,
|
||||
interval,
|
||||
final String inputColumn = getUnnestInputIfDirectAccess();
|
||||
final Pair<Filter, Filter> filterPair = computeBaseAndPostCorrelateFilters(
|
||||
filter,
|
||||
virtualColumns,
|
||||
inputColumn,
|
||||
inputColumn == null || virtualColumns.exists(inputColumn)
|
||||
? null
|
||||
: baseAdapter.getColumnCapabilities(inputColumn)
|
||||
);
|
||||
|
||||
final Sequence<Cursor> baseCursorSequence = baseAdapter.makeCursors(
|
||||
filterPair.lhs,
|
||||
interval,
|
||||
VirtualColumns.create(Collections.singletonList(unnestColumn)),
|
||||
gran,
|
||||
descending,
|
||||
queryMetrics
|
||||
|
@ -99,13 +110,16 @@ public class UnnestStorageAdapter implements StorageAdapter
|
|||
cursor -> {
|
||||
Objects.requireNonNull(cursor);
|
||||
Cursor retVal = cursor;
|
||||
ColumnCapabilities capabilities = cursor.getColumnSelectorFactory().getColumnCapabilities(dimensionToUnnest);
|
||||
ColumnCapabilities capabilities = unnestColumn.capabilities(
|
||||
cursor.getColumnSelectorFactory(),
|
||||
unnestColumn.getOutputName()
|
||||
);
|
||||
if (capabilities != null) {
|
||||
if (capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) {
|
||||
retVal = new UnnestDimensionCursor(
|
||||
retVal,
|
||||
retVal.getColumnSelectorFactory(),
|
||||
dimensionToUnnest,
|
||||
unnestColumn,
|
||||
outputColumnName,
|
||||
allowSet
|
||||
);
|
||||
|
@ -113,7 +127,7 @@ public class UnnestStorageAdapter implements StorageAdapter
|
|||
retVal = new UnnestColumnValueSelectorCursor(
|
||||
retVal,
|
||||
retVal.getColumnSelectorFactory(),
|
||||
dimensionToUnnest,
|
||||
unnestColumn,
|
||||
outputColumnName,
|
||||
allowSet
|
||||
);
|
||||
|
@ -122,12 +136,16 @@ public class UnnestStorageAdapter implements StorageAdapter
|
|||
retVal = new UnnestColumnValueSelectorCursor(
|
||||
retVal,
|
||||
retVal.getColumnSelectorFactory(),
|
||||
dimensionToUnnest,
|
||||
unnestColumn,
|
||||
outputColumnName,
|
||||
allowSet
|
||||
);
|
||||
}
|
||||
return retVal;
|
||||
return PostJoinCursor.wrap(
|
||||
retVal,
|
||||
virtualColumns,
|
||||
filterPair.rhs
|
||||
);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
@ -162,7 +180,7 @@ public class UnnestStorageAdapter implements StorageAdapter
|
|||
if (!outputColumnName.equals(column)) {
|
||||
return baseAdapter.getDimensionCardinality(column);
|
||||
}
|
||||
return baseAdapter.getDimensionCardinality(dimensionToUnnest);
|
||||
return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -181,30 +199,33 @@ public class UnnestStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public Comparable getMinValue(String column)
|
||||
{
|
||||
if (!outputColumnName.equals(column)) {
|
||||
return baseAdapter.getMinValue(column);
|
||||
if (outputColumnName.equals(column)) {
|
||||
return null;
|
||||
}
|
||||
return baseAdapter.getMinValue(dimensionToUnnest);
|
||||
|
||||
return baseAdapter.getMinValue(column);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Comparable getMaxValue(String column)
|
||||
{
|
||||
if (!outputColumnName.equals(column)) {
|
||||
return baseAdapter.getMaxValue(column);
|
||||
if (outputColumnName.equals(column)) {
|
||||
return null;
|
||||
}
|
||||
return baseAdapter.getMaxValue(dimensionToUnnest);
|
||||
|
||||
return baseAdapter.getMaxValue(column);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ColumnCapabilities getColumnCapabilities(String column)
|
||||
{
|
||||
if (!outputColumnName.equals(column)) {
|
||||
return baseAdapter.getColumnCapabilities(column);
|
||||
if (outputColumnName.equals(column)) {
|
||||
return unnestColumn.capabilities(baseAdapter, column);
|
||||
}
|
||||
return baseAdapter.getColumnCapabilities(dimensionToUnnest);
|
||||
|
||||
return baseAdapter.getColumnCapabilities(column);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -226,9 +247,146 @@ public class UnnestStorageAdapter implements StorageAdapter
|
|||
return baseAdapter.getMetadata();
|
||||
}
|
||||
|
||||
public String getDimensionToUnnest()
|
||||
public VirtualColumn getUnnestColumn()
|
||||
{
|
||||
return dimensionToUnnest;
|
||||
return unnestColumn;
|
||||
}
|
||||
|
||||
/**
|
||||
* Split queryFilter into pre- and post-correlate filters.
|
||||
*
|
||||
* @param queryFilter query filter passed to makeCursors
|
||||
* @param queryVirtualColumns query virtual columns passed to makeCursors
|
||||
* @param inputColumn input column to unnest if it's a direct access; otherwise null
|
||||
* @param inputColumnCapabilites input column capabilities if known; otherwise null
|
||||
*
|
||||
* @return pair of pre- and post-correlate filters
|
||||
*/
|
||||
private Pair<Filter, Filter> computeBaseAndPostCorrelateFilters(
|
||||
@Nullable final Filter queryFilter,
|
||||
final VirtualColumns queryVirtualColumns,
|
||||
@Nullable final String inputColumn,
|
||||
@Nullable final ColumnCapabilities inputColumnCapabilites
|
||||
)
|
||||
{
|
||||
class FilterSplitter
|
||||
{
|
||||
final List<Filter> preFilters = new ArrayList<>();
|
||||
final List<Filter> postFilters = new ArrayList<>();
|
||||
|
||||
void add(@Nullable final Filter filter)
|
||||
{
|
||||
if (filter == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Set<String> requiredColumns = filter.getRequiredColumns();
|
||||
|
||||
// Run filter post-correlate if it refers to any virtual columns.
|
||||
if (queryVirtualColumns.getVirtualColumns().length > 0) {
|
||||
for (String column : requiredColumns) {
|
||||
if (queryVirtualColumns.exists(column)) {
|
||||
postFilters.add(filter);
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (requiredColumns.contains(outputColumnName)) {
|
||||
// Try to move filter pre-correlate if possible.
|
||||
final Filter newFilter = rewriteFilterOnUnnestColumnIfPossible(filter, inputColumn, inputColumnCapabilites);
|
||||
if (newFilter != null) {
|
||||
preFilters.add(newFilter);
|
||||
} else {
|
||||
postFilters.add(filter);
|
||||
}
|
||||
} else {
|
||||
preFilters.add(filter);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final FilterSplitter filterSplitter = new FilterSplitter();
|
||||
|
||||
if (allowSet != null && !allowSet.isEmpty()) {
|
||||
// Filter on input column if possible (it may be faster); otherwise use output column.
|
||||
filterSplitter.add(new InDimFilter(inputColumn != null ? inputColumn : outputColumnName, allowSet));
|
||||
}
|
||||
|
||||
if (queryFilter instanceof AndFilter) {
|
||||
for (Filter filter : ((AndFilter) queryFilter).getFilters()) {
|
||||
filterSplitter.add(filter);
|
||||
}
|
||||
} else {
|
||||
filterSplitter.add(queryFilter);
|
||||
}
|
||||
|
||||
return Pair.of(
|
||||
Filters.maybeAnd(filterSplitter.preFilters).orElse(null),
|
||||
Filters.maybeAnd(filterSplitter.postFilters).orElse(null)
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the input of {@link #unnestColumn}, if it's a direct access; otherwise returns null.
|
||||
*/
|
||||
@Nullable
|
||||
private String getUnnestInputIfDirectAccess()
|
||||
{
|
||||
if (unnestColumn instanceof ExpressionVirtualColumn) {
|
||||
return ((ExpressionVirtualColumn) unnestColumn).getParsedExpression().get().getBindingIfIdentifier();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Rewrites a filter on {@link #outputColumnName} to operate on the input column from
|
||||
* {@link #getUnnestInputIfDirectAccess()}, if possible.
|
||||
*/
|
||||
@Nullable
|
||||
private Filter rewriteFilterOnUnnestColumnIfPossible(
|
||||
final Filter filter,
|
||||
@Nullable final String inputColumn,
|
||||
@Nullable final ColumnCapabilities inputColumnCapabilities
|
||||
)
|
||||
{
|
||||
// Only doing this for multi-value strings (not array types) at the moment.
|
||||
if (inputColumn == null
|
||||
|| inputColumnCapabilities == null
|
||||
|| inputColumnCapabilities.getType() != ValueType.STRING) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (filterMapsOverMultiValueStrings(filter)) {
|
||||
return filter.rewriteRequiredColumns(ImmutableMap.of(outputColumnName, inputColumn));
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Requirement for {@link #rewriteFilterOnUnnestColumnIfPossible}: filter must support rewrites and also must map
|
||||
* over multi-value strings. (Rather than treat them as arrays.) There isn't a method on the Filter interface that
|
||||
* tells us this, so resort to instanceof.
|
||||
*/
|
||||
private static boolean filterMapsOverMultiValueStrings(final Filter filter)
|
||||
{
|
||||
if (filter instanceof BooleanFilter) {
|
||||
for (Filter child : ((BooleanFilter) filter).getFilters()) {
|
||||
if (!filterMapsOverMultiValueStrings(child)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
} else if (filter instanceof NotFilter) {
|
||||
return filterMapsOverMultiValueStrings(((NotFilter) filter).getBaseFilter());
|
||||
} else {
|
||||
return filter instanceof SelectorFilter
|
||||
|| filter instanceof InDimFilter
|
||||
|| filter instanceof LikeFilter
|
||||
|| filter instanceof BoundFilter;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -49,6 +49,7 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
|
@ -217,6 +218,28 @@ public class AndFilter implements BooleanFilter
|
|||
return filters;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsRequiredColumnRewrite()
|
||||
{
|
||||
for (Filter filter : filters) {
|
||||
if (!filter.supportsRequiredColumnRewrite()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter rewriteRequiredColumns(Map<String, String> columnRewrites)
|
||||
{
|
||||
final List<Filter> newFilters = new ArrayList<>(filters.size());
|
||||
for (Filter filter : filters) {
|
||||
newFilters.add(filter.rewriteRequiredColumns(columnRewrites));
|
||||
}
|
||||
return new AndFilter(newFilters);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -48,6 +48,7 @@ import javax.annotation.Nullable;
|
|||
import java.util.ArrayList;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
|
@ -187,6 +188,28 @@ public class OrFilter implements BooleanFilter
|
|||
return filters;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsRequiredColumnRewrite()
|
||||
{
|
||||
for (Filter filter : filters) {
|
||||
if (!filter.supportsRequiredColumnRewrite()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter rewriteRequiredColumns(Map<String, String> columnRewrites)
|
||||
{
|
||||
final List<Filter> newFilters = new ArrayList<>(filters.size());
|
||||
for (Filter filter : filters) {
|
||||
newFilters.add(filter.rewriteRequiredColumns(columnRewrites));
|
||||
}
|
||||
return new OrFilter(newFilters);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -292,10 +292,18 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter
|
|||
);
|
||||
|
||||
final JoinFilterPreAnalysisKey keyCached = joinFilterPreAnalysis.getKey();
|
||||
final JoinFilterSplit joinFilterSplit;
|
||||
|
||||
if (!keyIn.equals(keyCached)) {
|
||||
// It is a bug if this happens. The implied key and the cached key should always match.
|
||||
throw new ISE("Pre-analysis mismatch, cannot execute query");
|
||||
if (keyIn.equals(keyCached)) {
|
||||
// Common case: key used during filter pre-analysis (keyCached) matches key implied by makeCursors call (keyIn).
|
||||
joinFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis, baseFilter);
|
||||
} else {
|
||||
// Less common case: key differs. Re-analyze the filter. This case can happen when an unnest datasource is
|
||||
// layered on top of a join datasource.
|
||||
joinFilterSplit = JoinFilterAnalyzer.splitFilter(
|
||||
JoinFilterAnalyzer.computeJoinFilterPreAnalysis(keyIn),
|
||||
baseFilter
|
||||
);
|
||||
}
|
||||
|
||||
final List<VirtualColumn> preJoinVirtualColumns = new ArrayList<>();
|
||||
|
@ -309,10 +317,8 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter
|
|||
|
||||
// We merge the filter on base table specified by the user and filter on the base table that is pushed from
|
||||
// the join
|
||||
JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis, baseFilter);
|
||||
preJoinVirtualColumns.addAll(joinFilterSplit.getPushDownVirtualColumns());
|
||||
|
||||
|
||||
final Sequence<Cursor> baseCursorSequence = baseAdapter.makeCursors(
|
||||
joinFilterSplit.getBaseTableFilter().isPresent() ? joinFilterSplit.getBaseTableFilter().get() : null,
|
||||
interval,
|
||||
|
|
|
@ -84,7 +84,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
|||
public ExpressionVirtualColumn(
|
||||
String name,
|
||||
Expr parsedExpression,
|
||||
ColumnType outputType
|
||||
@Nullable ColumnType outputType
|
||||
)
|
||||
{
|
||||
this.name = Preconditions.checkNotNull(name, "name");
|
||||
|
@ -129,6 +129,12 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
|||
final ColumnSelectorFactory columnSelectorFactory
|
||||
)
|
||||
{
|
||||
if (isDirectAccess(columnSelectorFactory)) {
|
||||
return columnSelectorFactory.makeDimensionSelector(
|
||||
dimensionSpec.withDimension(parsedExpression.get().getBindingIfIdentifier())
|
||||
);
|
||||
}
|
||||
|
||||
return dimensionSpec.decorate(
|
||||
ExpressionSelectors.makeDimensionSelector(
|
||||
columnSelectorFactory,
|
||||
|
@ -141,6 +147,10 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
|||
@Override
|
||||
public ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelectorFactory factory)
|
||||
{
|
||||
if (isDirectAccess(factory)) {
|
||||
return factory.makeColumnValueSelector(parsedExpression.get().getBindingIfIdentifier());
|
||||
}
|
||||
|
||||
final ColumnCapabilities capabilities = capabilities(factory, name);
|
||||
// we make a special column value selector for values that are expected to be STRING to conform to behavior of
|
||||
// other single and multi-value STRING selectors, whose getObject is expected to produce a single STRING value
|
||||
|
@ -154,6 +164,11 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
|||
@Override
|
||||
public boolean canVectorize(ColumnInspector inspector)
|
||||
{
|
||||
if (isDirectAccess(inspector)) {
|
||||
// Can vectorize if the underlying adapter can vectorize.
|
||||
return true;
|
||||
}
|
||||
|
||||
final ExpressionPlan plan = ExpressionPlanner.plan(inspector, parsedExpression.get());
|
||||
return plan.is(ExpressionPlan.Trait.VECTORIZABLE);
|
||||
}
|
||||
|
@ -164,18 +179,32 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
|||
VectorColumnSelectorFactory factory
|
||||
)
|
||||
{
|
||||
if (isDirectAccess(factory)) {
|
||||
return factory.makeSingleValueDimensionSelector(
|
||||
dimensionSpec.withDimension(parsedExpression.get().getBindingIfIdentifier())
|
||||
);
|
||||
}
|
||||
|
||||
return ExpressionVectorSelectors.makeSingleValueDimensionVectorSelector(factory, parsedExpression.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueSelector makeVectorValueSelector(String columnName, VectorColumnSelectorFactory factory)
|
||||
{
|
||||
if (isDirectAccess(factory)) {
|
||||
return factory.makeValueSelector(parsedExpression.get().getBindingIfIdentifier());
|
||||
}
|
||||
|
||||
return ExpressionVectorSelectors.makeVectorValueSelector(factory, parsedExpression.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorObjectSelector makeVectorObjectSelector(String columnName, VectorColumnSelectorFactory factory)
|
||||
{
|
||||
if (isDirectAccess(factory)) {
|
||||
return factory.makeObjectSelector(parsedExpression.get().getBindingIfIdentifier());
|
||||
}
|
||||
|
||||
return ExpressionVectorSelectors.makeVectorObjectSelector(factory, parsedExpression.get());
|
||||
}
|
||||
|
||||
|
@ -198,6 +227,10 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
|||
@Override
|
||||
public ColumnCapabilities capabilities(ColumnInspector inspector, String columnName)
|
||||
{
|
||||
if (isDirectAccess(inspector)) {
|
||||
return inspector.getColumnCapabilities(parsedExpression.get().getBindingIfIdentifier());
|
||||
}
|
||||
|
||||
final ExpressionPlan plan = ExpressionPlanner.plan(inspector, parsedExpression.get());
|
||||
final ColumnCapabilities inferred = plan.inferColumnCapabilities(outputType);
|
||||
// if we can infer the column capabilities from the expression plan, then use that
|
||||
|
@ -278,6 +311,28 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
|||
'}';
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether this expression is an identifier that directly accesses an underlying column. In this case we skip
|
||||
* the expression system entirely, and directly return backing columns.
|
||||
*/
|
||||
private boolean isDirectAccess(final ColumnInspector inspector)
|
||||
{
|
||||
if (parsedExpression.get().isIdentifier()) {
|
||||
final ColumnCapabilities baseCapabilities =
|
||||
inspector.getColumnCapabilities(parsedExpression.get().getBindingIfIdentifier());
|
||||
|
||||
if (outputType == null) {
|
||||
// No desired output type. Anything from the source is fine.
|
||||
return true;
|
||||
} else if (baseCapabilities != null && outputType.equals(baseCapabilities.toColumnType())) {
|
||||
// Desired output type matches the type from the source.
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
private Supplier<byte[]> makeCacheKeySupplier()
|
||||
{
|
||||
return Suppliers.memoize(() -> {
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.guava.MergeSequence;
|
|||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.js.JavaScriptConfig;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory;
|
||||
|
@ -66,6 +67,7 @@ import org.apache.druid.segment.Segment;
|
|||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.TestIndex;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.apache.druid.timeline.TimelineObjectHolder;
|
||||
import org.apache.druid.timeline.VersionedIntervalTimeline;
|
||||
|
@ -104,8 +106,12 @@ public class QueryRunnerTestHelper
|
|||
|
||||
public static final DataSource UNNEST_DATA_SOURCE = UnnestDataSource.create(
|
||||
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION,
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
new ExpressionVirtualColumn(
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
"\"" + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION + "\"",
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
),
|
||||
null
|
||||
);
|
||||
|
||||
|
|
|
@ -13398,7 +13398,7 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
"minExpression",
|
||||
NullHandling.replaceWithDefault() ? Float.POSITIVE_INFINITY : null,
|
||||
"minVc",
|
||||
NullHandling.replaceWithDefault() ? Float.POSITIVE_INFINITY : null
|
||||
Float.POSITIVE_INFINITY
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -26,7 +26,9 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.DirectQueryProcessingPool;
|
||||
import org.apache.druid.query.DruidProcessingConfig;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryRunner;
|
||||
|
@ -49,7 +51,6 @@ import org.apache.druid.segment.column.ColumnType;
|
|||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
|
@ -227,28 +228,18 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
return GroupByQueryRunnerTestHelper.createExpectedRow(query, timestamp, vals);
|
||||
}
|
||||
|
||||
private static ResultRow makeRow(final GroupByQuery query, final DateTime timestamp, final Object... vals)
|
||||
{
|
||||
return GroupByQueryRunnerTestHelper.createExpectedRow(query, timestamp, vals);
|
||||
}
|
||||
|
||||
private static List<ResultRow> makeRows(
|
||||
final GroupByQuery query,
|
||||
final String[] columnNames,
|
||||
final Object[]... values
|
||||
)
|
||||
{
|
||||
return GroupByQueryRunnerTestHelper.createExpectedRows(query, columnNames, values);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupBy()
|
||||
{
|
||||
GroupByQuery query = makeQueryBuilder()
|
||||
.setDataSource(UnnestDataSource.create(
|
||||
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION,
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
new ExpressionVirtualColumn(
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
"\"" + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION + "\"",
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
),
|
||||
null
|
||||
))
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD)
|
||||
|
@ -443,17 +434,8 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
252L
|
||||
)
|
||||
);
|
||||
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex();
|
||||
final QueryRunner queryRunner = QueryRunnerTestHelper.makeQueryRunnerWithSegmentMapFn(
|
||||
factory,
|
||||
new IncrementalIndexSegment(
|
||||
rtIndex,
|
||||
QueryRunnerTestHelper.SEGMENT_ID
|
||||
),
|
||||
query,
|
||||
"rtIndexvc"
|
||||
);
|
||||
Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, queryRunner, query);
|
||||
|
||||
Iterable<ResultRow> results = runQuery(query, TestIndex.getIncrementalTestIndex());
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "groupBy");
|
||||
}
|
||||
|
||||
|
@ -466,8 +448,12 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
GroupByQuery query = makeQueryBuilder()
|
||||
.setDataSource(UnnestDataSource.create(
|
||||
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION,
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
new ExpressionVirtualColumn(
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
"\"" + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION + "\"",
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
),
|
||||
null
|
||||
))
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD)
|
||||
|
@ -487,17 +473,8 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
"rows", 52L
|
||||
)
|
||||
);
|
||||
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex();
|
||||
final QueryRunner queryRunner = QueryRunnerTestHelper.makeQueryRunnerWithSegmentMapFn(
|
||||
factory,
|
||||
new IncrementalIndexSegment(
|
||||
rtIndex,
|
||||
QueryRunnerTestHelper.SEGMENT_ID
|
||||
),
|
||||
query,
|
||||
"rtIndexvc"
|
||||
);
|
||||
Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, queryRunner, query);
|
||||
|
||||
Iterable<ResultRow> results = runQuery(query, TestIndex.getIncrementalTestIndex());
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "missing-column");
|
||||
}
|
||||
|
||||
|
@ -509,9 +486,8 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
GroupByQuery query = makeQueryBuilder()
|
||||
.setDataSource(QueryRunnerTestHelper.UNNEST_DATA_SOURCE)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD)
|
||||
.setDimensions(
|
||||
new DefaultDimensionSpec(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, "alias0")
|
||||
).setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT)
|
||||
.setDimensions(new DefaultDimensionSpec(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, "alias0"))
|
||||
.setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT)
|
||||
.setGranularity(QueryRunnerTestHelper.ALL_GRAN)
|
||||
.build();
|
||||
|
||||
|
@ -524,12 +500,6 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
"alias0", "a",
|
||||
"rows", 2L
|
||||
),
|
||||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
"alias0", "preferred",
|
||||
"rows", 26L
|
||||
),
|
||||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
|
@ -566,6 +536,12 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
"alias0", "p",
|
||||
"rows", 6L
|
||||
),
|
||||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
"alias0", "preferred",
|
||||
"rows", 26L
|
||||
),
|
||||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
|
@ -573,17 +549,8 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
"rows", 4L
|
||||
)
|
||||
);
|
||||
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex();
|
||||
final QueryRunner queryRunner = QueryRunnerTestHelper.makeQueryRunnerWithSegmentMapFn(
|
||||
factory,
|
||||
new IncrementalIndexSegment(
|
||||
rtIndex,
|
||||
QueryRunnerTestHelper.SEGMENT_ID
|
||||
),
|
||||
query,
|
||||
"rtIndexvc"
|
||||
);
|
||||
Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, queryRunner, query);
|
||||
|
||||
Iterable<ResultRow> results = runQuery(query, TestIndex.getIncrementalTestIndex());
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "groupBy-on-unnested-column");
|
||||
}
|
||||
|
||||
|
@ -594,8 +561,12 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
|
||||
final DataSource unnestDataSource = UnnestDataSource.create(
|
||||
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
|
||||
"vc",
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
new ExpressionVirtualColumn(
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
"mv_to_array(placementish)",
|
||||
ColumnType.STRING_ARRAY,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
null
|
||||
);
|
||||
|
||||
|
@ -606,14 +577,6 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
new DefaultDimensionSpec(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, "alias0")
|
||||
).setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT)
|
||||
.setGranularity(QueryRunnerTestHelper.ALL_GRAN)
|
||||
.setVirtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"vc",
|
||||
"mv_to_array(placementish)",
|
||||
ColumnType.STRING_ARRAY,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.addOrderByColumn("alias0", OrderByColumnSpec.Direction.ASCENDING)
|
||||
.build();
|
||||
|
||||
|
@ -623,13 +586,7 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
"alias0", "preferred",
|
||||
"rows", 26L
|
||||
),
|
||||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
"alias0", "e",
|
||||
"alias0", "a",
|
||||
"rows", 2L
|
||||
),
|
||||
makeRow(
|
||||
|
@ -641,13 +598,13 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
"alias0", "h",
|
||||
"alias0", "e",
|
||||
"rows", 2L
|
||||
),
|
||||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
"alias0", "a",
|
||||
"alias0", "h",
|
||||
"rows", 2L
|
||||
),
|
||||
makeRow(
|
||||
|
@ -668,6 +625,12 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
"alias0", "p",
|
||||
"rows", 6L
|
||||
),
|
||||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
"alias0", "preferred",
|
||||
"rows", 26L
|
||||
),
|
||||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
|
@ -676,18 +639,7 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
)
|
||||
);
|
||||
|
||||
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex();
|
||||
final QueryRunner queryRunner = QueryRunnerTestHelper.makeQueryRunnerWithSegmentMapFn(
|
||||
factory,
|
||||
new IncrementalIndexSegment(
|
||||
rtIndex,
|
||||
QueryRunnerTestHelper.SEGMENT_ID
|
||||
),
|
||||
query,
|
||||
"rtIndexvc"
|
||||
);
|
||||
Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, queryRunner, query);
|
||||
|
||||
Iterable<ResultRow> results = runQuery(query, TestIndex.getIncrementalTestIndex());
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "groupBy-on-unnested-virtual-column");
|
||||
}
|
||||
|
||||
|
@ -698,32 +650,32 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
|
||||
final DataSource unnestDataSource = UnnestDataSource.create(
|
||||
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
|
||||
"vc",
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
new ExpressionVirtualColumn(
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
"array(\"market\",\"quality\")",
|
||||
ColumnType.STRING,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
null
|
||||
);
|
||||
|
||||
GroupByQuery query = makeQueryBuilder()
|
||||
.setDataSource(unnestDataSource)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD)
|
||||
.setDimensions(
|
||||
new DefaultDimensionSpec(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, "alias0")
|
||||
).setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT)
|
||||
.setDimensions(new DefaultDimensionSpec(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST, "alias0"))
|
||||
.setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT)
|
||||
.setGranularity(QueryRunnerTestHelper.ALL_GRAN)
|
||||
.setVirtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"vc",
|
||||
"array(\"market\",\"quality\")",
|
||||
ColumnType.STRING,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.setLimit(3)
|
||||
.build();
|
||||
|
||||
// Total rows should add up to 26 * 2 = 52
|
||||
// 26 rows and each has 2 entries in the column to be unnested
|
||||
// Each count should be 2, since we are unnesting "market" and "quality", which are singly-valued fields.
|
||||
List<ResultRow> expectedResults = Arrays.asList(
|
||||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
"alias0", "automotive",
|
||||
"rows", 2L
|
||||
),
|
||||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
|
@ -733,28 +685,12 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
"alias0", "health",
|
||||
"rows", 2L
|
||||
),
|
||||
makeRow(
|
||||
query,
|
||||
"2011-04-01",
|
||||
"alias0", "travel",
|
||||
"alias0", "entertainment",
|
||||
"rows", 2L
|
||||
)
|
||||
);
|
||||
|
||||
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex();
|
||||
final QueryRunner queryRunner = QueryRunnerTestHelper.makeQueryRunnerWithSegmentMapFn(
|
||||
factory,
|
||||
new IncrementalIndexSegment(
|
||||
rtIndex,
|
||||
QueryRunnerTestHelper.SEGMENT_ID
|
||||
),
|
||||
query,
|
||||
"rtIndexvc"
|
||||
);
|
||||
Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, queryRunner, query);
|
||||
Iterable<ResultRow> results = runQuery(query, TestIndex.getIncrementalTestIndex());
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "groupBy-on-unnested-virtual-columns");
|
||||
}
|
||||
|
||||
|
@ -767,13 +703,24 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
return GroupByQuery.builder().overrideContext(makeContext());
|
||||
}
|
||||
|
||||
/**
|
||||
* Use this method instead of makeQueryBuilder() to make sure the context is set properly. Also, avoid
|
||||
* setContext in tests. Only use overrideContext.
|
||||
*/
|
||||
private GroupByQuery.Builder makeQueryBuilder(final GroupByQuery query)
|
||||
private Iterable<ResultRow> runQuery(final GroupByQuery query, final IncrementalIndex index)
|
||||
{
|
||||
return new GroupByQuery.Builder(query).overrideContext(makeContext());
|
||||
final QueryRunner<?> queryRunner = factory.mergeRunners(
|
||||
DirectQueryProcessingPool.INSTANCE,
|
||||
Collections.singletonList(
|
||||
QueryRunnerTestHelper.makeQueryRunnerWithSegmentMapFn(
|
||||
factory,
|
||||
new IncrementalIndexSegment(
|
||||
index,
|
||||
QueryRunnerTestHelper.SEGMENT_ID
|
||||
),
|
||||
query,
|
||||
"rtIndexvc"
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
return GroupByQueryRunnerTestHelper.runQuery(factory, queryRunner, query);
|
||||
}
|
||||
|
||||
private Map<String, Object> makeContext()
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.query.scan;
|
|||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.DefaultGenericQueryMetricsFactory;
|
||||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.QueryPlus;
|
||||
|
@ -102,8 +103,12 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
|
|||
return Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION,
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
new ExpressionVirtualColumn(
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
"\"" + QueryRunnerTestHelper.PLACEMENTISH_DIMENSION + "\"",
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
),
|
||||
allowSet
|
||||
))
|
||||
.columns(Collections.emptyList())
|
||||
|
@ -178,21 +183,17 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
|
|||
.intervals(I_0112_0114)
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
|
||||
"vc",
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
new ExpressionVirtualColumn(
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
"mv_to_array(placementish)",
|
||||
ColumnType.STRING,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
null
|
||||
))
|
||||
.columns(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST)
|
||||
.eternityInterval()
|
||||
.legacy(legacy)
|
||||
.virtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"vc",
|
||||
"mv_to_array(placementish)",
|
||||
ColumnType.STRING,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.limit(3)
|
||||
.build();
|
||||
|
||||
|
@ -252,21 +253,17 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
|
|||
.intervals(I_0112_0114)
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
|
||||
"vc",
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
new ExpressionVirtualColumn(
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
"array(\"market\",\"quality\")",
|
||||
ColumnType.STRING,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
null
|
||||
))
|
||||
.columns(QueryRunnerTestHelper.MARKET_DIMENSION, QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST)
|
||||
.eternityInterval()
|
||||
.legacy(legacy)
|
||||
.virtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"vc",
|
||||
"array(\"market\",\"quality\")",
|
||||
ColumnType.STRING,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.limit(4)
|
||||
.build();
|
||||
|
||||
|
|
|
@ -253,19 +253,15 @@ public class UnnestTopNQueryRunnerTest extends InitializedNullHandlingTest
|
|||
TopNQuery query = new TopNQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
|
||||
"vc",
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
null
|
||||
))
|
||||
.granularity(QueryRunnerTestHelper.ALL_GRAN)
|
||||
.virtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"vc",
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
"mv_to_array(\"placementish\")",
|
||||
ColumnType.STRING_ARRAY,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
),
|
||||
null
|
||||
))
|
||||
.granularity(QueryRunnerTestHelper.ALL_GRAN)
|
||||
.dimension(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST)
|
||||
.metric("rows")
|
||||
.threshold(4)
|
||||
|
@ -340,19 +336,15 @@ public class UnnestTopNQueryRunnerTest extends InitializedNullHandlingTest
|
|||
TopNQuery query = new TopNQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE),
|
||||
"vc",
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
null
|
||||
))
|
||||
.granularity(QueryRunnerTestHelper.ALL_GRAN)
|
||||
.virtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"vc",
|
||||
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST,
|
||||
"array(\"market\",\"quality\")",
|
||||
ColumnType.STRING,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
),
|
||||
null
|
||||
))
|
||||
.granularity(QueryRunnerTestHelper.ALL_GRAN)
|
||||
.dimension(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST)
|
||||
.metric("rows")
|
||||
.threshold(2)
|
||||
|
|
|
@ -41,7 +41,7 @@ import java.util.List;
|
|||
*/
|
||||
public class ListCursor implements Cursor
|
||||
{
|
||||
List<Object> baseList;
|
||||
private final List<Object> baseList;
|
||||
private int index;
|
||||
|
||||
public ListCursor(List<Object> inputList)
|
||||
|
@ -62,19 +62,19 @@ public class ListCursor implements Cursor
|
|||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
return null;
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(@Nullable String value)
|
||||
{
|
||||
return null;
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(Predicate<String> predicate)
|
||||
{
|
||||
return null;
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -96,20 +96,20 @@ public class ListCursor implements Cursor
|
|||
@Override
|
||||
public Class<?> classOfObject()
|
||||
{
|
||||
return null;
|
||||
return Object.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return 0;
|
||||
return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return null;
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -175,7 +175,7 @@ public class ListCursor implements Cursor
|
|||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return null;
|
||||
return Object.class;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -192,7 +192,7 @@ public class ListCursor implements Cursor
|
|||
@Override
|
||||
public DateTime getTime()
|
||||
{
|
||||
return null;
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,9 +19,17 @@
|
|||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.math.expr.ExpressionProcessing;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.monomorphicprocessing.StringRuntimeShape;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -36,6 +44,18 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
private static LinkedHashSet<String> IGNORE_SET = null;
|
||||
private static LinkedHashSet<String> IGNORE_SET1 = new LinkedHashSet<>(Arrays.asList("b", "f"));
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpClass()
|
||||
{
|
||||
NullHandling.initializeForTests();
|
||||
ExpressionProcessing.initializeForTests(true); // Allow nested arrays
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownClass()
|
||||
{
|
||||
ExpressionProcessing.initializeForTests(null); // Clear special expression-processing config.
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_list_unnest_cursors()
|
||||
|
@ -52,7 +72,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -86,7 +106,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -118,7 +138,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -155,7 +175,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -189,7 +209,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -219,7 +239,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -246,14 +266,15 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
|
||||
List<Object> expectedResults = Arrays.asList("a", "b", "c", "e", "f", "g", "h", "i", "j", Arrays.asList("a", "b"));
|
||||
|
||||
//Create base cursor
|
||||
// Create base cursor. Need to set type to STRING; otherwise auto-detected type is STRING_ARRAY and the "j" will
|
||||
// be wrapped in an array (which we don't want).
|
||||
ListCursor listCursor = new ListCursor(inputList);
|
||||
|
||||
//Create unnest cursor
|
||||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", null, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -287,14 +308,14 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor childCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", null, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
UnnestColumnValueSelectorCursor parentCursor = new UnnestColumnValueSelectorCursor(
|
||||
childCursor,
|
||||
childCursor.getColumnSelectorFactory(),
|
||||
OUTPUT_NAME,
|
||||
new ExpressionVirtualColumn("__unnest__", "\"" + OUTPUT_NAME + "\"", null, ExprMacroTable.nil()),
|
||||
"tmp-out",
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -329,7 +350,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -367,7 +388,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -405,7 +426,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET1
|
||||
);
|
||||
|
@ -443,7 +464,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -477,7 +498,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -511,7 +532,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -548,7 +569,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", null, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -584,7 +605,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
@ -593,7 +614,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
.makeDimensionSelector(
|
||||
DefaultDimensionSpec.of(
|
||||
OUTPUT_NAME));
|
||||
unnestDimSelector.inspectRuntimeShape(null);
|
||||
StringRuntimeShape.of(unnestDimSelector); // Ensure no errors, infinite-loops, etc.
|
||||
int k = 0;
|
||||
while (!unnestCursor.isDone()) {
|
||||
if (k < 8) {
|
||||
|
@ -627,7 +648,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
|
|||
UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
|
||||
listCursor,
|
||||
listCursor.getColumnSelectorFactory(),
|
||||
"dummy",
|
||||
new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING, ExprMacroTable.nil()),
|
||||
OUTPUT_NAME,
|
||||
IGNORE_SET
|
||||
);
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.DateTimes;
|
|||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
@ -32,10 +33,13 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo;
|
|||
import org.apache.druid.segment.generator.SegmentGenerator;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.hamcrest.MatcherAssert;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -82,26 +86,22 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest
|
|||
INCREMENTAL_INDEX_STORAGE_ADAPTER = new IncrementalIndexStorageAdapter(INCREMENTAL_INDEX);
|
||||
UNNEST_STORAGE_ADAPTER = new UnnestStorageAdapter(
|
||||
INCREMENTAL_INDEX_STORAGE_ADAPTER,
|
||||
COLUMNNAME,
|
||||
OUTPUT_COLUMN_NAME,
|
||||
new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + COLUMNNAME + "\"", null, ExprMacroTable.nil()),
|
||||
null
|
||||
);
|
||||
UNNEST_STORAGE_ADAPTER1 = new UnnestStorageAdapter(
|
||||
INCREMENTAL_INDEX_STORAGE_ADAPTER,
|
||||
COLUMNNAME,
|
||||
OUTPUT_COLUMN_NAME,
|
||||
new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + COLUMNNAME + "\"", null, ExprMacroTable.nil()),
|
||||
IGNORE_SET
|
||||
);
|
||||
UNNEST_STORAGE_ADAPTER2 = new UnnestStorageAdapter(
|
||||
UNNEST_STORAGE_ADAPTER,
|
||||
COLUMNNAME,
|
||||
OUTPUT_COLUMN_NAME1,
|
||||
new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME1, "\"" + COLUMNNAME + "\"", null, ExprMacroTable.nil()),
|
||||
null
|
||||
);
|
||||
UNNEST_STORAGE_ADAPTER3 = new UnnestStorageAdapter(
|
||||
UNNEST_STORAGE_ADAPTER1,
|
||||
COLUMNNAME,
|
||||
OUTPUT_COLUMN_NAME1,
|
||||
new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME1, "\"" + COLUMNNAME + "\"", null, ExprMacroTable.nil()),
|
||||
IGNORE_SET
|
||||
);
|
||||
ADAPTERS = ImmutableList.of(
|
||||
|
@ -143,7 +143,8 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest
|
|||
adapter.getColumnCapabilities(colName).toColumnType(),
|
||||
INCREMENTAL_INDEX_STORAGE_ADAPTER.getColumnCapabilities(colName).toColumnType()
|
||||
);
|
||||
Assert.assertEquals(((UnnestStorageAdapter) adapter).getDimensionToUnnest(), colName);
|
||||
|
||||
assertColumnReadsIdentifier(((UnnestStorageAdapter) adapter).getUnnestColumn(), colName);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -173,7 +174,7 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest
|
|||
ColumnCapabilities capabilities = adapter.getColumnCapabilities(columnsInTable.get(i));
|
||||
Assert.assertEquals(capabilities.getType(), valueTypes.get(i));
|
||||
}
|
||||
Assert.assertEquals(adapter.getDimensionToUnnest(), colName);
|
||||
assertColumnReadsIdentifier(adapter.getUnnestColumn(), colName);
|
||||
|
||||
}
|
||||
|
||||
|
@ -312,13 +313,13 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest
|
|||
null
|
||||
);
|
||||
UnnestStorageAdapter adapter = UNNEST_STORAGE_ADAPTER3;
|
||||
Assert.assertEquals(adapter.getDimensionToUnnest(), columnName);
|
||||
assertColumnReadsIdentifier(adapter.getUnnestColumn(), columnName);
|
||||
Assert.assertEquals(
|
||||
adapter.getColumnCapabilities(OUTPUT_COLUMN_NAME).isDictionaryEncoded(),
|
||||
ColumnCapabilities.Capable.TRUE
|
||||
);
|
||||
Assert.assertEquals(adapter.getMaxValue(columnName), adapter.getMaxValue(OUTPUT_COLUMN_NAME));
|
||||
Assert.assertEquals(adapter.getMinValue(columnName), adapter.getMinValue(OUTPUT_COLUMN_NAME));
|
||||
Assert.assertNull(adapter.getMaxValue(OUTPUT_COLUMN_NAME));
|
||||
Assert.assertNull(adapter.getMinValue(OUTPUT_COLUMN_NAME));
|
||||
|
||||
cursorSequence.accumulate(null, (accumulated, cursor) -> {
|
||||
ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
|
||||
|
@ -364,13 +365,13 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest
|
|||
null
|
||||
);
|
||||
UnnestStorageAdapter adapter = UNNEST_STORAGE_ADAPTER1;
|
||||
Assert.assertEquals(adapter.getDimensionToUnnest(), columnName);
|
||||
assertColumnReadsIdentifier(adapter.getUnnestColumn(), columnName);
|
||||
Assert.assertEquals(
|
||||
adapter.getColumnCapabilities(OUTPUT_COLUMN_NAME).isDictionaryEncoded(),
|
||||
ColumnCapabilities.Capable.TRUE
|
||||
);
|
||||
Assert.assertEquals(adapter.getMaxValue(columnName), adapter.getMaxValue(OUTPUT_COLUMN_NAME));
|
||||
Assert.assertEquals(adapter.getMinValue(columnName), adapter.getMinValue(OUTPUT_COLUMN_NAME));
|
||||
Assert.assertNull(adapter.getMaxValue(OUTPUT_COLUMN_NAME));
|
||||
Assert.assertNull(adapter.getMinValue(OUTPUT_COLUMN_NAME));
|
||||
|
||||
cursorSequence.accumulate(null, (accumulated, cursor) -> {
|
||||
ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
|
||||
|
@ -396,4 +397,10 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest
|
|||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
private static void assertColumnReadsIdentifier(final VirtualColumn column, final String identifier)
|
||||
{
|
||||
MatcherAssert.assertThat(column, CoreMatchers.instanceOf(ExpressionVirtualColumn.class));
|
||||
Assert.assertEquals("\"" + identifier + "\"", ((ExpressionVirtualColumn) column).getExpression());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -189,6 +189,11 @@
|
|||
<artifactId>JUnitParams</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-text</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.calcite</groupId>
|
||||
<artifactId>calcite-core</artifactId>
|
||||
|
|
|
@ -187,7 +187,7 @@ public class ArraySqlAggregator implements SqlAggregator
|
|||
OperandTypes.or(
|
||||
OperandTypes.ANY,
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(StringUtils.format("'%s'(expr, maxSizeBytes)", NAME), OperandTypes.ANY, OperandTypes.POSITIVE_INTEGER_LITERAL),
|
||||
OperandTypes.sequence(StringUtils.format("%s(expr, maxSizeBytes)", NAME), OperandTypes.ANY, OperandTypes.POSITIVE_INTEGER_LITERAL),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC)
|
||||
)
|
||||
),
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
|
|||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.rel.type.RelDataTypeFactory;
|
||||
import org.apache.calcite.rex.RexCall;
|
||||
import org.apache.calcite.rex.RexFieldAccess;
|
||||
import org.apache.calcite.rex.RexInputRef;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
|
@ -36,7 +35,6 @@ import org.apache.calcite.sql.type.SqlTypeName;
|
|||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
|
@ -61,7 +59,6 @@ import org.apache.druid.sql.calcite.filtration.Bounds;
|
|||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.rel.CannotBuildQueryException;
|
||||
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
|
||||
import org.apache.druid.sql.calcite.table.RowSignatures;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -125,7 +122,7 @@ public class Expressions
|
|||
if (project == null) {
|
||||
return RexInputRef.of(fieldNumber, RowSignatures.toRelDataType(rowSignature, typeFactory));
|
||||
} else {
|
||||
return project.getChildExps().get(fieldNumber);
|
||||
return project.getProjects().get(fieldNumber);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -238,51 +235,12 @@ public class Expressions
|
|||
return rexCallToDruidExpression(plannerContext, rowSignature, rexNode, postAggregatorVisitor);
|
||||
} else if (kind == SqlKind.LITERAL) {
|
||||
return literalToDruidExpression(plannerContext, rexNode);
|
||||
} else if (kind == SqlKind.FIELD_ACCESS) {
|
||||
return fieldAccessToDruidExpression(rowSignature, rexNode);
|
||||
} else {
|
||||
// Can't translate.
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private static DruidExpression fieldAccessToDruidExpression(
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
)
|
||||
{
|
||||
// Translate field references.
|
||||
final RexFieldAccess ref = (RexFieldAccess) rexNode;
|
||||
if (ref.getField().getIndex() > rowSignature.size()) {
|
||||
// This case arises in the case of a correlation where the rexNode points to a table from the left subtree
|
||||
// while the underlying datasource is the scan stub created from LogicalValuesRule
|
||||
// In such a case we throw a CannotBuildQueryException so that Calcite does not go ahead with this path
|
||||
// This exception is caught while returning false from isValidDruidQuery() method
|
||||
throw new CannotBuildQueryException(StringUtils.format(
|
||||
"Cannot build query as column name [%s] does not exist in row [%s]", ref.getField().getName(), rowSignature)
|
||||
);
|
||||
}
|
||||
|
||||
final String columnName = ref.getField().getName();
|
||||
final int index = rowSignature.indexOf(columnName);
|
||||
|
||||
// This case arises when the rexNode has a name which is not in the underlying stub created using DruidUnnestDataSourceRule
|
||||
// The column name has name ZERO with rowtype as LONG
|
||||
// causes the index to be -1. In such a case we cannot build the query
|
||||
// and throw an exception while returning false from isValidDruidQuery() method
|
||||
if (index < 0) {
|
||||
throw new CannotBuildQueryException(StringUtils.format(
|
||||
"Expression referred to nonexistent index[%d] in row[%s]",
|
||||
index,
|
||||
rowSignature
|
||||
));
|
||||
}
|
||||
|
||||
final Optional<ColumnType> columnType = rowSignature.getColumnType(index);
|
||||
|
||||
return DruidExpression.ofColumn(columnType.get(), columnName);
|
||||
}
|
||||
|
||||
private static DruidExpression inputRefToDruidExpression(
|
||||
final RowSignature rowSignature,
|
||||
final RexNode rexNode
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
|||
|
||||
public class MultiValueStringToArrayOperatorConversion extends DirectOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
public static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder("MV_TO_ARRAY")
|
||||
.operandTypeChecker(OperandTypes.family(SqlTypeFamily.STRING))
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
|
|
|
@ -43,7 +43,7 @@ public class ExternalTableScanRule extends RelOptRule
|
|||
@Override
|
||||
public boolean matches(RelOptRuleCall call)
|
||||
{
|
||||
if (plannerContext.engineHasFeature(EngineFeature.READ_EXTERNAL_DATA)) {
|
||||
if (plannerContext.featureAvailable(EngineFeature.READ_EXTERNAL_DATA)) {
|
||||
return super.matches(call);
|
||||
} else {
|
||||
plannerContext.setPlanningError(
|
||||
|
@ -59,7 +59,7 @@ public class ExternalTableScanRule extends RelOptRule
|
|||
@Override
|
||||
public void onMatch(final RelOptRuleCall call)
|
||||
{
|
||||
if (!plannerContext.engineHasFeature(EngineFeature.READ_EXTERNAL_DATA)) {
|
||||
if (!plannerContext.featureAvailable(EngineFeature.READ_EXTERNAL_DATA)) {
|
||||
// Not called because "matches" returns false.
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
|
|
@ -88,6 +88,7 @@ import org.apache.druid.sql.calcite.rule.ExtensionCalciteRuleProvider;
|
|||
import org.apache.druid.sql.calcite.rule.FilterJoinExcludePushToChildRule;
|
||||
import org.apache.druid.sql.calcite.rule.ProjectAggregatePruneUnusedCallRule;
|
||||
import org.apache.druid.sql.calcite.rule.SortCollapseRule;
|
||||
import org.apache.druid.sql.calcite.run.EngineFeature;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
@ -311,7 +312,8 @@ public class CalciteRulesManager
|
|||
}
|
||||
|
||||
if (!plannerConfig.isUseApproximateCountDistinct()) {
|
||||
if (plannerConfig.isUseGroupingSetForExactDistinct()) {
|
||||
if (plannerConfig.isUseGroupingSetForExactDistinct()
|
||||
&& plannerContext.featureAvailable(EngineFeature.GROUPING_SETS)) {
|
||||
rules.add(AggregateExpandDistinctAggregatesRule.INSTANCE);
|
||||
} else {
|
||||
rules.add(AggregateExpandDistinctAggregatesRule.JOIN);
|
||||
|
|
|
@ -265,7 +265,7 @@ public abstract class IngestHandler extends QueryHandler
|
|||
@Override
|
||||
public void validate() throws ValidationException
|
||||
{
|
||||
if (!handlerContext.plannerContext().engineHasFeature(EngineFeature.CAN_INSERT)) {
|
||||
if (!handlerContext.plannerContext().featureAvailable(EngineFeature.CAN_INSERT)) {
|
||||
throw new ValidationException(StringUtils.format(
|
||||
"Cannot execute INSERT with SQL engine '%s'.",
|
||||
handlerContext.engine().name())
|
||||
|
@ -307,7 +307,7 @@ public abstract class IngestHandler extends QueryHandler
|
|||
@Override
|
||||
public void validate() throws ValidationException
|
||||
{
|
||||
if (!handlerContext.plannerContext().engineHasFeature(EngineFeature.CAN_REPLACE)) {
|
||||
if (!handlerContext.plannerContext().featureAvailable(EngineFeature.CAN_REPLACE)) {
|
||||
throw new ValidationException(StringUtils.format(
|
||||
"Cannot execute REPLACE with SQL engine '%s'.",
|
||||
handlerContext.engine().name())
|
||||
|
|
|
@ -75,6 +75,16 @@ public class PlannerContext
|
|||
*/
|
||||
public static final String CTX_SQL_OUTER_LIMIT = "sqlOuterLimit";
|
||||
|
||||
/**
|
||||
* Undocumented context key, used to enable window functions.
|
||||
*/
|
||||
public static final String CTX_ENABLE_WINDOW_FNS = "windowsAreForClosers";
|
||||
|
||||
/**
|
||||
* Undocumented context key, used to enable {@link org.apache.calcite.sql.fun.SqlStdOperatorTable#UNNEST}.
|
||||
*/
|
||||
public static final String CTX_ENABLE_UNNEST = "enableUnnest";
|
||||
|
||||
// DataContext keys
|
||||
public static final String DATA_CTX_AUTHENTICATION_RESULT = "authenticationResult";
|
||||
|
||||
|
@ -454,9 +464,28 @@ public class PlannerContext
|
|||
return engine;
|
||||
}
|
||||
|
||||
public boolean engineHasFeature(final EngineFeature feature)
|
||||
/**
|
||||
* Checks if the current {@link SqlEngine} supports a particular feature.
|
||||
*
|
||||
* When executing a specific query, use this method instead of
|
||||
* {@link SqlEngine#featureAvailable(EngineFeature, PlannerContext)}, because it also verifies feature flags such as
|
||||
* {@link #CTX_ENABLE_WINDOW_FNS}.
|
||||
*/
|
||||
public boolean featureAvailable(final EngineFeature feature)
|
||||
{
|
||||
return engine.feature(feature, this);
|
||||
if (feature == EngineFeature.WINDOW_FUNCTIONS &&
|
||||
!QueryContexts.getAsBoolean(CTX_ENABLE_WINDOW_FNS, queryContext.get(CTX_ENABLE_WINDOW_FNS), false)) {
|
||||
// Short-circuit: feature requires context flag.
|
||||
return false;
|
||||
}
|
||||
|
||||
if (feature == EngineFeature.UNNEST &&
|
||||
!QueryContexts.getAsBoolean(CTX_ENABLE_UNNEST, queryContext.get(CTX_ENABLE_UNNEST), false)) {
|
||||
// Short-circuit: feature requires context flag.
|
||||
return false;
|
||||
}
|
||||
|
||||
return engine.featureAvailable(feature, this);
|
||||
}
|
||||
|
||||
public QueryMaker getQueryMaker()
|
||||
|
|
|
@ -193,7 +193,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
if (!bindableTables.isEmpty()) {
|
||||
// Consider BINDABLE convention when necessary. Used for metadata tables.
|
||||
|
||||
if (!handlerContext.plannerContext().engineHasFeature(EngineFeature.ALLOW_BINDABLE_PLAN)) {
|
||||
if (!handlerContext.plannerContext().featureAvailable(EngineFeature.ALLOW_BINDABLE_PLAN)) {
|
||||
throw new ValidationException(
|
||||
StringUtils.format(
|
||||
"Cannot query table%s %s with SQL engine '%s'.",
|
||||
|
@ -611,7 +611,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
@Override
|
||||
public void validate() throws ValidationException
|
||||
{
|
||||
if (!handlerContext.plannerContext().engineHasFeature(EngineFeature.CAN_SELECT)) {
|
||||
if (!handlerContext.plannerContext().featureAvailable(EngineFeature.CAN_SELECT)) {
|
||||
throw new ValidationException(StringUtils.format(
|
||||
"Cannot execute SELECT with SQL engine '%s'.",
|
||||
handlerContext.engine().name())
|
||||
|
|
|
@ -54,7 +54,7 @@ public class QueryValidations
|
|||
) throws ValidationException
|
||||
{
|
||||
if (plannerContext.getJoinAlgorithm() == JoinAlgorithm.BROADCAST
|
||||
&& !plannerContext.engineHasFeature(EngineFeature.ALLOW_BROADCAST_RIGHTY_JOIN)) {
|
||||
&& !plannerContext.featureAvailable(EngineFeature.ALLOW_BROADCAST_RIGHTY_JOIN)) {
|
||||
class FindRightyJoin extends RelShuttleImpl
|
||||
{
|
||||
private Join found = null;
|
||||
|
|
|
@ -30,12 +30,16 @@ import org.apache.calcite.plan.RelTraitSet;
|
|||
import org.apache.calcite.rel.RelNode;
|
||||
import org.apache.calcite.rel.RelWriter;
|
||||
import org.apache.calcite.rel.core.Correlate;
|
||||
import org.apache.calcite.rel.core.Filter;
|
||||
import org.apache.calcite.rel.core.JoinRelType;
|
||||
import org.apache.calcite.rel.logical.LogicalProject;
|
||||
import org.apache.calcite.rel.core.CorrelationId;
|
||||
import org.apache.calcite.rel.metadata.RelMetadataQuery;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.calcite.rex.RexCall;
|
||||
import org.apache.calcite.rex.RexCorrelVariable;
|
||||
import org.apache.calcite.rex.RexFieldAccess;
|
||||
import org.apache.calcite.rex.RexInputRef;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.rex.RexShuttle;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.TableDataSource;
|
||||
|
@ -43,7 +47,8 @@ import org.apache.druid.query.UnnestDataSource;
|
|||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||
import org.apache.druid.sql.calcite.expression.Expressions;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.MultiValueStringToArrayOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.table.RowSignatures;
|
||||
|
||||
|
@ -57,28 +62,26 @@ import java.util.stream.Collectors;
|
|||
* This is the DruidRel to handle correlated rel nodes to be used for unnest.
|
||||
* Each correlate can be perceived as a join with the join type being inner
|
||||
* the left of a correlate as seen in the rule {@link org.apache.druid.sql.calcite.rule.DruidCorrelateUnnestRule}
|
||||
* is the {@link DruidQueryRel} while the right will always be an {@link DruidUnnestDatasourceRel}.
|
||||
* is the {@link DruidQueryRel} while the right will always be an {@link DruidUnnestRel}.
|
||||
*
|
||||
* Since this is a subclass of DruidRel it is automatically considered by other rules that involves DruidRels.
|
||||
* Some example being SELECT_PROJECT and SORT_PROJECT rules in {@link org.apache.druid.sql.calcite.rule.DruidRules.DruidQueryRule}
|
||||
*/
|
||||
public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
|
||||
{
|
||||
private static final TableDataSource DUMMY_DATA_SOURCE = new TableDataSource("unnest");
|
||||
private static final TableDataSource DUMMY_DATA_SOURCE = new TableDataSource("__correlate_unnest__");
|
||||
private static final String BASE_UNNEST_OUTPUT_COLUMN = "unnest";
|
||||
|
||||
private final Filter leftFilter;
|
||||
private final PartialDruidQuery partialQuery;
|
||||
private final PlannerConfig plannerConfig;
|
||||
private final Correlate correlateRel;
|
||||
private RelNode left;
|
||||
private RelNode right;
|
||||
private final RelNode left;
|
||||
private final RelNode right;
|
||||
private final PartialDruidQuery partialQuery;
|
||||
|
||||
private DruidCorrelateUnnestRel(
|
||||
RelOptCluster cluster,
|
||||
RelTraitSet traitSet,
|
||||
Correlate correlateRel,
|
||||
PartialDruidQuery partialQuery,
|
||||
Filter baseFilter,
|
||||
PlannerContext plannerContext
|
||||
)
|
||||
{
|
||||
|
@ -87,16 +90,13 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
|
|||
this.partialQuery = partialQuery;
|
||||
this.left = correlateRel.getLeft();
|
||||
this.right = correlateRel.getRight();
|
||||
this.leftFilter = baseFilter;
|
||||
this.plannerConfig = plannerContext.getPlannerConfig();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an instance from a Correlate that is based on a {@link DruidRel} and a {@link DruidUnnestDatasourceRel} inputs.
|
||||
* Create an instance from a Correlate that is based on a {@link DruidRel} and a {@link DruidUnnestRel} inputs.
|
||||
*/
|
||||
public static DruidCorrelateUnnestRel create(
|
||||
final Correlate correlateRel,
|
||||
final Filter leftFilter,
|
||||
final PlannerContext plannerContext
|
||||
)
|
||||
{
|
||||
|
@ -105,7 +105,6 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
|
|||
correlateRel.getTraitSet(),
|
||||
correlateRel,
|
||||
PartialDruidQuery.create(correlateRel),
|
||||
leftFilter,
|
||||
plannerContext
|
||||
);
|
||||
}
|
||||
|
@ -125,7 +124,6 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
|
|||
newQueryBuilder.getTraitSet(getConvention()),
|
||||
correlateRel,
|
||||
newQueryBuilder,
|
||||
leftFilter,
|
||||
getPlannerContext()
|
||||
);
|
||||
}
|
||||
|
@ -133,94 +131,58 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
|
|||
@Override
|
||||
public DruidQuery toDruidQuery(boolean finalizeAggregations)
|
||||
{
|
||||
final DruidRel<?> druidQueryRel = (DruidRel<?>) left;
|
||||
final DruidQuery leftQuery = Preconditions.checkNotNull((druidQueryRel).toDruidQuery(false), "leftQuery");
|
||||
final DruidRel<?> leftDruidRel = (DruidRel<?>) left;
|
||||
final DruidQuery leftQuery = Preconditions.checkNotNull(leftDruidRel.toDruidQuery(false), "leftQuery");
|
||||
final DruidUnnestRel unnestDatasourceRel = (DruidUnnestRel) right;
|
||||
final DataSource leftDataSource;
|
||||
final RowSignature leftDataSourceSignature;
|
||||
|
||||
if (DruidJoinQueryRel.computeLeftRequiresSubquery(getPlannerContext(), druidQueryRel)) {
|
||||
if (right.getRowType().getFieldNames().size() != 1) {
|
||||
throw new CannotBuildQueryException("Cannot perform correlated join + UNNEST with more than one column");
|
||||
}
|
||||
|
||||
if (computeLeftRequiresSubquery(leftDruidRel)) {
|
||||
// Left side is doing more than simple scan: generate a subquery.
|
||||
leftDataSource = new QueryDataSource(leftQuery.getQuery());
|
||||
leftDataSourceSignature = leftQuery.getOutputRowSignature();
|
||||
} else {
|
||||
leftDataSource = leftQuery.getDataSource();
|
||||
leftDataSourceSignature = DruidRels.dataSourceSignature(leftDruidRel);
|
||||
}
|
||||
|
||||
final DruidUnnestDatasourceRel unnestDatasourceRel = (DruidUnnestDatasourceRel) right;
|
||||
|
||||
|
||||
final RowSignature rowSignature = RowSignatures.fromRelDataType(
|
||||
correlateRel.getRowType().getFieldNames(),
|
||||
correlateRel.getRowType()
|
||||
);
|
||||
|
||||
final DruidExpression expression = Expressions.toDruidExpression(
|
||||
// Compute the expression to unnest.
|
||||
final RexNode rexNodeToUnnest = getRexNodeToUnnest(correlateRel, unnestDatasourceRel);
|
||||
final DruidExpression expressionToUnnest = Expressions.toDruidExpression(
|
||||
getPlannerContext(),
|
||||
rowSignature,
|
||||
unnestDatasourceRel.getUnnestProject().getProjects().get(0)
|
||||
leftDataSourceSignature,
|
||||
rexNodeToUnnest
|
||||
);
|
||||
|
||||
LogicalProject unnestProject = LogicalProject.create(
|
||||
this,
|
||||
ImmutableList.of(unnestDatasourceRel.getUnnestProject()
|
||||
.getProjects()
|
||||
.get(0)),
|
||||
unnestDatasourceRel.getUnnestProject().getRowType()
|
||||
);
|
||||
|
||||
// placeholder for dimension or expression to be unnested
|
||||
final String dimOrExpToUnnest;
|
||||
final VirtualColumnRegistry virtualColumnRegistry = VirtualColumnRegistry.create(
|
||||
rowSignature,
|
||||
getPlannerContext().getExprMacroTable(),
|
||||
getPlannerContext().getPlannerConfig().isForceExpressionVirtualColumns()
|
||||
);
|
||||
|
||||
// the unnest project is needed in case of a virtual column
|
||||
// unnest(mv_to_array(dim_1)) is reconciled as unnesting a MVD dim_1 not requiring a virtual column
|
||||
// while unnest(array(dim_2,dim_3)) is understood as unnesting a virtual column which is an array over dim_2 and dim_3 elements
|
||||
boolean unnestProjectNeeded = false;
|
||||
getPlannerContext().setJoinExpressionVirtualColumnRegistry(virtualColumnRegistry);
|
||||
|
||||
// handling for case when mv_to_array is used
|
||||
// No need to use virtual column in such a case
|
||||
if (StringUtils.toLowerCase(expression.getExpression()).startsWith("mv_to_array")) {
|
||||
dimOrExpToUnnest = expression.getArguments().get(0).getSimpleExtraction().getColumn();
|
||||
} else {
|
||||
if (expression.isDirectColumnAccess()) {
|
||||
dimOrExpToUnnest = expression.getDirectColumn();
|
||||
} else {
|
||||
// buckle up time to create virtual columns on expressions
|
||||
unnestProjectNeeded = true;
|
||||
dimOrExpToUnnest = virtualColumnRegistry.getOrCreateVirtualColumnForExpression(
|
||||
expression,
|
||||
expression.getDruidType()
|
||||
);
|
||||
}
|
||||
if (expressionToUnnest == null) {
|
||||
throw new CannotBuildQueryException(unnestDatasourceRel, unnestDatasourceRel.getInputRexNode());
|
||||
}
|
||||
|
||||
// add the unnest project to the partial query if required
|
||||
// This is necessary to handle the virtual columns on the unnestProject
|
||||
// Also create the unnest datasource to be used by the partial query
|
||||
PartialDruidQuery partialDruidQuery = unnestProjectNeeded ? partialQuery.withUnnest(unnestProject) : partialQuery;
|
||||
return partialDruidQuery.build(
|
||||
// Final output row signature.
|
||||
final RowSignature correlateRowSignature = getCorrelateRowSignature(correlateRel, leftQuery);
|
||||
|
||||
return partialQuery.build(
|
||||
UnnestDataSource.create(
|
||||
leftDataSource,
|
||||
dimOrExpToUnnest,
|
||||
unnestDatasourceRel.getUnnestProject().getRowType().getFieldNames().get(0),
|
||||
expressionToUnnest.toVirtualColumn(
|
||||
correlateRowSignature.getColumnName(correlateRowSignature.size() - 1),
|
||||
Calcites.getColumnTypeForRelDataType(rexNodeToUnnest.getType()),
|
||||
getPlannerContext().getExprMacroTable()
|
||||
),
|
||||
null
|
||||
),
|
||||
rowSignature,
|
||||
correlateRowSignature,
|
||||
getPlannerContext(),
|
||||
getCluster().getRexBuilder(),
|
||||
finalizeAggregations,
|
||||
virtualColumnRegistry
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DruidCorrelateUnnestRel clone()
|
||||
{
|
||||
return DruidCorrelateUnnestRel.create(correlateRel, leftFilter, getPlannerContext());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RelDataType deriveRowType()
|
||||
{
|
||||
|
@ -242,10 +204,6 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
|
|||
);
|
||||
}
|
||||
|
||||
// This is required to be overwritten as Calcite uses this method
|
||||
// to maintain a map of equivalent DruidCorrelateUnnestRel or in general any Rel nodes.
|
||||
// Without this method overwritten multiple RelNodes will produce the same key
|
||||
// which makes the planner plan incorrectly.
|
||||
@Override
|
||||
public RelWriter explainTerms(RelWriter pw)
|
||||
{
|
||||
|
@ -259,8 +217,9 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
|
|||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
return pw.item("query", queryString)
|
||||
.item("signature", druidQuery.getOutputRowSignature());
|
||||
return correlateRel.explainTerms(pw)
|
||||
.item("query", queryString)
|
||||
.item("signature", druidQuery.getOutputRowSignature());
|
||||
}
|
||||
|
||||
// This is called from the DruidRelToDruidRule which converts from the NONE convention to the DRUID convention
|
||||
|
@ -278,7 +237,6 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
|
|||
.collect(Collectors.toList())
|
||||
),
|
||||
partialQuery,
|
||||
leftFilter,
|
||||
getPlannerContext()
|
||||
);
|
||||
}
|
||||
|
@ -297,7 +255,6 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
|
|||
traitSet,
|
||||
correlateRel.copy(correlateRel.getTraitSet(), inputs),
|
||||
getPartialDruidQuery(),
|
||||
leftFilter,
|
||||
getPlannerContext()
|
||||
);
|
||||
}
|
||||
|
@ -305,15 +262,10 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
|
|||
@Override
|
||||
public RelOptCost computeSelfCost(final RelOptPlanner planner, final RelMetadataQuery mq)
|
||||
{
|
||||
double cost;
|
||||
double cost = partialQuery.estimateCost();
|
||||
|
||||
if (DruidJoinQueryRel.computeLeftRequiresSubquery(getPlannerContext(), DruidJoinQueryRel.getSomeDruidChild(left))) {
|
||||
cost = CostEstimates.COST_SUBQUERY;
|
||||
} else {
|
||||
cost = partialQuery.estimateCost();
|
||||
if (correlateRel.getJoinType() == JoinRelType.INNER && plannerConfig.isComputeInnerJoinCostAsFilter()) {
|
||||
cost *= CostEstimates.MULTIPLIER_FILTER;
|
||||
}
|
||||
if (computeLeftRequiresSubquery(DruidJoinQueryRel.getSomeDruidChild(left))) {
|
||||
cost += CostEstimates.COST_SUBQUERY;
|
||||
}
|
||||
|
||||
return planner.getCostFactory().makeCost(cost, 0, 0);
|
||||
|
@ -327,4 +279,110 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
|
|||
retVal.addAll(((DruidRel<?>) right).getDataSourceNames());
|
||||
return retVal;
|
||||
}
|
||||
|
||||
/**
|
||||
* Computes whether a particular left-side rel requires a subquery, or if we can operate on its underlying
|
||||
* datasource directly.
|
||||
*
|
||||
* Stricter than {@link DruidJoinQueryRel#computeLeftRequiresSubquery}: this method only allows scans (not mappings).
|
||||
* This is OK because any mapping or other simple projection would have been pulled above the {@link Correlate} by
|
||||
* {@link org.apache.druid.sql.calcite.rule.DruidCorrelateUnnestRule}.
|
||||
*/
|
||||
public static boolean computeLeftRequiresSubquery(final DruidRel<?> left)
|
||||
{
|
||||
return left == null || left.getPartialDruidQuery().stage() != PartialDruidQuery.Stage.SCAN;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether an expr is MV_TO_ARRAY of an input reference.
|
||||
*/
|
||||
private static boolean isMvToArrayOfInputRef(final RexNode expr)
|
||||
{
|
||||
return expr.isA(SqlKind.OTHER_FUNCTION)
|
||||
&& ((RexCall) expr).op.equals(MultiValueStringToArrayOperatorConversion.SQL_FUNCTION)
|
||||
&& ((RexCall) expr).getOperands().get(0).isA(SqlKind.INPUT_REF);
|
||||
}
|
||||
|
||||
/**
|
||||
* Unwrap MV_TO_ARRAY at the outer layer of an expr, if it refers to an input ref.
|
||||
*/
|
||||
private static RexNode unwrapMvToArray(final RexNode expr)
|
||||
{
|
||||
if (isMvToArrayOfInputRef(expr)) {
|
||||
return ((RexCall) expr).getOperands().get(0);
|
||||
} else {
|
||||
return expr;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Compute the row signature of this rel, given a particular left-hand {@link DruidQuery}.
|
||||
* The right-hand side is assumed to have a single column with the name {@link #BASE_UNNEST_OUTPUT_COLUMN}.
|
||||
*/
|
||||
private static RowSignature getCorrelateRowSignature(
|
||||
final Correlate correlate,
|
||||
final DruidQuery leftQuery
|
||||
)
|
||||
{
|
||||
// Compute signature of the correlation operation. It's like a join: the left and right sides are concatenated.
|
||||
// On the native query side, this is what is ultimately emitted by the UnnestStorageAdapter.
|
||||
//
|
||||
// Ignore prefix (lhs) from computeJoinRowSignature; we don't need this since we will declare the name of the
|
||||
// single output column directly. (And we know it's the last column in the signature.)
|
||||
final RelDataType unnestedType =
|
||||
correlate.getRowType().getFieldList().get(correlate.getRowType().getFieldCount() - 1).getType();
|
||||
|
||||
return DruidJoinQueryRel.computeJoinRowSignature(
|
||||
leftQuery.getOutputRowSignature(),
|
||||
RowSignature.builder().add(
|
||||
BASE_UNNEST_OUTPUT_COLUMN,
|
||||
Calcites.getColumnTypeForRelDataType(unnestedType)
|
||||
).build()
|
||||
).rhs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the expression to unnest from the left-hand side. Correlation variable references are rewritten to
|
||||
* regular field accesses, i.e., {@link RexInputRef}.
|
||||
*/
|
||||
private static RexNode getRexNodeToUnnest(
|
||||
final Correlate correlate,
|
||||
final DruidUnnestRel unnestDatasourceRel
|
||||
)
|
||||
{
|
||||
// Update unnestDatasourceRel.getUnnestProject() so it refers to the left-hand side rather than the correlation
|
||||
// variable. This is the expression to unnest.
|
||||
final RexNode rexNodeToUnnest =
|
||||
new CorrelatedFieldAccessToInputRef(correlate.getCorrelationId())
|
||||
.apply(unnestDatasourceRel.getInputRexNode());
|
||||
|
||||
// Unwrap MV_TO_ARRAY if present.
|
||||
return unwrapMvToArray(rexNodeToUnnest);
|
||||
}
|
||||
|
||||
/**
|
||||
* Shuttle that replaces correlating variables with regular field accesses to the left-hand side.
|
||||
*/
|
||||
private static class CorrelatedFieldAccessToInputRef extends RexShuttle
|
||||
{
|
||||
private final CorrelationId correlationId;
|
||||
|
||||
public CorrelatedFieldAccessToInputRef(final CorrelationId correlationId)
|
||||
{
|
||||
this.correlationId = correlationId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RexNode visitFieldAccess(final RexFieldAccess fieldAccess)
|
||||
{
|
||||
if (fieldAccess.getReferenceExpr() instanceof RexCorrelVariable) {
|
||||
final RexCorrelVariable encounteredCorrelationId = (RexCorrelVariable) fieldAccess.getReferenceExpr();
|
||||
if (encounteredCorrelationId.id.equals(correlationId)) {
|
||||
return new RexInputRef(fieldAccess.getField().getIndex(), fieldAccess.getType());
|
||||
}
|
||||
}
|
||||
|
||||
return super.visitFieldAccess(fieldAccess);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -186,10 +186,9 @@ public class DruidOuterQueryRel extends DruidRel<DruidOuterQueryRel>
|
|||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
return super.explainTerms(pw)
|
||||
.input("innerQuery", sourceRel)
|
||||
.item("query", queryString)
|
||||
.item("signature", druidQuery.getOutputRowSignature());
|
||||
return pw.input("innerQuery", sourceRel)
|
||||
.item("query", queryString)
|
||||
.item("signature", druidQuery.getOutputRowSignature());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -50,7 +50,6 @@ import org.apache.druid.java.util.common.granularity.Granularity;
|
|||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.JoinDataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContext;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
|
||||
|
@ -129,7 +128,6 @@ public class DruidQuery
|
|||
* Used by {@link #canUseQueryGranularity}.
|
||||
*/
|
||||
private static final int MAX_TIME_GRAINS_NON_DRUID_TABLE = 100000;
|
||||
public static final String CTX_ENABLE_WINDOW_FNS = "windowsAreForClosers";
|
||||
|
||||
private final DataSource dataSource;
|
||||
private final PlannerContext plannerContext;
|
||||
|
@ -140,9 +138,6 @@ public class DruidQuery
|
|||
@Nullable
|
||||
private final Projection selectProjection;
|
||||
|
||||
@Nullable
|
||||
private final Projection unnestProjection;
|
||||
|
||||
@Nullable
|
||||
private final Grouping grouping;
|
||||
|
||||
|
@ -163,7 +158,6 @@ public class DruidQuery
|
|||
final PlannerContext plannerContext,
|
||||
@Nullable final DimFilter filter,
|
||||
@Nullable final Projection selectProjection,
|
||||
@Nullable final Projection unnestProjection,
|
||||
@Nullable final Grouping grouping,
|
||||
@Nullable final Sorting sorting,
|
||||
@Nullable final Windowing windowing,
|
||||
|
@ -176,7 +170,6 @@ public class DruidQuery
|
|||
this.plannerContext = Preconditions.checkNotNull(plannerContext, "plannerContext");
|
||||
this.filter = filter;
|
||||
this.selectProjection = selectProjection;
|
||||
this.unnestProjection = unnestProjection;
|
||||
this.grouping = grouping;
|
||||
this.sorting = sorting;
|
||||
this.windowing = windowing;
|
||||
|
@ -279,8 +272,7 @@ public class DruidQuery
|
|||
}
|
||||
|
||||
if (partialQuery.getWindow() != null) {
|
||||
final QueryContext queryContext = plannerContext.queryContext();
|
||||
if (queryContext.getBoolean(CTX_ENABLE_WINDOW_FNS, false)) {
|
||||
if (plannerContext.featureAvailable(EngineFeature.WINDOW_FUNCTIONS)) {
|
||||
windowing = Preconditions.checkNotNull(
|
||||
Windowing.fromCalciteStuff(
|
||||
partialQuery,
|
||||
|
@ -290,32 +282,18 @@ public class DruidQuery
|
|||
)
|
||||
);
|
||||
} else {
|
||||
plannerContext.setPlanningError("Windowing Not Currently Supported");
|
||||
throw new CannotBuildQueryException("Windowing Not Currently Supported");
|
||||
plannerContext.setPlanningError("Windowing not supported");
|
||||
throw new CannotBuildQueryException("Windowing not supported");
|
||||
}
|
||||
} else {
|
||||
windowing = null;
|
||||
}
|
||||
|
||||
if (partialQuery.getUnnestProject() != null) {
|
||||
unnestProjection = Preconditions.checkNotNull(
|
||||
computeUnnestProjection(
|
||||
partialQuery,
|
||||
plannerContext,
|
||||
computeOutputRowSignature(sourceRowSignature, null, null, null, null),
|
||||
virtualColumnRegistry
|
||||
)
|
||||
);
|
||||
} else {
|
||||
unnestProjection = null;
|
||||
}
|
||||
|
||||
return new DruidQuery(
|
||||
dataSource,
|
||||
plannerContext,
|
||||
filter,
|
||||
selectProjection,
|
||||
unnestProjection,
|
||||
grouping,
|
||||
sorting,
|
||||
windowing,
|
||||
|
@ -392,18 +370,6 @@ public class DruidQuery
|
|||
}
|
||||
}
|
||||
|
||||
@Nonnull
|
||||
private static Projection computeUnnestProjection(
|
||||
final PartialDruidQuery partialQuery,
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final VirtualColumnRegistry virtualColumnRegistry
|
||||
)
|
||||
{
|
||||
final Project project = Preconditions.checkNotNull(partialQuery.getUnnestProject(), "unnestProject");
|
||||
return Projection.preAggregation(project, plannerContext, rowSignature, virtualColumnRegistry);
|
||||
}
|
||||
|
||||
@Nonnull
|
||||
private static Grouping computeGrouping(
|
||||
final PartialDruidQuery partialQuery,
|
||||
|
@ -801,16 +767,6 @@ public class DruidQuery
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
if (unnestProjection != null) {
|
||||
for (String columnName : unnestProjection.getVirtualColumns()) {
|
||||
if (virtualColumnRegistry.isVirtualColumnDefined(columnName)) {
|
||||
virtualColumns.add(virtualColumnRegistry.getVirtualColumn(columnName));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
for (String columnName : specialized) {
|
||||
if (virtualColumnRegistry.isVirtualColumnDefined(columnName)) {
|
||||
virtualColumns.add(virtualColumnRegistry.getVirtualColumn(columnName));
|
||||
|
@ -1006,7 +962,7 @@ public class DruidQuery
|
|||
@Nullable
|
||||
private TimeBoundaryQuery toTimeBoundaryQuery()
|
||||
{
|
||||
if (!plannerContext.engineHasFeature(EngineFeature.TIME_BOUNDARY_QUERY)
|
||||
if (!plannerContext.featureAvailable(EngineFeature.TIME_BOUNDARY_QUERY)
|
||||
|| grouping == null
|
||||
|| grouping.getSubtotals().hasEffect(grouping.getDimensionSpecs())
|
||||
|| grouping.getHavingFilter() != null
|
||||
|
@ -1072,7 +1028,7 @@ public class DruidQuery
|
|||
@Nullable
|
||||
private TimeseriesQuery toTimeseriesQuery()
|
||||
{
|
||||
if (!plannerContext.engineHasFeature(EngineFeature.TIMESERIES_QUERY)
|
||||
if (!plannerContext.featureAvailable(EngineFeature.TIMESERIES_QUERY)
|
||||
|| grouping == null
|
||||
|| grouping.getSubtotals().hasEffect(grouping.getDimensionSpecs())
|
||||
|| grouping.getHavingFilter() != null
|
||||
|
@ -1191,7 +1147,7 @@ public class DruidQuery
|
|||
private TopNQuery toTopNQuery()
|
||||
{
|
||||
// Must be allowed by the QueryMaker.
|
||||
if (!plannerContext.engineHasFeature(EngineFeature.TOPN_QUERY)) {
|
||||
if (!plannerContext.featureAvailable(EngineFeature.TOPN_QUERY)) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -1482,7 +1438,7 @@ public class DruidQuery
|
|||
orderByColumns = Collections.emptyList();
|
||||
}
|
||||
|
||||
if (!plannerContext.engineHasFeature(EngineFeature.SCAN_ORDER_BY_NON_TIME) && !orderByColumns.isEmpty()) {
|
||||
if (!plannerContext.featureAvailable(EngineFeature.SCAN_ORDER_BY_NON_TIME) && !orderByColumns.isEmpty()) {
|
||||
if (orderByColumns.size() > 1 || !ColumnHolder.TIME_COLUMN_NAME.equals(orderByColumns.get(0).getColumnName())) {
|
||||
// Cannot handle this ordering.
|
||||
// Scan cannot ORDER BY non-time columns.
|
||||
|
@ -1541,7 +1497,7 @@ public class DruidQuery
|
|||
final Map<String, Object> queryContext
|
||||
)
|
||||
{
|
||||
if (!plannerContext.engineHasFeature(EngineFeature.SCAN_NEEDS_SIGNATURE)) {
|
||||
if (!plannerContext.featureAvailable(EngineFeature.SCAN_NEEDS_SIGNATURE)) {
|
||||
return queryContext;
|
||||
}
|
||||
// Compute the signature of the columns that we are selecting.
|
||||
|
|
|
@ -27,14 +27,16 @@ import org.apache.calcite.plan.RelOptCost;
|
|||
import org.apache.calcite.plan.RelOptPlanner;
|
||||
import org.apache.calcite.plan.RelOptTable;
|
||||
import org.apache.calcite.plan.RelTraitSet;
|
||||
import org.apache.calcite.rel.RelNode;
|
||||
import org.apache.calcite.rel.RelWriter;
|
||||
import org.apache.calcite.rel.logical.LogicalTableScan;
|
||||
import org.apache.calcite.rel.logical.LogicalValues;
|
||||
import org.apache.calcite.rel.metadata.RelMetadataQuery;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.sql.calcite.external.ExternalTableScan;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.table.DruidTable;
|
||||
import org.apache.druid.sql.calcite.table.InlineTable;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Set;
|
||||
|
@ -105,19 +107,19 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
|
|||
/**
|
||||
* Create a DruidQueryRel representing a full scan of inline, literal values.
|
||||
*/
|
||||
public static DruidQueryRel scanValues(
|
||||
final LogicalValues valuesRel,
|
||||
final DruidTable druidTable,
|
||||
public static DruidQueryRel scanConstantRel(
|
||||
final RelNode rel,
|
||||
final InlineDataSource dataSource,
|
||||
final PlannerContext plannerContext
|
||||
)
|
||||
{
|
||||
return new DruidQueryRel(
|
||||
valuesRel.getCluster(),
|
||||
valuesRel.getTraitSet(), // the traitSet of valuesRel should be kept
|
||||
rel.getCluster(),
|
||||
rel.getTraitSet().replace(Convention.NONE), // keep traitSet of input rel, except for convention
|
||||
null,
|
||||
druidTable,
|
||||
new InlineTable(dataSource),
|
||||
plannerContext,
|
||||
PartialDruidQuery.create(valuesRel)
|
||||
PartialDruidQuery.create(rel)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.sql.calcite.rel;
|
|||
import org.apache.calcite.plan.RelOptCluster;
|
||||
import org.apache.calcite.plan.RelTraitSet;
|
||||
import org.apache.calcite.rel.AbstractRelNode;
|
||||
import org.apache.calcite.rel.RelWriter;
|
||||
import org.apache.druid.server.QueryResponse;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
|
||||
|
@ -95,6 +96,26 @@ public abstract class DruidRel<T extends DruidRel<?>> extends AbstractRelNode
|
|||
return plannerContext;
|
||||
}
|
||||
|
||||
/**
|
||||
* Overridden to ensure that subclasses provide a proper implementation. The default implementation from
|
||||
* {@link AbstractRelNode} does nothing and is not appropriate.
|
||||
*/
|
||||
@Override
|
||||
public RelWriter explainTerms(RelWriter pw)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Object clone() throws CloneNotSupportedException
|
||||
{
|
||||
// RelNode implements Cloneable, but our class of rels is not cloned, so does not need to implement clone().
|
||||
throw new CloneNotSupportedException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a copy of this rel with the {@link DruidConvention} trait.
|
||||
*/
|
||||
public abstract T asDruidConvention();
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.sql.calcite.rel;
|
||||
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.sql.calcite.table.DruidTable;
|
||||
|
||||
import java.util.Optional;
|
||||
|
@ -63,7 +64,7 @@ public class DruidRels
|
|||
* @param canBeJoinOrUnion consider a {@link DruidJoinQueryRel} or {@link DruidUnionDataSourceRel} as possible
|
||||
* scans-and-mappings too.
|
||||
*/
|
||||
private static boolean isScanOrProject(final DruidRel<?> druidRel, final boolean canBeJoinOrUnion)
|
||||
public static boolean isScanOrProject(final DruidRel<?> druidRel, final boolean canBeJoinOrUnion)
|
||||
{
|
||||
if (druidRel instanceof DruidQueryRel || (canBeJoinOrUnion && (druidRel instanceof DruidJoinQueryRel
|
||||
|| druidRel instanceof DruidUnionDataSourceRel))) {
|
||||
|
@ -75,4 +76,23 @@ public class DruidRels
|
|||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the signature of the datasource of a {@link DruidRel}.
|
||||
*
|
||||
* This is not the signature of the {@link DruidRel} itself: in particular, it ignores any operations that are layered
|
||||
* on top of the datasource.
|
||||
*/
|
||||
public static RowSignature dataSourceSignature(final DruidRel<?> druidRel)
|
||||
{
|
||||
if (druidRel instanceof DruidQueryRel) {
|
||||
// Get signature directly from the table.
|
||||
return ((DruidQueryRel) druidRel).getDruidTable().getRowSignature();
|
||||
} else {
|
||||
// Build the query with a no-op PartialDruidQuery.
|
||||
return druidRel.withPartialQuery(
|
||||
PartialDruidQuery.create(druidRel.getPartialDruidQuery().getScan())
|
||||
).toDruidQuery(false).getOutputRowSignature();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -221,8 +221,6 @@ public class DruidUnionRel extends DruidRel<DruidUnionRel>
|
|||
@Override
|
||||
public RelWriter explainTerms(RelWriter pw)
|
||||
{
|
||||
super.explainTerms(pw);
|
||||
|
||||
for (int i = 0; i < rels.size(); i++) {
|
||||
pw.input(StringUtils.format("input#%d", i), rels.get(i));
|
||||
}
|
||||
|
|
|
@ -1,180 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.rel;
|
||||
|
||||
import org.apache.calcite.rel.RelWriter;
|
||||
import org.apache.calcite.rel.core.Uncollect;
|
||||
import org.apache.calcite.rel.logical.LogicalProject;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.math.expr.InputBindings;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.UnnestDataSource;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
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.table.RowSignatures;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* The Rel node to capture the unnest (or uncollect) part in a query. This covers 2 cases:
|
||||
*
|
||||
* Case 1:
|
||||
* If this is an unnest on a constant and no input table is required, the final query is built using
|
||||
* an UnnestDataSource with a base InlineDataSource in this rel.
|
||||
*
|
||||
* Case 2:
|
||||
* If the unnest has an input table, this rel resolves the unnest part and delegates the rel to be consumed by other
|
||||
* rule ({@link org.apache.druid.sql.calcite.rule.DruidCorrelateUnnestRule}
|
||||
*/
|
||||
public class DruidUnnestDatasourceRel extends DruidRel<DruidUnnestDatasourceRel>
|
||||
{
|
||||
private final Uncollect uncollect;
|
||||
private final DruidQueryRel druidQueryRel;
|
||||
private final LogicalProject unnestProject;
|
||||
|
||||
public DruidUnnestDatasourceRel(
|
||||
Uncollect uncollect,
|
||||
DruidQueryRel queryRel,
|
||||
LogicalProject unnestProject,
|
||||
PlannerContext plannerContext
|
||||
)
|
||||
{
|
||||
super(uncollect.getCluster(), uncollect.getTraitSet(), plannerContext);
|
||||
this.uncollect = uncollect;
|
||||
this.druidQueryRel = queryRel;
|
||||
this.unnestProject = unnestProject;
|
||||
}
|
||||
|
||||
public LogicalProject getUnnestProject()
|
||||
{
|
||||
return unnestProject;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public PartialDruidQuery getPartialDruidQuery()
|
||||
{
|
||||
return druidQueryRel.getPartialDruidQuery();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidUnnestDatasourceRel withPartialQuery(PartialDruidQuery newQueryBuilder)
|
||||
{
|
||||
return new DruidUnnestDatasourceRel(
|
||||
uncollect,
|
||||
druidQueryRel.withPartialQuery(newQueryBuilder),
|
||||
unnestProject,
|
||||
getPlannerContext()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidQuery toDruidQuery(boolean finalizeAggregations)
|
||||
{
|
||||
VirtualColumnRegistry virtualColumnRegistry = VirtualColumnRegistry.create(
|
||||
druidQueryRel.getDruidTable().getRowSignature(),
|
||||
getPlannerContext().getExprMacroTable(),
|
||||
getPlannerContext().getPlannerConfig().isForceExpressionVirtualColumns()
|
||||
);
|
||||
getPlannerContext().setJoinExpressionVirtualColumnRegistry(virtualColumnRegistry);
|
||||
|
||||
final DruidExpression expression = Expressions.toDruidExpression(
|
||||
getPlannerContext(),
|
||||
druidQueryRel.getDruidTable().getRowSignature(),
|
||||
unnestProject.getProjects().get(0)
|
||||
);
|
||||
if (expression == null) {
|
||||
return null;
|
||||
}
|
||||
Expr parsed = expression.parse(getPlannerContext().getExprMacroTable());
|
||||
ExprEval eval = parsed.eval(InputBindings.nilBindings());
|
||||
|
||||
// If query unnests a constant expression and not use any table
|
||||
// the unnest would be on an inline data source
|
||||
// with the input column being called "inline" in the native query
|
||||
UnnestDataSource dataSource = UnnestDataSource.create(
|
||||
InlineDataSource.fromIterable(
|
||||
Collections.singletonList(new Object[]{eval.valueOrDefault()}),
|
||||
RowSignature.builder().add("inline", ExpressionType.toColumnType(eval.type())).build()
|
||||
),
|
||||
"inline",
|
||||
druidQueryRel.getRowType().getFieldNames().get(0),
|
||||
null
|
||||
);
|
||||
|
||||
DruidQuery query = druidQueryRel.getPartialDruidQuery().build(
|
||||
dataSource,
|
||||
RowSignatures.fromRelDataType(uncollect.getRowType().getFieldNames(), uncollect.getRowType()),
|
||||
getPlannerContext(),
|
||||
getCluster().getRexBuilder(),
|
||||
finalizeAggregations
|
||||
);
|
||||
getPlannerContext().setJoinExpressionVirtualColumnRegistry(null);
|
||||
return query;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidQuery toDruidQueryForExplaining()
|
||||
{
|
||||
return toDruidQuery(false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidUnnestDatasourceRel asDruidConvention()
|
||||
{
|
||||
return new DruidUnnestDatasourceRel(
|
||||
new Uncollect(getCluster(), traitSet.replace(DruidConvention.instance()), uncollect.getInput(), false),
|
||||
druidQueryRel.asDruidConvention(),
|
||||
unnestProject,
|
||||
getPlannerContext()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RelWriter explainTerms(RelWriter pw)
|
||||
{
|
||||
return super.explainTerms(pw);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getDataSourceNames()
|
||||
{
|
||||
return druidQueryRel.getDruidTable().getDataSource().getTableNames();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RelDataType deriveRowType()
|
||||
{
|
||||
return uncollect.getRowType();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DruidUnnestDatasourceRel clone()
|
||||
{
|
||||
return new DruidUnnestDatasourceRel(uncollect, druidQueryRel, unnestProject, getPlannerContext());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,191 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.rel;
|
||||
|
||||
import org.apache.calcite.plan.RelOptCluster;
|
||||
import org.apache.calcite.plan.RelOptUtil;
|
||||
import org.apache.calcite.plan.RelTraitSet;
|
||||
import org.apache.calcite.rel.RelNode;
|
||||
import org.apache.calcite.rel.RelWriter;
|
||||
import org.apache.calcite.rel.core.Uncollect;
|
||||
import org.apache.calcite.rel.logical.LogicalProject;
|
||||
import org.apache.calcite.rel.logical.LogicalValues;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.rex.RexShuttle;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Captures an unnest expression for a correlated join. Derived from an {@link Uncollect}.
|
||||
*
|
||||
* This rel cannot be executed directly. It is a holder of information for {@link DruidCorrelateUnnestRel}.
|
||||
*
|
||||
* Unnest on literal values, without correlated join, is handled directly by
|
||||
* {@link org.apache.druid.sql.calcite.rule.DruidUnnestRule}. is applied without a correlated join, This covers the case where an unnest has an
|
||||
* input table, this rel resolves the unnest part and delegates the rel to be consumed by other
|
||||
* rule ({@link org.apache.druid.sql.calcite.rule.DruidCorrelateUnnestRule}
|
||||
*/
|
||||
public class DruidUnnestRel extends DruidRel<DruidUnnestRel>
|
||||
{
|
||||
private static final String FIELD_NAME = "UNNEST";
|
||||
|
||||
/**
|
||||
* Expression to be unnested. May be constant or may reference a correlation variable through a
|
||||
* {@link org.apache.calcite.rex.RexFieldAccess}.
|
||||
*/
|
||||
private final RexNode inputRexNode;
|
||||
|
||||
private DruidUnnestRel(
|
||||
final RelOptCluster cluster,
|
||||
final RelTraitSet traits,
|
||||
final RexNode inputRexNode,
|
||||
final PlannerContext plannerContext
|
||||
)
|
||||
{
|
||||
super(cluster, traits, plannerContext);
|
||||
this.inputRexNode = inputRexNode;
|
||||
}
|
||||
|
||||
public static DruidUnnestRel create(
|
||||
final RelOptCluster cluster,
|
||||
final RelTraitSet traits,
|
||||
final RexNode unnestRexNode,
|
||||
final PlannerContext plannerContext
|
||||
)
|
||||
{
|
||||
if (!RelOptUtil.InputFinder.bits(unnestRexNode).isEmpty()) {
|
||||
throw new ISE("Expression must not include field references");
|
||||
}
|
||||
|
||||
return new DruidUnnestRel(
|
||||
cluster,
|
||||
traits,
|
||||
unnestRexNode,
|
||||
plannerContext
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("ObjectEquality")
|
||||
public RelNode accept(RexShuttle shuttle)
|
||||
{
|
||||
final RexNode newInputRexNode = shuttle.apply(inputRexNode);
|
||||
|
||||
if (newInputRexNode == inputRexNode) {
|
||||
return this;
|
||||
} else {
|
||||
return new DruidUnnestRel(
|
||||
getCluster(),
|
||||
getTraitSet(),
|
||||
newInputRexNode,
|
||||
getPlannerContext()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Expression to be unnested.
|
||||
*/
|
||||
public RexNode getInputRexNode()
|
||||
{
|
||||
return inputRexNode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PartialDruidQuery getPartialDruidQuery()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidUnnestRel withPartialQuery(PartialDruidQuery newQueryBuilder)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a new rel with a new input. The output type is unchanged.
|
||||
*/
|
||||
public DruidUnnestRel withUnnestRexNode(final RexNode newInputRexNode)
|
||||
{
|
||||
return new DruidUnnestRel(
|
||||
getCluster(),
|
||||
getTraitSet(),
|
||||
newInputRexNode,
|
||||
getPlannerContext()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidQuery toDruidQuery(boolean finalizeAggregations)
|
||||
{
|
||||
// DruidUnnestRel is a holder for info for DruidCorrelateUnnestRel. It cannot be executed on its own.
|
||||
throw new CannotBuildQueryException("Cannot execute UNNEST directly");
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidQuery toDruidQueryForExplaining()
|
||||
{
|
||||
// DruidUnnestRel is a holder for info for DruidCorrelateUnnestRel. It cannot be executed on its own.
|
||||
throw new CannotBuildQueryException("Cannot execute UNNEST directly");
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public DruidUnnestRel asDruidConvention()
|
||||
{
|
||||
return new DruidUnnestRel(
|
||||
getCluster(),
|
||||
getTraitSet().replace(DruidConvention.instance()),
|
||||
inputRexNode,
|
||||
getPlannerContext()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RelWriter explainTerms(RelWriter pw)
|
||||
{
|
||||
return pw.item("expr", inputRexNode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getDataSourceNames()
|
||||
{
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RelDataType deriveRowType()
|
||||
{
|
||||
return Uncollect.deriveUncollectRowType(
|
||||
LogicalProject.create(
|
||||
LogicalValues.createOneRow(getCluster()),
|
||||
Collections.singletonList(inputRexNode),
|
||||
Collections.singletonList(FIELD_NAME)
|
||||
),
|
||||
false
|
||||
);
|
||||
}
|
||||
}
|
|
@ -63,7 +63,6 @@ public class PartialDruidQuery
|
|||
private final RelNode scan;
|
||||
private final Filter whereFilter;
|
||||
private final Project selectProject;
|
||||
private final Project unnestProject;
|
||||
private final Aggregate aggregate;
|
||||
private final Filter havingFilter;
|
||||
private final Project aggregateProject;
|
||||
|
@ -141,8 +140,7 @@ public class PartialDruidQuery
|
|||
final Sort sort,
|
||||
final Project sortProject,
|
||||
final Window window,
|
||||
final Project windowProject,
|
||||
final Project unnestProject
|
||||
final Project windowProject
|
||||
)
|
||||
{
|
||||
this.builderSupplier = Preconditions.checkNotNull(builderSupplier, "builderSupplier");
|
||||
|
@ -156,7 +154,6 @@ public class PartialDruidQuery
|
|||
this.sortProject = sortProject;
|
||||
this.window = window;
|
||||
this.windowProject = windowProject;
|
||||
this.unnestProject = unnestProject;
|
||||
}
|
||||
|
||||
public static PartialDruidQuery create(final RelNode inputRel)
|
||||
|
@ -165,7 +162,7 @@ public class PartialDruidQuery
|
|||
inputRel.getCluster(),
|
||||
inputRel.getTable() != null ? inputRel.getTable().getRelOptSchema() : null
|
||||
);
|
||||
return new PartialDruidQuery(builderSupplier, inputRel, null, null, null, null, null, null, null, null, null, null);
|
||||
return new PartialDruidQuery(builderSupplier, inputRel, null, null, null, null, null, null, null, null, null);
|
||||
}
|
||||
|
||||
public static PartialDruidQuery createOuterQuery(final PartialDruidQuery inputQuery)
|
||||
|
@ -194,11 +191,6 @@ public class PartialDruidQuery
|
|||
return selectProject;
|
||||
}
|
||||
|
||||
public Project getUnnestProject()
|
||||
{
|
||||
return unnestProject;
|
||||
}
|
||||
|
||||
public Aggregate getAggregate()
|
||||
{
|
||||
return aggregate;
|
||||
|
@ -248,8 +240,7 @@ public class PartialDruidQuery
|
|||
sort,
|
||||
sortProject,
|
||||
window,
|
||||
windowProject,
|
||||
unnestProject
|
||||
windowProject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -293,8 +284,7 @@ public class PartialDruidQuery
|
|||
sort,
|
||||
sortProject,
|
||||
window,
|
||||
windowProject,
|
||||
unnestProject
|
||||
windowProject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -312,8 +302,7 @@ public class PartialDruidQuery
|
|||
sort,
|
||||
sortProject,
|
||||
window,
|
||||
windowProject,
|
||||
unnestProject
|
||||
windowProject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -331,8 +320,7 @@ public class PartialDruidQuery
|
|||
sort,
|
||||
sortProject,
|
||||
window,
|
||||
windowProject,
|
||||
unnestProject
|
||||
windowProject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -350,8 +338,7 @@ public class PartialDruidQuery
|
|||
sort,
|
||||
sortProject,
|
||||
window,
|
||||
windowProject,
|
||||
unnestProject
|
||||
windowProject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -369,8 +356,7 @@ public class PartialDruidQuery
|
|||
newSort,
|
||||
sortProject,
|
||||
window,
|
||||
windowProject,
|
||||
unnestProject
|
||||
windowProject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -388,8 +374,7 @@ public class PartialDruidQuery
|
|||
sort,
|
||||
newSortProject,
|
||||
window,
|
||||
windowProject,
|
||||
unnestProject
|
||||
windowProject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -407,26 +392,7 @@ public class PartialDruidQuery
|
|||
sort,
|
||||
sortProject,
|
||||
newWindow,
|
||||
windowProject,
|
||||
unnestProject
|
||||
);
|
||||
}
|
||||
|
||||
public PartialDruidQuery withUnnest(final Project newUnnestProject)
|
||||
{
|
||||
return new PartialDruidQuery(
|
||||
builderSupplier,
|
||||
scan,
|
||||
whereFilter,
|
||||
selectProject,
|
||||
aggregate,
|
||||
aggregateProject,
|
||||
havingFilter,
|
||||
sort,
|
||||
sortProject,
|
||||
window,
|
||||
windowProject,
|
||||
newUnnestProject
|
||||
windowProject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -444,8 +410,7 @@ public class PartialDruidQuery
|
|||
sort,
|
||||
sortProject,
|
||||
window,
|
||||
newWindowProject,
|
||||
unnestProject
|
||||
newWindowProject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -715,8 +680,7 @@ public class PartialDruidQuery
|
|||
&& Objects.equals(sort, that.sort)
|
||||
&& Objects.equals(sortProject, that.sortProject)
|
||||
&& Objects.equals(window, that.window)
|
||||
&& Objects.equals(windowProject, that.windowProject)
|
||||
&& Objects.equals(unnestProject, that.unnestProject);
|
||||
&& Objects.equals(windowProject, that.windowProject);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -732,8 +696,7 @@ public class PartialDruidQuery
|
|||
sort,
|
||||
sortProject,
|
||||
window,
|
||||
windowProject,
|
||||
unnestProject
|
||||
windowProject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -751,7 +714,6 @@ public class PartialDruidQuery
|
|||
", sortProject=" + sortProject +
|
||||
", window=" + window +
|
||||
", windowProject=" + windowProject +
|
||||
", unnestProject=" + unnestProject +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.rule;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.calcite.plan.RelOptRule;
|
||||
import org.apache.calcite.plan.RelOptRuleCall;
|
||||
import org.apache.calcite.rel.RelNode;
|
||||
import org.apache.calcite.rel.core.Correlate;
|
||||
import org.apache.calcite.rel.core.Filter;
|
||||
|
||||
/**
|
||||
* Rule that pulls a {@link Filter} from the left-hand side of a {@link Correlate} above the Correlate.
|
||||
* Allows subquery elimination.
|
||||
*
|
||||
* @see CorrelateFilterRTransposeRule similar, but for right-hand side filters
|
||||
*/
|
||||
public class CorrelateFilterLTransposeRule extends RelOptRule
|
||||
{
|
||||
private static final CorrelateFilterLTransposeRule INSTANCE = new CorrelateFilterLTransposeRule();
|
||||
|
||||
public CorrelateFilterLTransposeRule()
|
||||
{
|
||||
super(
|
||||
operand(
|
||||
Correlate.class,
|
||||
operand(Filter.class, any()),
|
||||
operand(RelNode.class, any())
|
||||
));
|
||||
}
|
||||
|
||||
public static CorrelateFilterLTransposeRule instance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onMatch(final RelOptRuleCall call)
|
||||
{
|
||||
final Correlate correlate = call.rel(0);
|
||||
final Filter left = call.rel(1);
|
||||
final RelNode right = call.rel(2);
|
||||
|
||||
call.transformTo(
|
||||
call.builder()
|
||||
.push(correlate.copy(correlate.getTraitSet(), ImmutableList.of(left.getInput(), right)))
|
||||
.filter(left.getCondition())
|
||||
.build()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,112 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.rule;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.calcite.plan.RelOptRule;
|
||||
import org.apache.calcite.plan.RelOptRuleCall;
|
||||
import org.apache.calcite.rel.RelNode;
|
||||
import org.apache.calcite.rel.core.Correlate;
|
||||
import org.apache.calcite.rel.core.CorrelationId;
|
||||
import org.apache.calcite.rel.core.Filter;
|
||||
import org.apache.calcite.rex.RexCorrelVariable;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.rex.RexUtil;
|
||||
import org.apache.calcite.rex.RexVisitorImpl;
|
||||
|
||||
/**
|
||||
* Rule that pulls a {@link Filter} from the right-hand side of a {@link Correlate} above the Correlate.
|
||||
* Allows filters on unnested fields to be added to queries that use {@link org.apache.druid.query.UnnestDataSource}.
|
||||
*
|
||||
* @see CorrelateFilterLTransposeRule similar, but for left-hand side filters
|
||||
*/
|
||||
public class CorrelateFilterRTransposeRule extends RelOptRule
|
||||
{
|
||||
private static final CorrelateFilterRTransposeRule INSTANCE = new CorrelateFilterRTransposeRule();
|
||||
|
||||
public CorrelateFilterRTransposeRule()
|
||||
{
|
||||
super(
|
||||
operand(
|
||||
Correlate.class,
|
||||
operand(RelNode.class, any()),
|
||||
operand(Filter.class, any())
|
||||
));
|
||||
}
|
||||
|
||||
public static CorrelateFilterRTransposeRule instance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean matches(RelOptRuleCall call)
|
||||
{
|
||||
final Correlate correlate = call.rel(0);
|
||||
final Filter right = call.rel(2);
|
||||
|
||||
// Can't pull up filters that explicitly refer to the correlation variable.
|
||||
return !usesCorrelationId(correlate.getCorrelationId(), right.getCondition());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onMatch(final RelOptRuleCall call)
|
||||
{
|
||||
final Correlate correlate = call.rel(0);
|
||||
final RelNode left = call.rel(1);
|
||||
final Filter right = call.rel(2);
|
||||
|
||||
call.transformTo(
|
||||
call.builder()
|
||||
.push(correlate.copy(correlate.getTraitSet(), ImmutableList.of(left, right.getInput())))
|
||||
.filter(RexUtil.shift(right.getCondition(), left.getRowType().getFieldCount()))
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether an expression refers to correlation variables.
|
||||
*/
|
||||
private static boolean usesCorrelationId(final CorrelationId correlationId, final RexNode rexNode)
|
||||
{
|
||||
class CorrelationVisitor extends RexVisitorImpl<Void>
|
||||
{
|
||||
private boolean found = false;
|
||||
|
||||
public CorrelationVisitor()
|
||||
{
|
||||
super(true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Void visitCorrelVariable(RexCorrelVariable correlVariable)
|
||||
{
|
||||
if (correlVariable.id.equals(correlationId)) {
|
||||
found = true;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
final CorrelationVisitor visitor = new CorrelationVisitor();
|
||||
rexNode.accept(visitor);
|
||||
return visitor.found;
|
||||
}
|
||||
}
|
|
@ -19,22 +19,29 @@
|
|||
|
||||
package org.apache.druid.sql.calcite.rule;
|
||||
|
||||
import it.unimi.dsi.fastutil.ints.IntAVLTreeSet;
|
||||
import it.unimi.dsi.fastutil.ints.IntSet;
|
||||
import org.apache.calcite.plan.RelOptRule;
|
||||
import org.apache.calcite.plan.RelOptRuleCall;
|
||||
import org.apache.calcite.plan.RelOptUtil;
|
||||
import org.apache.calcite.rel.RelNode;
|
||||
import org.apache.calcite.rel.core.Correlate;
|
||||
import org.apache.calcite.rel.core.Filter;
|
||||
import org.apache.calcite.rel.core.CorrelationId;
|
||||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexCorrelVariable;
|
||||
import org.apache.calcite.rex.RexFieldAccess;
|
||||
import org.apache.calcite.rex.RexInputRef;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.rex.RexShuttle;
|
||||
import org.apache.calcite.rex.RexUtil;
|
||||
import org.apache.calcite.tools.RelBuilder;
|
||||
import org.apache.calcite.util.ImmutableBitSet;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.rel.DruidCorrelateUnnestRel;
|
||||
import org.apache.druid.sql.calcite.rel.DruidQueryRel;
|
||||
import org.apache.druid.sql.calcite.rel.DruidRel;
|
||||
import org.apache.druid.sql.calcite.rel.DruidUnnestDatasourceRel;
|
||||
import org.apache.druid.sql.calcite.rel.DruidRels;
|
||||
import org.apache.druid.sql.calcite.rel.DruidUnnestRel;
|
||||
import org.apache.druid.sql.calcite.rel.PartialDruidQuery;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -44,26 +51,28 @@ import java.util.List;
|
|||
* This class creates the rule to abide by for creating correlations during unnest.
|
||||
* Typically, Calcite plans the unnest query such as
|
||||
* SELECT * from numFoo, unnest(dim3) in the following way:
|
||||
*
|
||||
* <pre>
|
||||
* 80:LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{3}])
|
||||
* 6:LogicalTableScan(subset=[rel#74:Subset#0.NONE.[]], table=[[druid, numfoo]])
|
||||
* 78:Uncollect(subset=[rel#79:Subset#3.NONE.[]])
|
||||
* 76:LogicalProject(subset=[rel#77:Subset#2.NONE.[]], EXPR$0=[MV_TO_ARRAY($cor0.dim3)])
|
||||
* 7:LogicalValues(subset=[rel#75:Subset#1.NONE.[0]], tuples=[[{ 0 }]])
|
||||
* </pre>
|
||||
*
|
||||
* {@link DruidUnnestDatasourceRule} takes care of the Uncollect(last 3 lines) to generate a {@link DruidUnnestDatasourceRel}
|
||||
* thereby reducing the logical plan to:
|
||||
* {@link DruidUnnestRule} takes care of the Uncollect(last 3 lines) to generate a {@link DruidUnnestRel}
|
||||
* thereby reducing the logical plan to:
|
||||
* <pre>
|
||||
* LogicalCorrelate
|
||||
* / \
|
||||
* DruidRel DruidUnnestDataSourceRel
|
||||
*
|
||||
* This forms the premise of this rule. The goal is to transform the above-mentioned structure in the tree
|
||||
* with a new rel {@link DruidCorrelateUnnestRel} which shall be created here.
|
||||
*
|
||||
* </pre>
|
||||
* This forms the premise of this rule. The goal is to transform the above-mentioned structure in the tree
|
||||
* with a new rel {@link DruidCorrelateUnnestRel} which shall be created here.
|
||||
*/
|
||||
public class DruidCorrelateUnnestRule extends RelOptRule
|
||||
{
|
||||
private final PlannerContext plannerContext;
|
||||
private final boolean enableLeftScanDirect;
|
||||
|
||||
public DruidCorrelateUnnestRule(final PlannerContext plannerContext)
|
||||
{
|
||||
|
@ -71,107 +80,135 @@ public class DruidCorrelateUnnestRule extends RelOptRule
|
|||
operand(
|
||||
Correlate.class,
|
||||
operand(DruidRel.class, any()),
|
||||
operand(DruidUnnestDatasourceRel.class, any())
|
||||
operand(DruidUnnestRel.class, any())
|
||||
)
|
||||
);
|
||||
|
||||
this.plannerContext = plannerContext;
|
||||
this.enableLeftScanDirect = plannerContext.queryContext().getEnableJoinLeftScanDirect();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean matches(RelOptRuleCall call)
|
||||
{
|
||||
final DruidRel<?> druidRel = call.rel(1);
|
||||
final DruidRel<?> uncollectRel = call.rel(2);
|
||||
|
||||
return druidRel.getPartialDruidQuery() != null
|
||||
&& uncollectRel.getPartialDruidQuery() != null;
|
||||
final DruidRel<?> left = call.rel(1);
|
||||
return left.getPartialDruidQuery() != null;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void onMatch(RelOptRuleCall call)
|
||||
{
|
||||
final Correlate correlate = call.rel(0);
|
||||
final DruidRel<?> druidRel = call.rel(1);
|
||||
DruidUnnestDatasourceRel druidUnnestDatasourceRel = call.rel(2);
|
||||
final DruidRel<?> left = call.rel(1);
|
||||
final DruidUnnestRel right = call.rel(2);
|
||||
|
||||
if (DruidRels.isScanOrProject(left, true)
|
||||
&& left.getPartialDruidQuery().getSelectProject() != null
|
||||
&& RelOptUtil.InputFinder.bits(right.getInputRexNode()).isEmpty()) {
|
||||
// Pull left-side Project above the Correlate, so we can eliminate a subquery.
|
||||
final RelNode leftScan = left.getPartialDruidQuery().getScan();
|
||||
final Project leftProject = left.getPartialDruidQuery().getSelectProject();
|
||||
|
||||
final RexBuilder rexBuilder = correlate.getCluster().getRexBuilder();
|
||||
// Rewrite right-side expression on top of leftScan rather than leftProject.
|
||||
final CorrelationId newCorrelationId = correlate.getCluster().createCorrel();
|
||||
final PushCorrelatedFieldAccessPastProject correlatedFieldRewriteShuttle =
|
||||
new PushCorrelatedFieldAccessPastProject(correlate.getCorrelationId(), newCorrelationId, leftProject);
|
||||
final RexNode newUnnestRexNode = correlatedFieldRewriteShuttle.apply(right.getInputRexNode());
|
||||
|
||||
final Filter druidRelFilter;
|
||||
final DruidRel<?> newDruidRelFilter;
|
||||
final List<RexNode> newProjectExprs = new ArrayList<>();
|
||||
// Build the new Correlate rel and a DruidCorrelateUnnestRel wrapper.
|
||||
final DruidCorrelateUnnestRel druidCorrelateUnnest = DruidCorrelateUnnestRel.create(
|
||||
correlate.copy(
|
||||
correlate.getTraitSet(),
|
||||
|
||||
final boolean isLeftDirectAccessPossible = enableLeftScanDirect && (druidRel instanceof DruidQueryRel);
|
||||
// Left side: remove Project.
|
||||
left.withPartialQuery(PartialDruidQuery.create(leftScan)),
|
||||
|
||||
if (druidRel.getPartialDruidQuery().stage() == PartialDruidQuery.Stage.SELECT_PROJECT
|
||||
&& (isLeftDirectAccessPossible || druidRel.getPartialDruidQuery().getWhereFilter() == null)) {
|
||||
// Swap the druidRel-side projection above the correlate, so the druidRel side is a simple scan or mapping.
|
||||
// This helps us avoid subqueries.
|
||||
final RelNode leftScan = druidRel.getPartialDruidQuery().getScan();
|
||||
final Project leftProject = druidRel.getPartialDruidQuery().getSelectProject();
|
||||
druidRelFilter = druidRel.getPartialDruidQuery().getWhereFilter();
|
||||
// Right side: use rewritten newUnnestRexNode, pushed past the left Project.
|
||||
right.withUnnestRexNode(newUnnestRexNode),
|
||||
newCorrelationId,
|
||||
ImmutableBitSet.of(correlatedFieldRewriteShuttle.getRequiredColumns()),
|
||||
correlate.getJoinType()
|
||||
),
|
||||
plannerContext
|
||||
);
|
||||
|
||||
// Left-side projection expressions rewritten to be on top of the correlate.
|
||||
newProjectExprs.addAll(leftProject.getProjects());
|
||||
newDruidRelFilter = druidRel.withPartialQuery(PartialDruidQuery.create(leftScan));
|
||||
} else {
|
||||
// Leave druidRel as-is. Write input refs that do nothing.
|
||||
for (int i = 0; i < druidRel.getRowType().getFieldCount(); i++) {
|
||||
newProjectExprs.add(rexBuilder.makeInputRef(correlate.getRowType().getFieldList().get(i).getType(), i));
|
||||
// Add right-side input refs to the Project, so it matches the full original Correlate.
|
||||
final RexBuilder rexBuilder = correlate.getCluster().getRexBuilder();
|
||||
final List<RexNode> pulledUpProjects = new ArrayList<>(leftProject.getProjects());
|
||||
for (int i = 0; i < right.getRowType().getFieldCount(); i++) {
|
||||
pulledUpProjects.add(rexBuilder.makeInputRef(druidCorrelateUnnest, i + leftScan.getRowType().getFieldCount()));
|
||||
}
|
||||
newDruidRelFilter = druidRel;
|
||||
druidRelFilter = null;
|
||||
|
||||
// Now push the Project back on top of the Correlate.
|
||||
final RelBuilder relBuilder =
|
||||
call.builder()
|
||||
.push(druidCorrelateUnnest)
|
||||
.project(
|
||||
RexUtil.fixUp(
|
||||
rexBuilder,
|
||||
pulledUpProjects,
|
||||
RelOptUtil.getFieldTypeList(druidCorrelateUnnest.getRowType())
|
||||
)
|
||||
);
|
||||
|
||||
final RelNode build = relBuilder.build();
|
||||
call.transformTo(build);
|
||||
} else {
|
||||
call.transformTo(DruidCorrelateUnnestRel.create(correlate, plannerContext));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Shuttle that pushes correlating variable accesses past a Project.
|
||||
*/
|
||||
private static class PushCorrelatedFieldAccessPastProject extends RexShuttle
|
||||
{
|
||||
private final CorrelationId correlationId;
|
||||
private final CorrelationId newCorrelationId;
|
||||
private final Project project;
|
||||
|
||||
// "Sidecar" return value: computed along with the shuttling.
|
||||
private final IntSet requiredColumns = new IntAVLTreeSet();
|
||||
|
||||
public PushCorrelatedFieldAccessPastProject(
|
||||
final CorrelationId correlationId,
|
||||
final CorrelationId newCorrelationId,
|
||||
final Project project
|
||||
)
|
||||
{
|
||||
this.correlationId = correlationId;
|
||||
this.newCorrelationId = newCorrelationId;
|
||||
this.project = project;
|
||||
}
|
||||
|
||||
if (druidUnnestDatasourceRel.getPartialDruidQuery().stage() == PartialDruidQuery.Stage.SELECT_PROJECT) {
|
||||
for (final RexNode rexNode : RexUtil.shift(
|
||||
druidUnnestDatasourceRel.getPartialDruidQuery()
|
||||
.getSelectProject()
|
||||
.getProjects(),
|
||||
newDruidRelFilter.getRowType().getFieldCount()
|
||||
)) {
|
||||
newProjectExprs.add(rexNode);
|
||||
}
|
||||
} else {
|
||||
for (int i = 0; i < druidUnnestDatasourceRel.getRowType().getFieldCount(); i++) {
|
||||
newProjectExprs.add(
|
||||
rexBuilder.makeInputRef(
|
||||
correlate.getRowType()
|
||||
.getFieldList()
|
||||
.get(druidRel.getRowType().getFieldCount() + i)
|
||||
.getType(),
|
||||
newDruidRelFilter.getRowType().getFieldCount() + i
|
||||
)
|
||||
);
|
||||
}
|
||||
public IntSet getRequiredColumns()
|
||||
{
|
||||
return requiredColumns;
|
||||
}
|
||||
|
||||
final DruidCorrelateUnnestRel druidCorr = DruidCorrelateUnnestRel.create(
|
||||
correlate.copy(
|
||||
correlate.getTraitSet(),
|
||||
newDruidRelFilter,
|
||||
druidUnnestDatasourceRel,
|
||||
correlate.getCorrelationId(),
|
||||
correlate.getRequiredColumns(),
|
||||
correlate.getJoinType()
|
||||
),
|
||||
druidRelFilter,
|
||||
plannerContext
|
||||
);
|
||||
@Override
|
||||
public RexNode visitFieldAccess(final RexFieldAccess fieldAccess)
|
||||
{
|
||||
if (fieldAccess.getReferenceExpr() instanceof RexCorrelVariable) {
|
||||
final RexCorrelVariable encounteredCorrelVariable = (RexCorrelVariable) fieldAccess.getReferenceExpr();
|
||||
if (encounteredCorrelVariable.id.equals(correlationId)) {
|
||||
final RexNode projectExpr = project.getProjects().get(fieldAccess.getField().getIndex());
|
||||
|
||||
// Rewrite RexInputRefs as correlation variable accesses.
|
||||
final RexBuilder rexBuilder = project.getCluster().getRexBuilder();
|
||||
final RexNode newCorrel = rexBuilder.makeCorrel(project.getInput().getRowType(), newCorrelationId);
|
||||
return new RexShuttle()
|
||||
{
|
||||
@Override
|
||||
public RexNode visitInputRef(RexInputRef inputRef)
|
||||
{
|
||||
requiredColumns.add(inputRef.getIndex());
|
||||
return project.getCluster().getRexBuilder().makeFieldAccess(newCorrel, inputRef.getIndex());
|
||||
}
|
||||
}.apply(projectExpr);
|
||||
}
|
||||
}
|
||||
|
||||
final RelBuilder relBuilder =
|
||||
call.builder()
|
||||
.push(druidCorr)
|
||||
.project(RexUtil.fixUp(
|
||||
rexBuilder,
|
||||
newProjectExprs,
|
||||
RelOptUtil.getFieldTypeList(druidCorr.getRowType())
|
||||
));
|
||||
|
||||
call.transformTo(relBuilder.build());
|
||||
return super.visitFieldAccess(fieldAccess);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,12 +31,9 @@ import org.apache.druid.sql.calcite.planner.Calcites;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException;
|
||||
import org.apache.druid.sql.calcite.rel.DruidQueryRel;
|
||||
import org.apache.druid.sql.calcite.table.DruidTable;
|
||||
import org.apache.druid.sql.calcite.table.InlineTable;
|
||||
import org.apache.druid.sql.calcite.table.RowSignatures;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
@ -78,12 +75,12 @@ public class DruidLogicalValuesRule extends RelOptRule
|
|||
values.getRowType().getFieldNames(),
|
||||
values.getRowType()
|
||||
);
|
||||
final DruidTable druidTable = new InlineTable(
|
||||
InlineDataSource.fromIterable(objectTuples, rowSignature),
|
||||
rowSignature
|
||||
);
|
||||
call.transformTo(
|
||||
DruidQueryRel.scanValues(values, druidTable, plannerContext)
|
||||
DruidQueryRel.scanConstantRel(
|
||||
values,
|
||||
InlineDataSource.fromIterable(objectTuples, rowSignature),
|
||||
plannerContext
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -29,12 +29,13 @@ import org.apache.calcite.rel.core.Filter;
|
|||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.rel.core.Sort;
|
||||
import org.apache.calcite.rel.core.Window;
|
||||
import org.apache.calcite.rel.rules.ProjectCorrelateTransposeRule;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.rel.DruidOuterQueryRel;
|
||||
import org.apache.druid.sql.calcite.rel.DruidQuery;
|
||||
import org.apache.druid.sql.calcite.rel.DruidRel;
|
||||
import org.apache.druid.sql.calcite.rel.PartialDruidQuery;
|
||||
import org.apache.druid.sql.calcite.run.EngineFeature;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
@ -97,13 +98,11 @@ public class DruidRules
|
|||
new DruidUnionRule(plannerContext),
|
||||
new DruidUnionDataSourceRule(plannerContext),
|
||||
DruidSortUnionRule.instance(),
|
||||
DruidJoinRule.instance(plannerContext),
|
||||
new DruidUnnestDatasourceRule(plannerContext),
|
||||
new DruidCorrelateUnnestRule(plannerContext)
|
||||
DruidJoinRule.instance(plannerContext)
|
||||
)
|
||||
);
|
||||
|
||||
if (plannerContext.queryContext().getBoolean(DruidQuery.CTX_ENABLE_WINDOW_FNS, false)) {
|
||||
if (plannerContext.featureAvailable(EngineFeature.WINDOW_FUNCTIONS)) {
|
||||
retVal.add(new DruidQueryRule<>(Window.class, PartialDruidQuery.Stage.WINDOW, PartialDruidQuery::withWindow));
|
||||
retVal.add(
|
||||
new DruidQueryRule<>(
|
||||
|
@ -115,6 +114,15 @@ public class DruidRules
|
|||
);
|
||||
retVal.add(DruidOuterQueryRule.WINDOW);
|
||||
}
|
||||
|
||||
if (plannerContext.featureAvailable(EngineFeature.UNNEST)) {
|
||||
retVal.add(new DruidUnnestRule(plannerContext));
|
||||
retVal.add(new DruidCorrelateUnnestRule(plannerContext));
|
||||
retVal.add(ProjectCorrelateTransposeRule.INSTANCE);
|
||||
retVal.add(CorrelateFilterLTransposeRule.instance());
|
||||
retVal.add(CorrelateFilterRTransposeRule.instance());
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
|
|
|
@ -1,107 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.rule;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.calcite.plan.RelOptRule;
|
||||
import org.apache.calcite.plan.RelOptRuleCall;
|
||||
import org.apache.calcite.rel.core.Uncollect;
|
||||
import org.apache.calcite.rel.logical.LogicalProject;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.tools.RelBuilder;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.rel.DruidQueryRel;
|
||||
import org.apache.druid.sql.calcite.rel.DruidUnnestDatasourceRel;
|
||||
|
||||
/**
|
||||
* This class creates the rule to abide by for creating unnest (internally uncollect) in Calcite.
|
||||
* Typically, Calcite plans the *unnest* part of the query involving a table such as
|
||||
* SELECT * from numFoo, unnest(dim3)
|
||||
* or even a standalone unnest query such as
|
||||
* SELECT * from unnest(ARRAY[1,2,3]) in the following way:
|
||||
* 78:Uncollect(subset=[rel#79:Subset#3.NONE.[]])
|
||||
* 76:LogicalProject(subset=[rel#77:Subset#2.NONE.[]], EXPR$0=[MV_TO_ARRAY($cor0.dim3)])
|
||||
* 7:LogicalValues(subset=[rel#75:Subset#1.NONE.[0]], tuples=[[{ 0 }]])
|
||||
*
|
||||
* Calcite tackles plans bottom up. Therefore,
|
||||
* {@link DruidLogicalValuesRule} converts the LogicalValues part into a leaf level {@link DruidQueryRel}
|
||||
* thereby creating the following subtree in the call tree
|
||||
*
|
||||
* Uncollect
|
||||
* \
|
||||
* LogicalProject
|
||||
* \
|
||||
* DruidQueryRel
|
||||
*
|
||||
*
|
||||
* This forms the premise of this rule. The goal is to transform the above-mentioned structure in the tree
|
||||
* with a new rel {@link DruidUnnestDatasourceRel} which shall be created here.
|
||||
*
|
||||
*/
|
||||
public class DruidUnnestDatasourceRule extends RelOptRule
|
||||
{
|
||||
private final PlannerContext plannerContext;
|
||||
|
||||
public DruidUnnestDatasourceRule(PlannerContext plannerContext)
|
||||
{
|
||||
super(
|
||||
operand(
|
||||
Uncollect.class,
|
||||
operand(LogicalProject.class, operand(DruidQueryRel.class, none()))
|
||||
)
|
||||
);
|
||||
this.plannerContext = plannerContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean matches(RelOptRuleCall call)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onMatch(final RelOptRuleCall call)
|
||||
{
|
||||
final Uncollect uncollectRel = call.rel(0);
|
||||
final LogicalProject logicalProject = call.rel(1);
|
||||
final DruidQueryRel druidQueryRel = call.rel(2);
|
||||
|
||||
final RexBuilder rexBuilder = logicalProject.getCluster().getRexBuilder();
|
||||
|
||||
final LogicalProject queryProject = LogicalProject.create(
|
||||
uncollectRel,
|
||||
ImmutableList.of(rexBuilder.makeInputRef(uncollectRel.getRowType().getFieldList().get(0).getType(), 0)),
|
||||
uncollectRel.getRowType()
|
||||
);
|
||||
|
||||
DruidUnnestDatasourceRel unnestDatasourceRel = new DruidUnnestDatasourceRel(
|
||||
uncollectRel,
|
||||
druidQueryRel.withPartialQuery(druidQueryRel.getPartialDruidQuery().withSelectProject(queryProject)),
|
||||
logicalProject,
|
||||
plannerContext
|
||||
);
|
||||
|
||||
final RelBuilder relBuilder =
|
||||
call.builder()
|
||||
.push(unnestDatasourceRel);
|
||||
|
||||
call.transformTo(relBuilder.build());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,180 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.rule;
|
||||
|
||||
import org.apache.calcite.plan.RelOptRule;
|
||||
import org.apache.calcite.plan.RelOptRuleCall;
|
||||
import org.apache.calcite.plan.RelOptUtil;
|
||||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.rel.core.Uncollect;
|
||||
import org.apache.calcite.rel.core.Values;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.rex.RexUtil;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.InputBindings;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
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.DruidQueryRel;
|
||||
import org.apache.druid.sql.calcite.rel.DruidUnnestRel;
|
||||
import org.apache.druid.sql.calcite.table.RowSignatures;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* This class creates the rule to abide by for creating unnest (internally uncollect) in Calcite.
|
||||
* Typically, Calcite plans the *unnest* part of the query involving a table such as
|
||||
* <pre>
|
||||
* SELECT * from numFoo, unnest(dim3)
|
||||
* </pre>
|
||||
* or even a standalone unnest query such as
|
||||
* <pre>
|
||||
* SELECT * from unnest(ARRAY[1,2,3]) in the following way:
|
||||
* 78:Uncollect(subset=[rel#79:Subset#3.NONE.[]])
|
||||
* 76:LogicalProject(subset=[rel#77:Subset#2.NONE.[]], EXPR$0=[MV_TO_ARRAY($cor0.dim3)])
|
||||
* 7:LogicalValues(subset=[rel#75:Subset#1.NONE.[0]], tuples=[[{ 0 }]])
|
||||
* </pre>
|
||||
* Calcite tackles plans bottom up. Therefore,
|
||||
* {@link DruidLogicalValuesRule} converts the LogicalValues part into a leaf level {@link DruidQueryRel}
|
||||
* thereby creating the following subtree in the call tree
|
||||
*
|
||||
* <pre>
|
||||
* Uncollect
|
||||
* \
|
||||
* LogicalProject
|
||||
* \
|
||||
* DruidQueryRel
|
||||
* </pre>
|
||||
*
|
||||
* This forms the premise of this rule. The goal is to transform the above-mentioned structure in the tree
|
||||
* with a new rel {@link DruidUnnestRel} which shall be created here.
|
||||
*/
|
||||
public class DruidUnnestRule extends RelOptRule
|
||||
{
|
||||
private final PlannerContext plannerContext;
|
||||
|
||||
public DruidUnnestRule(PlannerContext plannerContext)
|
||||
{
|
||||
super(
|
||||
operand(
|
||||
Uncollect.class,
|
||||
operand(Project.class, operand(Values.class, none()))
|
||||
)
|
||||
);
|
||||
this.plannerContext = plannerContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean matches(RelOptRuleCall call)
|
||||
{
|
||||
final Project projectRel = call.rel(1);
|
||||
final Values valuesRel = call.rel(2);
|
||||
|
||||
// Project must be a single field on top of a single row, and not refer to any bits of the input.
|
||||
// (The single row is a dummy row. We expect the Project expr to be a constant or a correlated field access.)
|
||||
return projectRel.getProjects().size() == 1
|
||||
&& valuesRel.getTuples().size() == 1
|
||||
&& RelOptUtil.InputFinder.bits(projectRel.getProjects(), null).isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onMatch(final RelOptRuleCall call)
|
||||
{
|
||||
final Uncollect uncollectRel = call.rel(0);
|
||||
final Project projectRel = call.rel(1);
|
||||
|
||||
final RexNode exprToUnnest = projectRel.getProjects().get(0);
|
||||
if (RexUtil.isConstant(exprToUnnest)) {
|
||||
// Constant expression: transform to DruidQueryRel on an inline datasource.
|
||||
final InlineDataSource inlineDataSource = toInlineDataSource(
|
||||
uncollectRel,
|
||||
exprToUnnest,
|
||||
plannerContext
|
||||
);
|
||||
|
||||
if (inlineDataSource != null) {
|
||||
call.transformTo(
|
||||
DruidQueryRel.scanConstantRel(
|
||||
uncollectRel,
|
||||
inlineDataSource,
|
||||
plannerContext
|
||||
)
|
||||
);
|
||||
}
|
||||
} else {
|
||||
// Transform to DruidUnnestRel, a holder for an unnest of a correlated variable.
|
||||
call.transformTo(
|
||||
DruidUnnestRel.create(
|
||||
uncollectRel.getCluster(),
|
||||
uncollectRel.getTraitSet(),
|
||||
exprToUnnest,
|
||||
plannerContext
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private static InlineDataSource toInlineDataSource(
|
||||
final Uncollect uncollectRel,
|
||||
final RexNode projectExpr,
|
||||
final PlannerContext plannerContext
|
||||
)
|
||||
{
|
||||
final DruidExpression expression = Expressions.toDruidExpression(
|
||||
plannerContext,
|
||||
RowSignature.empty(),
|
||||
projectExpr
|
||||
);
|
||||
|
||||
if (expression == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// Evaluate the expression. It's a constant, so no bindings are needed.
|
||||
final Expr parsedExpression = expression.parse(plannerContext.getExprMacroTable());
|
||||
final ExprEval<?> eval = parsedExpression.eval(InputBindings.nilBindings());
|
||||
final List<Object[]> rows = new ArrayList<>();
|
||||
|
||||
if (eval.isArray()) {
|
||||
final Object[] evalArray = eval.asArray();
|
||||
if (evalArray != null) {
|
||||
for (Object o : evalArray) {
|
||||
rows.add(new Object[]{o});
|
||||
}
|
||||
}
|
||||
} else {
|
||||
rows.add(new Object[]{eval.valueOrDefault()});
|
||||
}
|
||||
|
||||
// Transform to inline datasource.
|
||||
final RowSignature rowSignature = RowSignatures.fromRelDataType(
|
||||
uncollectRel.getRowType().getFieldNames(),
|
||||
uncollectRel.getRowType()
|
||||
);
|
||||
|
||||
return InlineDataSource.fromIterable(rows, rowSignature);
|
||||
}
|
||||
}
|
|
@ -23,7 +23,7 @@ import org.apache.druid.sql.calcite.external.ExternalDataSource;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
|
||||
/**
|
||||
* Arguments to {@link SqlEngine#feature(EngineFeature, PlannerContext)}.
|
||||
* Arguments to {@link SqlEngine#featureAvailable(EngineFeature, PlannerContext)}.
|
||||
*/
|
||||
public enum EngineFeature
|
||||
{
|
||||
|
@ -80,6 +80,21 @@ public enum EngineFeature
|
|||
*/
|
||||
ALLOW_BINDABLE_PLAN,
|
||||
|
||||
/**
|
||||
* Queries can use GROUPING SETS.
|
||||
*/
|
||||
GROUPING_SETS,
|
||||
|
||||
/**
|
||||
* Queries can use window functions.
|
||||
*/
|
||||
WINDOW_FUNCTIONS,
|
||||
|
||||
/**
|
||||
* Queries can use {@link org.apache.calcite.sql.fun.SqlStdOperatorTable#UNNEST}.
|
||||
*/
|
||||
UNNEST,
|
||||
|
||||
/**
|
||||
* Planner is permitted to use {@link org.apache.druid.sql.calcite.planner.JoinAlgorithm#BROADCAST} with RIGHT
|
||||
* and FULL join. Not guaranteed to produce correct results in either the native or MSQ engines, but we allow
|
||||
|
@ -87,5 +102,5 @@ public enum EngineFeature
|
|||
* that it actually *does* generate correct results in native when the join is processed on the Broker. It is much
|
||||
* less likely that MSQ will plan in such a way that generates correct results.
|
||||
*/
|
||||
ALLOW_BROADCAST_RIGHTY_JOIN
|
||||
ALLOW_BROADCAST_RIGHTY_JOIN;
|
||||
}
|
||||
|
|
|
@ -96,13 +96,16 @@ public class NativeSqlEngine implements SqlEngine
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean feature(EngineFeature feature, PlannerContext plannerContext)
|
||||
public boolean featureAvailable(EngineFeature feature, PlannerContext plannerContext)
|
||||
{
|
||||
switch (feature) {
|
||||
case CAN_SELECT:
|
||||
case ALLOW_BINDABLE_PLAN:
|
||||
case TIMESERIES_QUERY:
|
||||
case TOPN_QUERY:
|
||||
case GROUPING_SETS:
|
||||
case WINDOW_FUNCTIONS:
|
||||
case UNNEST:
|
||||
case ALLOW_BROADCAST_RIGHTY_JOIN:
|
||||
return true;
|
||||
case TIME_BOUNDARY_QUERY:
|
||||
|
|
|
@ -40,7 +40,7 @@ public interface SqlEngine
|
|||
/**
|
||||
* Whether a feature applies to this engine or not.
|
||||
*/
|
||||
boolean feature(EngineFeature feature, PlannerContext plannerContext);
|
||||
boolean featureAvailable(EngineFeature feature, PlannerContext plannerContext);
|
||||
|
||||
/**
|
||||
* Validates a provided query context. Returns quietly if the context is OK; throws {@link ValidationException}
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.calcite.rel.RelNode;
|
|||
import org.apache.calcite.rel.logical.LogicalTableScan;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
/**
|
||||
* Represents a specialized table used within Druid's Calcite-based planner.
|
||||
|
@ -38,12 +37,9 @@ public class InlineTable extends DruidTable
|
|||
{
|
||||
private final DataSource dataSource;
|
||||
|
||||
public InlineTable(
|
||||
final InlineDataSource dataSource,
|
||||
final RowSignature rowSignature
|
||||
)
|
||||
public InlineTable(final InlineDataSource dataSource)
|
||||
{
|
||||
super(rowSignature);
|
||||
super(dataSource.getRowSignature());
|
||||
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
|
||||
}
|
||||
|
||||
|
|
|
@ -50,7 +50,7 @@ public class ViewSqlEngine implements SqlEngine
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean feature(EngineFeature feature, PlannerContext plannerContext)
|
||||
public boolean featureAvailable(EngineFeature feature, PlannerContext plannerContext)
|
||||
{
|
||||
switch (feature) {
|
||||
// Use most permissive set of SELECT features, since our goal is to get the row type of the view.
|
||||
|
@ -60,6 +60,9 @@ public class ViewSqlEngine implements SqlEngine
|
|||
case ALLOW_BINDABLE_PLAN:
|
||||
case READ_EXTERNAL_DATA:
|
||||
case SCAN_ORDER_BY_NON_TIME:
|
||||
case GROUPING_SETS:
|
||||
case WINDOW_FUNCTIONS:
|
||||
case UNNEST:
|
||||
return true;
|
||||
|
||||
// Views can't sit on top of INSERT or REPLACE.
|
||||
|
|
|
@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.inject.Injector;
|
||||
import org.apache.calcite.plan.RelOptPlanner;
|
||||
import org.apache.commons.text.StringEscapeUtils;
|
||||
import org.apache.druid.annotations.UsedByJUnitParamsRunner;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.guice.DruidInjectorBuilder;
|
||||
|
@ -79,8 +80,6 @@ import org.apache.druid.server.security.AuthConfig;
|
|||
import org.apache.druid.server.security.AuthenticationResult;
|
||||
import org.apache.druid.server.security.ForbiddenException;
|
||||
import org.apache.druid.server.security.ResourceAction;
|
||||
import org.apache.druid.sql.PreparedStatement;
|
||||
import org.apache.druid.sql.SqlQueryPlus;
|
||||
import org.apache.druid.sql.SqlStatementFactory;
|
||||
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
|
@ -894,16 +893,6 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
}
|
||||
}
|
||||
|
||||
public Set<ResourceAction> analyzeResources(
|
||||
final SqlStatementFactory sqlStatementFactory,
|
||||
final SqlQueryPlus query
|
||||
)
|
||||
{
|
||||
PreparedStatement stmt = sqlStatementFactory.preparedStatement(query);
|
||||
stmt.prepare();
|
||||
return stmt.allResources();
|
||||
}
|
||||
|
||||
public void assertResultsEquals(String sql, List<Object[]> expectedResults, List<Object[]> results)
|
||||
{
|
||||
for (int i = 0; i < results.size(); i++) {
|
||||
|
@ -1231,30 +1220,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
PrintStream out = System.out;
|
||||
out.println("-- Actual results --");
|
||||
for (int rowIndex = 0; rowIndex < results.size(); rowIndex++) {
|
||||
Object[] row = results.get(rowIndex);
|
||||
out.print("new Object[] {");
|
||||
for (int colIndex = 0; colIndex < row.length; colIndex++) {
|
||||
Object col = row[colIndex];
|
||||
if (colIndex > 0) {
|
||||
out.print(", ");
|
||||
}
|
||||
if (col == null) {
|
||||
out.print("null");
|
||||
} else if (col instanceof String) {
|
||||
out.print("\"");
|
||||
out.print(col);
|
||||
out.print("\"");
|
||||
} else if (col instanceof Long) {
|
||||
out.print(col);
|
||||
out.print("L");
|
||||
} else if (col instanceof Double) {
|
||||
out.print(col);
|
||||
out.print("D");
|
||||
} else {
|
||||
out.print(col);
|
||||
}
|
||||
}
|
||||
out.print("}");
|
||||
printArray(results.get(rowIndex), out);
|
||||
if (rowIndex < results.size() - 1) {
|
||||
out.print(",");
|
||||
}
|
||||
|
@ -1262,4 +1228,45 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
}
|
||||
out.println("----");
|
||||
}
|
||||
|
||||
private static void printArray(final Object[] array, final PrintStream out)
|
||||
{
|
||||
printArrayImpl(array, out, "new Object[]{", "}");
|
||||
}
|
||||
|
||||
private static void printList(final List<?> list, final PrintStream out)
|
||||
{
|
||||
printArrayImpl(list.toArray(new Object[0]), out, "ImmutableList.of(", ")");
|
||||
}
|
||||
|
||||
private static void printArrayImpl(final Object[] array, final PrintStream out, final String pre, final String post)
|
||||
{
|
||||
out.print(pre);
|
||||
for (int colIndex = 0; colIndex < array.length; colIndex++) {
|
||||
Object col = array[colIndex];
|
||||
if (colIndex > 0) {
|
||||
out.print(", ");
|
||||
}
|
||||
if (col == null) {
|
||||
out.print("null");
|
||||
} else if (col instanceof String) {
|
||||
out.print("\"");
|
||||
out.print(StringEscapeUtils.escapeJava((String) col));
|
||||
out.print("\"");
|
||||
} else if (col instanceof Long) {
|
||||
out.print(col);
|
||||
out.print("L");
|
||||
} else if (col instanceof Double) {
|
||||
out.print(col);
|
||||
out.print("D");
|
||||
} else if (col instanceof Object[]) {
|
||||
printArray(array, out);
|
||||
} else if (col instanceof List) {
|
||||
printList((List<?>) col, out);
|
||||
} else {
|
||||
out.print(col);
|
||||
}
|
||||
}
|
||||
out.print(post);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.sql.calcite;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.HumanReadableBytes;
|
||||
|
@ -31,6 +32,7 @@ import org.apache.druid.math.expr.ExpressionProcessing;
|
|||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.TableDataSource;
|
||||
import org.apache.druid.query.UnnestDataSource;
|
||||
|
@ -43,9 +45,11 @@ import org.apache.druid.query.expression.TestExprMacroTable;
|
|||
import org.apache.druid.query.filter.AndDimFilter;
|
||||
import org.apache.druid.query.filter.ExpressionDimFilter;
|
||||
import org.apache.druid.query.filter.InDimFilter;
|
||||
import org.apache.druid.query.filter.LikeDimFilter;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.groupby.having.DimFilterHavingSpec;
|
||||
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
|
||||
import org.apache.druid.query.groupby.orderby.NoopLimitSpec;
|
||||
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
|
||||
|
@ -58,6 +62,7 @@ import org.apache.druid.segment.column.ColumnType;
|
|||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.join.JoinType;
|
||||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -65,12 +70,20 @@ import org.junit.Test;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Tests for array functions and array types
|
||||
*/
|
||||
public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
||||
{
|
||||
private static final Map<String, Object> QUERY_CONTEXT_UNNEST =
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.putAll(QUERY_CONTEXT_DEFAULT)
|
||||
.put(PlannerContext.CTX_ENABLE_UNNEST, true)
|
||||
.put(QueryContexts.CTX_SQL_STRINGIFY_ARRAYS, false)
|
||||
.build();
|
||||
|
||||
// test some query stuffs, sort of limited since no native array column types so either need to use constructor or
|
||||
// array aggregator
|
||||
@Test
|
||||
|
@ -1889,7 +1902,8 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
),
|
||||
expected -> {
|
||||
expected.expect(IAE.class);
|
||||
expected.expectMessage("Cannot create a nested array type [ARRAY<ARRAY<LONG>>], 'druid.expressions.allowNestedArrays' must be set to true");
|
||||
expected.expectMessage(
|
||||
"Cannot create a nested array type [ARRAY<ARRAY<LONG>>], 'druid.expressions.allowNestedArrays' must be set to true");
|
||||
}
|
||||
);
|
||||
}
|
||||
|
@ -2125,6 +2139,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
}
|
||||
testQuery(
|
||||
"SELECT numfoo.dim4, j.arr, ARRAY_TO_STRING(j.arr, ',') FROM numfoo INNER JOIN (SELECT dim4, ARRAY_AGG(DISTINCT dim1) as arr FROM numfoo WHERE dim1 is not null GROUP BY 1) as j ON numfoo.dim4 = j.dim4",
|
||||
QUERY_CONTEXT_DEFAULT,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
|
@ -2480,6 +2495,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
}
|
||||
testQuery(
|
||||
"SELECT dim1,dim2 FROM foo WHERE ARRAY_CONTAINS((SELECT ARRAY_AGG(DISTINCT dim1) FROM foo WHERE dim1 is not null), dim1)",
|
||||
QUERY_CONTEXT_DEFAULT,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
|
@ -2559,6 +2575,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
}
|
||||
testQuery(
|
||||
"SELECT dim2, COUNT(*) FROM foo WHERE ARRAY_CONTAINS((SELECT ARRAY_AGG(DISTINCT dim1) FROM foo WHERE dim1 is not null), dim1) GROUP BY 1",
|
||||
QUERY_CONTEXT_DEFAULT,
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(
|
||||
|
@ -2650,23 +2667,23 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT * FROM UNNEST(ARRAY[1,2,3])",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
UnnestDataSource.create(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(new Object[]{new Object[]{1L, 2L, 3L}}),
|
||||
RowSignature.builder().add("inline", ColumnType.LONG_ARRAY).build()
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(
|
||||
new Object[]{1L},
|
||||
new Object[]{2L},
|
||||
new Object[]{3L}
|
||||
),
|
||||
"inline",
|
||||
"EXPR$0",
|
||||
null
|
||||
RowSignature.builder().add("EXPR$0", ColumnType.LONG).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of(
|
||||
"EXPR$0"
|
||||
))
|
||||
|
@ -2691,21 +2708,19 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
"dim3",
|
||||
"EXPR$0",
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.columns(ImmutableList.of(
|
||||
"EXPR$0"
|
||||
))
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
useDefault ?
|
||||
|
@ -2732,6 +2747,141 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestTwice()
|
||||
{
|
||||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT dim1, MV_TO_ARRAY(dim3), STRING_TO_ARRAY(dim1, U&'\\005C.') AS dim1_split, dim1_split_unnest, dim3_unnest\n"
|
||||
+ "FROM\n"
|
||||
+ " druid.numfoo,\n"
|
||||
+ " UNNEST(STRING_TO_ARRAY(dim1, U&'\\005C.')) as t2 (dim1_split_unnest),\n"
|
||||
+ " UNNEST(MV_TO_ARRAY(dim3)) as t3 (dim3_unnest)",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
UnnestDataSource.create(
|
||||
UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
expressionVirtualColumn(
|
||||
"j0.unnest",
|
||||
"string_to_array(\"dim1\",'\\u005C.')",
|
||||
ColumnType.STRING_ARRAY
|
||||
),
|
||||
null
|
||||
),
|
||||
expressionVirtualColumn(
|
||||
"_j0.unnest",
|
||||
"\"dim3\"",
|
||||
ColumnType.STRING
|
||||
),
|
||||
null
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(
|
||||
expressionVirtualColumn(
|
||||
"v0",
|
||||
"mv_to_array(\"dim3\")",
|
||||
ColumnType.STRING_ARRAY
|
||||
),
|
||||
expressionVirtualColumn(
|
||||
"v1",
|
||||
"string_to_array(\"dim1\",'\\u005C.')",
|
||||
ColumnType.STRING_ARRAY
|
||||
)
|
||||
)
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("_j0.unnest", "dim1", "j0.unnest", "v0", "v1"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"", ImmutableList.of("a", "b"), useDefault ? null : ImmutableList.of(""), "", "a"},
|
||||
new Object[]{"", ImmutableList.of("a", "b"), useDefault ? null : ImmutableList.of(""), "", "b"},
|
||||
new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "10", "b"},
|
||||
new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "10", "c"},
|
||||
new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "1", "b"},
|
||||
new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "1", "c"},
|
||||
new Object[]{"2", ImmutableList.of("d"), ImmutableList.of("2"), "2", "d"},
|
||||
new Object[]{"1", useDefault ? null : ImmutableList.of(""), ImmutableList.of("1"), "1", ""},
|
||||
new Object[]{"def", null, ImmutableList.of("def"), "def", NullHandling.defaultStringValue()},
|
||||
new Object[]{"abc", null, ImmutableList.of("abc"), "abc", NullHandling.defaultStringValue()}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestTwiceWithFiltersAndExpressions()
|
||||
{
|
||||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT dim1, MV_TO_ARRAY(dim3), STRING_TO_ARRAY(dim1, U&'\\005C.') AS dim1_split, dim1_split_unnest, dim3_unnest || 'xx'\n"
|
||||
+ "FROM\n"
|
||||
+ " druid.numfoo,\n"
|
||||
+ " UNNEST(STRING_TO_ARRAY(dim1, U&'\\005C.')) as t2 (dim1_split_unnest),\n"
|
||||
+ " UNNEST(MV_TO_ARRAY(dim3)) as t3 (dim3_unnest)"
|
||||
+ "WHERE dim1_split_unnest IN ('1', '2') AND dim3_unnest LIKE '_'",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
UnnestDataSource.create(
|
||||
UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
expressionVirtualColumn(
|
||||
"j0.unnest",
|
||||
"string_to_array(\"dim1\",'\\u005C.')",
|
||||
ColumnType.STRING_ARRAY
|
||||
),
|
||||
null
|
||||
),
|
||||
expressionVirtualColumn(
|
||||
"_j0.unnest",
|
||||
"\"dim3\"",
|
||||
ColumnType.STRING
|
||||
),
|
||||
null
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(
|
||||
expressionVirtualColumn(
|
||||
"v0",
|
||||
"mv_to_array(\"dim3\")",
|
||||
ColumnType.STRING_ARRAY
|
||||
),
|
||||
expressionVirtualColumn(
|
||||
"v1",
|
||||
"string_to_array(\"dim1\",'\\u005C.')",
|
||||
ColumnType.STRING_ARRAY
|
||||
),
|
||||
expressionVirtualColumn(
|
||||
"v2",
|
||||
"concat(\"_j0.unnest\",'xx')",
|
||||
ColumnType.STRING
|
||||
)
|
||||
)
|
||||
.filters(and(
|
||||
in("j0.unnest", ImmutableList.of("1", "2"), null),
|
||||
new LikeDimFilter("_j0.unnest", "_", null, null)
|
||||
))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("dim1", "j0.unnest", "v0", "v1", "v2"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "1", "bxx"},
|
||||
new Object[]{"10.1", ImmutableList.of("b", "c"), ImmutableList.of("10", "1"), "1", "cxx"},
|
||||
new Object[]{"2", ImmutableList.of("d"), ImmutableList.of("2"), "2", "dxx"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithGroupBy()
|
||||
{
|
||||
|
@ -2743,19 +2893,19 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) GROUP BY d3 ",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
"dim3",
|
||||
"EXPR$0",
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.setDimensions(new DefaultDimensionSpec("EXPR$0", "_d0", ColumnType.STRING))
|
||||
.setContext(QUERY_CONTEXT_UNNEST)
|
||||
.setDimensions(new DefaultDimensionSpec("j0.unnest", "_d0", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.setContext(QUERY_CONTEXT_UNNEST)
|
||||
.build()
|
||||
),
|
||||
useDefault ?
|
||||
|
@ -2788,17 +2938,17 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT d3, COUNT(*) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) AS unnested(d3) GROUP BY d3 ORDER BY d3 DESC ",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
"dim3",
|
||||
"EXPR$0",
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.setDimensions(new DefaultDimensionSpec("EXPR$0", "_d0", ColumnType.STRING))
|
||||
.setContext(QUERY_CONTEXT_UNNEST)
|
||||
.setDimensions(new DefaultDimensionSpec("j0.unnest", "_d0", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setLimitSpec(
|
||||
DefaultLimitSpec
|
||||
|
@ -2811,7 +2961,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
.build()
|
||||
)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("a0"))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.setContext(QUERY_CONTEXT_UNNEST)
|
||||
.build()
|
||||
),
|
||||
useDefault ?
|
||||
|
@ -2844,20 +2994,20 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT d3, COUNT(*) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) AS unnested(d3) GROUP BY d3 ORDER BY d3 ASC LIMIT 4 ",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
new TopNQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
"dim3",
|
||||
"EXPR$0",
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.dimension(new DefaultDimensionSpec("EXPR$0", "_d0", ColumnType.STRING))
|
||||
.dimension(new DefaultDimensionSpec("j0.unnest", "_d0", ColumnType.STRING))
|
||||
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
|
||||
.threshold(4)
|
||||
.aggregators(aggregators(new CountAggregatorFactory("a0")))
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.build()
|
||||
),
|
||||
useDefault ?
|
||||
|
@ -2876,6 +3026,44 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithGroupByHaving()
|
||||
{
|
||||
skipVectorize();
|
||||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT d3, COUNT(*) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) AS unnested(d3) GROUP BY d3 HAVING COUNT(*) = 1",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setContext(QUERY_CONTEXT_UNNEST)
|
||||
.setDimensions(new DefaultDimensionSpec("j0.unnest", "_d0", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("a0"))
|
||||
.setHavingSpec(new DimFilterHavingSpec(selector("a0", "1", null), true))
|
||||
.setContext(QUERY_CONTEXT_UNNEST)
|
||||
.build()
|
||||
),
|
||||
useDefault ?
|
||||
ImmutableList.of(
|
||||
new Object[]{"a", 1L},
|
||||
new Object[]{"c", 1L},
|
||||
new Object[]{"d", 1L}
|
||||
) :
|
||||
ImmutableList.of(
|
||||
new Object[]{"", 1L},
|
||||
new Object[]{"a", 1L},
|
||||
new Object[]{"c", 1L},
|
||||
new Object[]{"d", 1L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithLimit()
|
||||
|
@ -2888,21 +3076,19 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) LIMIT 3",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
"dim3",
|
||||
"EXPR$0",
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.columns(ImmutableList.of(
|
||||
"EXPR$0"
|
||||
))
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.limit(3)
|
||||
.build()
|
||||
),
|
||||
|
@ -2925,21 +3111,19 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT d3 FROM (select dim1, dim2, dim3 from druid.numfoo), UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
"dim3",
|
||||
"EXPR$0",
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.columns(ImmutableList.of(
|
||||
"EXPR$0"
|
||||
))
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
useDefault ?
|
||||
|
@ -2977,6 +3161,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT d3 FROM (select * from druid.numfoo where dim2='a'), UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
|
@ -2986,43 +3171,21 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
new TableDataSource(CalciteTests.DATASOURCE3)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.filters(new SelectorDimFilter("dim2", "a", null))
|
||||
.columns(
|
||||
"__time",
|
||||
"cnt",
|
||||
"d1",
|
||||
"d2",
|
||||
"dim1",
|
||||
"dim3",
|
||||
"dim4",
|
||||
"dim5",
|
||||
"dim6",
|
||||
"f1",
|
||||
"f2",
|
||||
"l1",
|
||||
"l2",
|
||||
"m1",
|
||||
"m2",
|
||||
"unique_dim1",
|
||||
"v0"
|
||||
)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.columns("dim3")
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.build()
|
||||
),
|
||||
"dim3",
|
||||
"EXPR$0",
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.columns(ImmutableList.of(
|
||||
"EXPR$0"
|
||||
))
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
|
@ -3033,6 +3196,99 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithFiltersInsideAndOutside()
|
||||
{
|
||||
skipVectorize();
|
||||
testQuery(
|
||||
"SELECT d3 FROM\n"
|
||||
+ " (select * from druid.numfoo where dim2='a') t,\n"
|
||||
+ " UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)\n"
|
||||
+ "WHERE t.dim1 <> 'foo'\n"
|
||||
+ "AND unnested.d3 <> 'b'",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new QueryDataSource(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.filters(
|
||||
and(
|
||||
selector("dim2", "a", null),
|
||||
not(selector("dim1", "foo", null))
|
||||
)
|
||||
)
|
||||
.columns("dim3")
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.build()
|
||||
),
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.filters(not(selector("j0.unnest", "b", null)))
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"a"},
|
||||
new Object[]{""}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithFiltersOutside()
|
||||
{
|
||||
skipVectorize();
|
||||
testQuery(
|
||||
"SELECT d3 FROM\n"
|
||||
+ " druid.numfoo t,\n"
|
||||
+ " UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)\n"
|
||||
+ "WHERE t.dim2='a'\n"
|
||||
+ "AND t.dim1 <> 'foo'\n"
|
||||
+ "AND (unnested.d3 IN ('a', 'c') OR unnested.d3 LIKE '_')",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.filters(
|
||||
and(
|
||||
or(
|
||||
new LikeDimFilter("j0.unnest", "_", null, null),
|
||||
in("j0.unnest", ImmutableList.of("a", "c"), null)
|
||||
),
|
||||
selector("dim2", "a", null),
|
||||
not(selector("dim1", "foo", null))
|
||||
)
|
||||
)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"a"},
|
||||
new Object[]{"b"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithInFilters()
|
||||
{
|
||||
|
@ -3044,51 +3300,20 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT d3 FROM (select * from druid.numfoo where dim2 IN ('a','b','ab','abc')), UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new QueryDataSource(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.filters(new InDimFilter("dim2", ImmutableList.of("a", "b", "ab", "abc"), null))
|
||||
.columns(
|
||||
"__time",
|
||||
"cnt",
|
||||
"d1",
|
||||
"d2",
|
||||
"dim1",
|
||||
"dim2",
|
||||
"dim3",
|
||||
"dim4",
|
||||
"dim5",
|
||||
"dim6",
|
||||
"f1",
|
||||
"f2",
|
||||
"l1",
|
||||
"l2",
|
||||
"m1",
|
||||
"m2",
|
||||
"unique_dim1"
|
||||
)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
"dim3",
|
||||
"EXPR$0",
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.filters(new InDimFilter("dim2", ImmutableList.of("a", "b", "ab", "abc"), null))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.columns(ImmutableList.of(
|
||||
"EXPR$0"
|
||||
))
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
|
@ -3112,22 +3337,19 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT strings FROM druid.numfoo, UNNEST(ARRAY[dim4, dim5]) as unnested (strings)",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
"v0",
|
||||
"EXPR$0",
|
||||
expressionVirtualColumn("j0.unnest", "array(\"dim4\",\"dim5\")", ColumnType.STRING_ARRAY),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(expressionVirtualColumn("v0", "array(\"dim4\",\"dim5\")", ColumnType.STRING_ARRAY))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.columns(ImmutableList.of(
|
||||
"EXPR$0"
|
||||
))
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
|
@ -3154,22 +3376,23 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT d24, COUNT(*) FROM druid.numfoo, UNNEST(ARRAY[dim2, dim4]) AS unnested(d24) GROUP BY d24 ORDER BY d24 DESC ",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
"v0",
|
||||
"EXPR$0",
|
||||
null
|
||||
))
|
||||
.setVirtualColumns(expressionVirtualColumn(
|
||||
"v0",
|
||||
"array(\"dim2\",\"dim4\")",
|
||||
ColumnType.STRING_ARRAY
|
||||
))
|
||||
.setDataSource(
|
||||
UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
expressionVirtualColumn(
|
||||
"j0.unnest",
|
||||
"array(\"dim2\",\"dim4\")",
|
||||
ColumnType.STRING_ARRAY
|
||||
),
|
||||
null
|
||||
)
|
||||
)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.setDimensions(new DefaultDimensionSpec("EXPR$0", "_d0", ColumnType.STRING))
|
||||
.setContext(QUERY_CONTEXT_UNNEST)
|
||||
.setDimensions(new DefaultDimensionSpec("j0.unnest", "_d0", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setLimitSpec(
|
||||
DefaultLimitSpec
|
||||
|
@ -3182,7 +3405,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
.build()
|
||||
)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("a0"))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.setContext(QUERY_CONTEXT_UNNEST)
|
||||
.build()
|
||||
),
|
||||
useDefault ?
|
||||
|
@ -3209,6 +3432,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT d3 from (SELECT * from druid.numfoo JOIN (select dim2 as t from druid.numfoo where dim2 IN ('a','b','ab','abc')) ON dim2=t), UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
|
@ -3223,27 +3447,22 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.filters(new InDimFilter("dim2", ImmutableList.of("a", "b", "ab", "abc"), null))
|
||||
.columns(
|
||||
"dim2"
|
||||
)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.columns("dim2")
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.build()
|
||||
),
|
||||
"j0.",
|
||||
"(\"dim2\" == \"j0.dim2\")",
|
||||
JoinType.INNER
|
||||
),
|
||||
"dim3",
|
||||
"EXPR$0",
|
||||
expressionVirtualColumn("_j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.columns(ImmutableList.of(
|
||||
"EXPR$0"
|
||||
))
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("_j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
useDefault ?
|
||||
|
@ -3278,32 +3497,19 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT longs FROM druid.numfoo, UNNEST(ARRAY[1,2,3]) as unnested (longs)",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
join(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
new QueryDataSource(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(
|
||||
UnnestDataSource.create(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(new Object[]{new Object[]{1L, 2L, 3L}}),
|
||||
RowSignature.builder().add("inline", ColumnType.LONG_ARRAY).build()
|
||||
),
|
||||
"inline",
|
||||
"EXPR$0",
|
||||
null
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.columns(ImmutableList.of(
|
||||
"EXPR$0"
|
||||
))
|
||||
.build()
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(
|
||||
new Object[]{1L},
|
||||
new Object[]{2L},
|
||||
new Object[]{3L}
|
||||
),
|
||||
RowSignature.builder().add("EXPR$0", ColumnType.LONG).build()
|
||||
),
|
||||
"j0.",
|
||||
"1",
|
||||
|
@ -3313,10 +3519,8 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.columns(ImmutableList.of(
|
||||
"j0.EXPR$0"
|
||||
))
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.EXPR$0"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
|
|
|
@ -108,9 +108,9 @@ public class CalciteScanSignatureTest extends BaseCalciteQueryTest
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean feature(EngineFeature feature, PlannerContext plannerContext)
|
||||
public boolean featureAvailable(EngineFeature feature, PlannerContext plannerContext)
|
||||
{
|
||||
return feature == EngineFeature.SCAN_NEEDS_SIGNATURE || parent.feature(feature, plannerContext);
|
||||
return feature == EngineFeature.SCAN_NEEDS_SIGNATURE || parent.featureAvailable(feature, plannerContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.druid.query.operator.OperatorFactory;
|
|||
import org.apache.druid.query.operator.WindowOperatorQuery;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
@ -123,7 +124,7 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest
|
|||
testBuilder()
|
||||
.skipVectorize(true)
|
||||
.sql(input.sql)
|
||||
.queryContext(ImmutableMap.of("windowsAreForClosers", true))
|
||||
.queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true))
|
||||
.addCustomVerification(QueryVerification.ofResults(results -> {
|
||||
if (results.exception != null) {
|
||||
throw new RE(results.exception, "Failed to execute because of exception.");
|
||||
|
|
|
@ -74,7 +74,7 @@ public class IngestionTestSqlEngine implements SqlEngine
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean feature(final EngineFeature feature, final PlannerContext plannerContext)
|
||||
public boolean featureAvailable(final EngineFeature feature, final PlannerContext plannerContext)
|
||||
{
|
||||
switch (feature) {
|
||||
case CAN_SELECT:
|
||||
|
|
Loading…
Reference in New Issue