mirror of https://github.com/apache/druid.git
Intervals are updated properly for Unnest queries (#15020)
Fixes a bug where the unnest queries were not updated with the correct intervals.
This commit is contained in:
parent
64754b6799
commit
cb050282a0
|
@ -53,6 +53,10 @@ public class MaterializedViewQueryTest
|
|||
private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
|
||||
private DataSourceOptimizer optimizer;
|
||||
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
|
|
|
@ -1798,7 +1798,7 @@ public class MSQSelectTest extends MSQTestBase
|
|||
.setExpectedValidationErrorMatcher(
|
||||
new DruidExceptionMatcher(DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "general")
|
||||
.expectMessageIs(
|
||||
"Query planning failed for unknown reason, our best guess is this "
|
||||
"Query could not be planned. A possible reason is "
|
||||
+ "[LATEST and EARLIEST aggregators implicitly depend on the __time column, "
|
||||
+ "but the table queried doesn't contain a __time column. "
|
||||
+ "Please use LATEST_BY or EARLIEST_BY and specify the column explicitly.]"
|
||||
|
|
|
@ -696,7 +696,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
.ofCategory(DruidException.Category.INVALID_INPUT)
|
||||
.build(
|
||||
exception,
|
||||
"Query planning failed for unknown reason, our best guess is this [%s]",
|
||||
"Query could not be planned. A possible reason is [%s]",
|
||||
errorMessage
|
||||
);
|
||||
}
|
||||
|
|
|
@ -42,22 +42,26 @@ import org.apache.calcite.rex.RexInputRef;
|
|||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.util.ImmutableBitSet;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.FilteredDataSource;
|
||||
import org.apache.druid.query.JoinDataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.TableDataSource;
|
||||
import org.apache.druid.query.UnnestDataSource;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.PostAggregator;
|
||||
import org.apache.druid.query.aggregation.SimpleLongAggregatorFactory;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.filter.AndDimFilter;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.groupby.having.DimFilterHavingSpec;
|
||||
|
@ -773,6 +777,17 @@ public class DruidQuery
|
|||
return VirtualColumns.create(columns);
|
||||
}
|
||||
|
||||
public static List<DimFilter> getAllFiltersUnderDataSource(DataSource d, List<DimFilter> dimFilterList)
|
||||
{
|
||||
if (d instanceof FilteredDataSource) {
|
||||
dimFilterList.add(((FilteredDataSource) d).getFilter());
|
||||
}
|
||||
for (DataSource ds : d.getChildren()) {
|
||||
dimFilterList.addAll(getAllFiltersUnderDataSource(ds, dimFilterList));
|
||||
}
|
||||
return dimFilterList;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a pair of DataSource and Filtration object created on the query filter. In case the, data source is
|
||||
* a join datasource, the datasource may be altered and left filter of join datasource may
|
||||
|
@ -786,8 +801,44 @@ public class DruidQuery
|
|||
JoinableFactoryWrapper joinableFactoryWrapper
|
||||
)
|
||||
{
|
||||
if (!canUseIntervalFiltering(dataSource)) {
|
||||
if (dataSource instanceof UnnestDataSource) {
|
||||
// UnnestDataSource can have another unnest data source
|
||||
// join datasource, filtered data source, etc as base
|
||||
Pair<DataSource, Filtration> pair = getFiltration(
|
||||
((UnnestDataSource) dataSource).getBase(),
|
||||
filter,
|
||||
virtualColumnRegistry,
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
return Pair.of(dataSource, pair.rhs);
|
||||
} else if (!canUseIntervalFiltering(dataSource)) {
|
||||
return Pair.of(dataSource, toFiltration(filter, virtualColumnRegistry.getFullRowSignature(), false));
|
||||
} else if (dataSource instanceof FilteredDataSource) {
|
||||
// A filteredDS is created only inside the rel for Unnest, ensuring it only grabs the outermost filter
|
||||
// and, if possible, pushes it down inside the data source.
|
||||
// So a chain of Filter->Unnest->Filter is typically impossible when the query is done through SQL.
|
||||
// Also, Calcite has filter reduction rules that push filters deep into base data sources for better query planning.
|
||||
// A base table with a chain of filters is synonymous with a filteredDS.
|
||||
// We recursively find all filters under a filteredDS and then
|
||||
// 1) creating a filtration from the filteredDS's filters and
|
||||
// 2) Updating the interval of the outer filter with the intervals in step 1, and you'll see these 2 calls in the code
|
||||
List<DimFilter> dimFilterList = getAllFiltersUnderDataSource(dataSource, new ArrayList<>());
|
||||
final FilteredDataSource filteredDataSource = (FilteredDataSource) dataSource;
|
||||
// Defensive check as in the base of a filter cannot be another filter
|
||||
final DataSource baseOfFilterDataSource = filteredDataSource.getBase();
|
||||
if (baseOfFilterDataSource instanceof FilteredDataSource) {
|
||||
throw DruidException.defensive("Cannot create a filteredDataSource using another filteredDataSource as a base");
|
||||
}
|
||||
final boolean useIntervalFiltering = canUseIntervalFiltering(filteredDataSource);
|
||||
final Filtration baseFiltration = toFiltration(
|
||||
new AndDimFilter(dimFilterList),
|
||||
virtualColumnRegistry.getFullRowSignature(),
|
||||
useIntervalFiltering
|
||||
);
|
||||
// Adds the intervals from the filter of filtered data source to query filtration
|
||||
final Filtration queryFiltration = Filtration.create(filter, baseFiltration.getIntervals())
|
||||
.optimize(virtualColumnRegistry.getFullRowSignature());
|
||||
return Pair.of(filteredDataSource, queryFiltration);
|
||||
} else if (dataSource instanceof JoinDataSource && ((JoinDataSource) dataSource).getLeftFilter() != null) {
|
||||
final JoinDataSource joinDataSource = (JoinDataSource) dataSource;
|
||||
|
||||
|
@ -809,7 +860,6 @@ public class DruidQuery
|
|||
leftFiltration.getDimFilter(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
return Pair.of(newDataSource, queryFiltration);
|
||||
} else {
|
||||
return Pair.of(dataSource, toFiltration(filter, virtualColumnRegistry.getFullRowSignature(), true));
|
||||
|
|
|
@ -785,7 +785,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
new DruidExceptionMatcher(DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "general")
|
||||
.expectMessageIs(
|
||||
StringUtils.format(
|
||||
"Query planning failed for unknown reason, our best guess is this [%s]",
|
||||
"Query could not be planned. A possible reason is [%s]",
|
||||
expectedError
|
||||
)
|
||||
)
|
||||
|
|
|
@ -24,12 +24,14 @@ 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;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.FilteredDataSource;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.LookupDataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
|
@ -4869,4 +4871,362 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithTimeFilterOnly()
|
||||
{
|
||||
testQuery(
|
||||
"select c from foo, unnest(MV_TO_ARRAY(dim3)) as u(c)"
|
||||
+ " where __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
FilteredDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE1),
|
||||
range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false)
|
||||
),
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z")))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"b"},
|
||||
new Object[]{"c"},
|
||||
new Object[]{"d"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithTimeFilterAndAnotherFilter()
|
||||
{
|
||||
testQuery(
|
||||
"select c from foo, unnest(MV_TO_ARRAY(dim3)) as u(c) "
|
||||
+ " where m1=2 and __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
FilteredDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE1),
|
||||
and(
|
||||
useDefault ? equality("m1", 2, ColumnType.FLOAT) :
|
||||
equality("m1", 2.0, ColumnType.FLOAT),
|
||||
range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false)
|
||||
)
|
||||
),
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z")))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"b"},
|
||||
new Object[]{"c"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithTimeFilterOrAnotherFilter()
|
||||
{
|
||||
testQuery(
|
||||
"select c from foo, unnest(MV_TO_ARRAY(dim3)) as u(c) "
|
||||
+ " where m1=2 or __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
FilteredDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE1),
|
||||
or(
|
||||
useDefault ? equality("m1", 2, ColumnType.FLOAT) :
|
||||
equality("m1", 2.0, ColumnType.FLOAT),
|
||||
range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false)
|
||||
)
|
||||
),
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"b"},
|
||||
new Object[]{"c"},
|
||||
new Object[]{"d"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithTimeFilterOnlyNested()
|
||||
{
|
||||
testQuery(
|
||||
"select c from foo CROSS JOIN UNNEST(ARRAY[m1,m2]) as un(d) CROSS JOIN unnest(MV_TO_ARRAY(dim3)) as u(c)"
|
||||
+ " where __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
UnnestDataSource.create(
|
||||
FilteredDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE1),
|
||||
range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false)
|
||||
),
|
||||
expressionVirtualColumn("j0.unnest", "array(\"m1\",\"m2\")", ColumnType.FLOAT_ARRAY),
|
||||
null
|
||||
),
|
||||
expressionVirtualColumn("_j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z")))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("_j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"b"},
|
||||
new Object[]{"c"},
|
||||
new Object[]{"b"},
|
||||
new Object[]{"c"},
|
||||
new Object[]{"d"},
|
||||
new Object[]{"d"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithTimeFilterOnlyNestedAndNestedAgain()
|
||||
{
|
||||
testQuery(
|
||||
"select c from foo CROSS JOIN UNNEST(ARRAY[m1,m2]) as un(d) CROSS JOIN UNNEST(ARRAY[dim1,dim2]) as ud(a) "
|
||||
+ " CROSS JOIN unnest(MV_TO_ARRAY(dim3)) as u(c)"
|
||||
+ " where __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
UnnestDataSource.create(
|
||||
UnnestDataSource.create(
|
||||
FilteredDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE1),
|
||||
range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false)
|
||||
),
|
||||
expressionVirtualColumn("j0.unnest", "array(\"m1\",\"m2\")", ColumnType.FLOAT_ARRAY),
|
||||
null
|
||||
),
|
||||
expressionVirtualColumn("_j0.unnest", "array(\"dim1\",\"dim2\")", ColumnType.STRING_ARRAY),
|
||||
null
|
||||
),
|
||||
expressionVirtualColumn("__j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z")))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("__j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"b"},
|
||||
new Object[]{"c"},
|
||||
new Object[]{"b"},
|
||||
new Object[]{"c"},
|
||||
new Object[]{"b"},
|
||||
new Object[]{"c"},
|
||||
new Object[]{"b"},
|
||||
new Object[]{"c"},
|
||||
new Object[]{"d"},
|
||||
new Object[]{"d"},
|
||||
new Object[]{"d"},
|
||||
new Object[]{"d"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithTimeFilterInsideSubquery()
|
||||
{
|
||||
testQuery(
|
||||
"select d3 from (select * from foo, UNNEST(MV_TO_ARRAY(dim3)) as u(d3)"
|
||||
+ " where __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00' LIMIT 2) \n"
|
||||
+ " where m1 IN (1,2)",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
new QueryDataSource(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(
|
||||
UnnestDataSource.create(
|
||||
FilteredDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE1),
|
||||
range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false)
|
||||
),
|
||||
expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING),
|
||||
null
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Intervals.of(
|
||||
"2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z")))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.columns("j0.unnest", "m1")
|
||||
.limit(2)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.filters(
|
||||
NullHandling.sqlCompatible() ?
|
||||
or(
|
||||
equality("m1", 1.0f, ColumnType.FLOAT),
|
||||
equality("m1", 2.0f, ColumnType.FLOAT)
|
||||
) :
|
||||
new InDimFilter("m1", ImmutableList.of("1", "2"), null)
|
||||
)
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"b"},
|
||||
new Object[]{"c"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithFilterAndUnnestNestedBackToBack()
|
||||
{
|
||||
testQuery(
|
||||
"SELECT m1, dim3_unnest1, dim3_unnest2, dim3_unnest3 FROM \n"
|
||||
+ " ( SELECT * FROM \n"
|
||||
+ " ( SELECT * FROM foo, UNNEST(MV_TO_ARRAY(dim3)) as ut(dim3_unnest1) ), \n"
|
||||
+ " UNNEST(MV_TO_ARRAY(dim3)) as ut(dim3_unnest2) \n"
|
||||
+ " ), UNNEST(MV_TO_ARRAY(dim3)) as ut(dim3_unnest3) "
|
||||
+ " WHERE m1=2 AND (dim3_unnest1='a' OR dim3_unnest2='b') AND dim3_unnest3='c' "
|
||||
+ " AND __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
UnnestDataSource.create(
|
||||
FilteredDataSource.create(
|
||||
UnnestDataSource.create(
|
||||
FilteredDataSource.create(
|
||||
UnnestDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE1),
|
||||
expressionVirtualColumn(
|
||||
"j0.unnest",
|
||||
"\"dim3\"",
|
||||
ColumnType.STRING
|
||||
),
|
||||
null
|
||||
),
|
||||
NullHandling.sqlCompatible() ?
|
||||
and(
|
||||
equality("m1", 2.0f, ColumnType.FLOAT),
|
||||
range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false)
|
||||
) :
|
||||
and(
|
||||
selector("m1", "2", null),
|
||||
bound(
|
||||
"__time",
|
||||
"946771200000",
|
||||
"946858200000",
|
||||
false,
|
||||
false,
|
||||
null,
|
||||
StringComparators.NUMERIC
|
||||
)
|
||||
)
|
||||
),
|
||||
expressionVirtualColumn(
|
||||
"_j0.unnest",
|
||||
"\"dim3\"",
|
||||
ColumnType.STRING
|
||||
),
|
||||
null
|
||||
),
|
||||
or(
|
||||
equality("j0.unnest", "a", ColumnType.STRING),
|
||||
equality("_j0.unnest", "b", ColumnType.STRING)
|
||||
)
|
||||
),
|
||||
expressionVirtualColumn(
|
||||
"__j0.unnest",
|
||||
"\"dim3\"",
|
||||
ColumnType.STRING
|
||||
),
|
||||
equality("__j0.unnest", "c", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z")))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.virtualColumns(expressionVirtualColumn("v0", "2.0", ColumnType.FLOAT))
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "j0.unnest", "v0"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{2.0f, "b", "b", "c"},
|
||||
new Object[]{2.0f, "c", "b", "c"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnnestWithLookup()
|
||||
{
|
||||
testQuery(
|
||||
"SELECT * FROM lookup.lookyloo, unnest(mv_to_array(v)) as u(d) where k='a'",
|
||||
QUERY_CONTEXT_UNNEST,
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(UnnestDataSource.create(
|
||||
FilteredDataSource.create(
|
||||
new LookupDataSource("lookyloo"),
|
||||
equality("k", "a", ColumnType.STRING)
|
||||
),
|
||||
expressionVirtualColumn("j0.unnest", "\"v\"", ColumnType.STRING),
|
||||
null
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.virtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING))
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_UNNEST)
|
||||
.columns(ImmutableList.of("j0.unnest", "v", "v0"))
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"a", "xa", "xa"}
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1504,7 +1504,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
|
|||
e,
|
||||
new DruidExceptionMatcher(DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "general")
|
||||
.expectMessageIs(
|
||||
"Query planning failed for unknown reason, our best guess is this "
|
||||
"Query could not be planned. A possible reason is "
|
||||
+ "[LATEST and EARLIEST aggregators implicitly depend on the __time column, "
|
||||
+ "but the table queried doesn't contain a __time column. "
|
||||
+ "Please use LATEST_BY or EARLIEST_BY and specify the column explicitly.]"
|
||||
|
|
|
@ -1390,7 +1390,7 @@ public class SqlResourceTest extends CalciteTestBase
|
|||
"general",
|
||||
DruidException.Persona.ADMIN,
|
||||
DruidException.Category.INVALID_INPUT,
|
||||
"Query planning failed for unknown reason, our best guess is this "
|
||||
"Query could not be planned. A possible reason is "
|
||||
+ "[SQL query requires order by non-time column [[dim1 ASC]], which is not supported.]"
|
||||
);
|
||||
checkSqlRequestLog(false);
|
||||
|
|
Loading…
Reference in New Issue