remove native scan query legacy mode (#16659)

This commit is contained in:
Clint Wylie 2024-07-18 23:33:27 -07:00 committed by GitHub
parent 0a274d56a1
commit 35b876436b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
232 changed files with 185 additions and 1251 deletions

View File

@ -262,12 +262,8 @@ public class ScanBenchmark
rowsPerSegment
);
final ScanQueryConfig config = new ScanQueryConfig().setLegacy(false);
factory = new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(
config,
DefaultGenericQueryMetricsFactory.instance()
),
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
);

View File

@ -474,7 +474,6 @@ The response shows an example report for a query.
"agent_type",
"timestamp"
],
"legacy": false,
"context": {
"finalize": false,
"finalizeAggregations": false,

View File

@ -217,7 +217,6 @@ The following shows an example log emitter output:
"user",
"v0"
],
"legacy": false,
"context":
{
"populateCache": false,

View File

@ -66,7 +66,6 @@ The following are the main parameters for Scan queries:
|limit|How many rows to return. If not specified, all rows will be returned.|no|
|offset|Skip this many rows when returning results. Skipped rows will still need to be generated internally and then discarded, meaning that raising offsets to high values can cause queries to use additional resources.<br /><br />Together, "limit" and "offset" can be used to implement pagination. However, note that if the underlying datasource is modified in between page fetches in ways that affect overall query results, then the different pages will not necessarily align with each other.|no|
|order|The ordering of returned rows based on timestamp. "ascending", "descending", and "none" (default) are supported. Currently, "ascending" and "descending" are only supported for queries where the `__time` column is included in the `columns` field and the requirements outlined in the [time ordering](#time-ordering) section are met.|none|
|legacy|Return results consistent with the legacy "scan-query" contrib extension. Defaults to the value set by `druid.query.scan.legacy`, which in turn defaults to false. See [Legacy mode](#legacy-mode) for details.|no|
|context|An additional JSON Object which can be used to specify certain flags (see the `query context properties` section below).|no|
## Example results
@ -159,14 +158,14 @@ The format of the result when resultFormat equals `compactedList`:
## Time ordering
The Scan query currently supports ordering based on timestamp for non-legacy queries. Note that using time ordering
will yield results that do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore,
time ordering is only supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForOrdering`
rows **or** all segments scanned have fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` partitions. Also,
time ordering is not supported for queries issued directly to historicals unless a list of segments is specified. The
reasoning behind these limitations is that the implementation of time ordering uses two strategies that can consume too
much heap memory if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on
query result set limit and the number of segments being scanned.
The Scan query currently supports ordering based on timestamp. Note that using time ordering will yield results that
do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore, time ordering is only
supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForOrdering` rows **or** all segments
scanned have fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` partitions. Also, time ordering is not
supported for queries issued directly to historicals unless a list of segments is specified. The reasoning behind
these limitations is that the implementation of time ordering uses two strategies that can consume too much heap memory
if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on query result set
limit and the number of segments being scanned.
1. Priority Queue: Each segment on a Historical is opened sequentially. Every row is added to a bounded priority
queue which is ordered by timestamp. For every row above the result set limit, the row with the earliest (if descending)
@ -187,21 +186,6 @@ configurable and can be tuned based on hardware specs and number of dimensions b
can also be overridden using the `maxRowsQueuedForOrdering` and `maxSegmentPartitionsOrderedInMemory` properties in
the query context (see the Query Context Properties section).
## Legacy mode
The Scan query supports a legacy mode designed for protocol compatibility with the former scan-query contrib extension.
In legacy mode you can expect the following behavior changes:
- The `__time` column is returned as `"timestamp"` rather than `"__time"`. This will take precedence over any other column
you may have that is named `"timestamp"`.
- The `__time` column is included in the list of columns even if you do not specifically ask for it.
- Timestamps are returned as ISO8601 time strings rather than integers (milliseconds since 1970-01-01 00:00:00 UTC).
Legacy mode can be triggered either by passing `"legacy" : true` in your query JSON, or by setting
`druid.query.scan.legacy = true` on your Druid processes. If you were previously using the scan-query contrib extension,
the best way to migrate is to activate legacy mode during a rolling upgrade, then switch it off after the upgrade
is complete.
## Configuration Properties
Configuration properties:
@ -210,7 +194,6 @@ Configuration properties:
|--------|-----------|------|-------|
|druid.query.scan.maxRowsQueuedForOrdering|The maximum number of rows returned when time ordering is used|An integer in [1, 2147483647]|100000|
|druid.query.scan.maxSegmentPartitionsOrderedInMemory|The maximum number of segments scanned per historical when time ordering is used|An integer in [1, 2147483647]|50|
|druid.query.scan.legacy|Whether legacy mode should be turned on for Scan queries|true or false|false|
## Query context properties
@ -228,3 +211,7 @@ Sample query context JSON object:
"maxSegmentPartitionsOrderedInMemory": 100
}
```
## Legacy mode
In older versions of Druid, the scan query supported a legacy mode designed for protocol compatibility with the former scan-query contrib extension from versions of Druid older than 0.11. This mode has been removed.

View File

@ -342,7 +342,6 @@ The above EXPLAIN PLAN returns the following result:
"regionName",
"v0"
],
"legacy": false,
"context": {
"finalizeAggregations": false,
"forceExpressionVirtualColumns": true,
@ -562,7 +561,6 @@ The above EXPLAIN PLAN query returns the following result:
"regionName",
"v0"
],
"legacy": false,
"context": {
"finalizeAggregations": false,
"groupByEnableMultiValueUnnesting": false,

View File

@ -304,7 +304,6 @@ The following native Scan query returns the rows of the datasource and unnests t
"m2",
"unnest-dim3"
],
"legacy": false,
"granularity": {
"type": "all"
},
@ -480,7 +479,6 @@ This query joins the `nested_data` table with itself and outputs the unnested da
"m2",
"v0"
],
"legacy": false,
"context": {
"sqlOuterLimit": 1001,
"useNativeQueryExplain": true
@ -523,7 +521,6 @@ This query joins the `nested_data` table with itself and outputs the unnested da
"m2",
"unnest-dim3"
],
"legacy": false,
"context": {
"sqlOuterLimit": 1001,
"useNativeQueryExplain": true
@ -572,7 +569,6 @@ The following query returns the columns `dim45` and `m1`. The `dim45` column is
"dim45",
"m1"
],
"legacy": false,
"granularity": {
"type": "all"
},
@ -632,7 +628,6 @@ The following Scan query unnests the column `dim3` into `d3` and a virtual colum
"dim5",
"d45"
],
"legacy": false,
"context": {
"queryId": "2618b9ce-6c0d-414e-b88d-16fb59b9c481",
"sqlOuterLimit": 1001,

View File

@ -2908,10 +2908,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
.put(
ScanQuery.class,
new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(
new ScanQueryConfig(),
new DefaultGenericQueryMetricsFactory()
),
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)

View File

@ -359,7 +359,6 @@ public class MSQCompactionRunner implements CompactionRunner
.virtualColumns(getVirtualColumns(dataSchema, interval))
.columnTypes(rowSignature.getColumnTypes())
.intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval)))
.legacy(false)
.filters(dataSchema.getTransformSpec().getFilter())
.context(compactionTask.getContext())
.build();

View File

@ -149,12 +149,6 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor
@Override
public ReturnOrAwait<Object> runIncrementally(final IntSet readableInputs) throws IOException
{
final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter");
if (legacy) {
throw new ISE("Cannot use this engine in legacy mode");
}
if (runningCountForLimit != null
&& runningCountForLimit.get() > query.getScanRowsOffset() + query.getScanRowsLimit()) {
return ReturnOrAwait.returnObject(Unit.instance());

View File

@ -132,7 +132,7 @@ public class DataServerQueryHandlerTest
.build();
QueryToolChestWarehouse queryToolChestWarehouse = new MapQueryToolChestWarehouse(
ImmutableMap.<Class<? extends Query>, QueryToolChest>builder()
.put(ScanQuery.class, new ScanQueryQueryToolChest(null, null))
.put(ScanQuery.class, new ScanQueryQueryToolChest(null))
.build()
);
target = spy(

View File

@ -2594,7 +2594,6 @@ public class MSQSelectTest extends MSQTestBase
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(defaultScanQueryContext(
context,
resultSignature
@ -2668,7 +2667,6 @@ public class MSQSelectTest extends MSQTestBase
)
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.filters(equality("dim2", "a", ColumnType.STRING))
.columns("dim3")
.context(defaultScanQueryContext(
@ -2683,7 +2681,6 @@ public class MSQSelectTest extends MSQTestBase
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(defaultScanQueryContext(
context,
resultSignature
@ -2751,7 +2748,6 @@ public class MSQSelectTest extends MSQTestBase
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(defaultScanQueryContext(
context,
rowSignature

View File

@ -609,7 +609,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("m1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -678,7 +677,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("m1", "m2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -750,7 +748,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("m1", "m2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -896,7 +893,6 @@ public class MSQWindowTest extends MSQTestBase
.virtualColumns(expressionVirtualColumn("v0", "strlen(\"dim1\")", ColumnType.LONG))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -969,7 +965,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("m1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -1063,7 +1058,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("m2", "v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature1)
.legacy(false)
.build()
),
"j0.",
@ -1078,7 +1072,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("j0.m2", "m1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -1176,7 +1169,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("m2", "v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature1)
.legacy(false)
.build()
),
"j0.",
@ -1191,7 +1183,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("j0.m2", "m1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -1265,7 +1256,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("dim2", "m1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -1359,7 +1349,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("j0.unnest", "m1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -1450,7 +1439,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("j0.unnest", "m1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -1832,7 +1820,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("added", "cityName")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(contextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -1924,7 +1911,6 @@ public class MSQWindowTest extends MSQTestBase
.columns("added", "cityName", "countryIsoCode")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(innerContextWithRowSignature)
.legacy(false)
.build()),
new LegacySegmentSpec(Intervals.ETERNITY),
context,
@ -1954,7 +1940,6 @@ public class MSQWindowTest extends MSQTestBase
.limit(5)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(outerContextWithRowSignature)
.legacy(false)
.build();
testSelectQuery()

View File

@ -62,7 +62,6 @@ public class MSQControllerTaskTest
))
.query(new Druids.ScanQueryBuilder()
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.intervals(new MultipleIntervalSegmentSpec(INTERVALS))
.dataSource("target")
.build()

View File

@ -108,7 +108,6 @@ public class ScanQueryFrameProcessorTest extends InitializedNullHandlingTest
.dataSource("test")
.intervals(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY))
.columns(adapter.getRowSignature().getColumnNames())
.legacy(false)
.build();
final StagePartition stagePartition = new StagePartition(new StageId("query", 0), 0);

View File

@ -307,7 +307,6 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"resultFormat\":\"compactedList\","
+ "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],"
+ "\"legacy\":false,"
+ "\"context\":{\"__resultFormat\":\"object\",\"executionMode\":\"ASYNC\",\"scanSignature\":\"[{\\\"name\\\":\\\"__time\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"cnt\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"dim1\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim2\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim3\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"m1\\\",\\\"type\\\":\\\"FLOAT\\\"},{\\\"name\\\":\\\"m2\\\",\\\"type\\\":\\\"DOUBLE\\\"},{\\\"name\\\":\\\"unique_dim1\\\",\\\"type\\\":\\\"COMPLEX<hyperUnique>\\\"}]\",\"sqlQueryId\":\"queryId\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}],"
+ " RESOURCES=[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}],"
+ " ATTRIBUTES={\"statementType\":\"SELECT\"}}",

View File

@ -123,7 +123,6 @@ public class SqlStatementResourceTest extends MSQTestBase
private static final Query<?> QUERY = new Druids.ScanQueryBuilder().resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.intervals(new MultipleIntervalSegmentSpec(
Collections.singletonList(Intervals.of(
"2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z"))))

View File

@ -74,7 +74,6 @@ public class SleepSqlTest extends BaseCalciteQueryTest
.columns("v0")
.filters(range("m1", ColumnType.DOUBLE, null, 2.0, false, true))
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),

View File

@ -74,10 +74,7 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
protected static final Granularity SEGMENT_GRANULARITY = Granularities.DAY;
private static final ScanQueryRunnerFactory SCAN_QUERY_RUNNER_FACTORY = new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(
new ScanQueryConfig().setLegacy(false),
DefaultGenericQueryMetricsFactory.instance()
),
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
);
@ -293,7 +290,6 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
null,
null,
columns,
false,
null,
null
)

View File

@ -129,7 +129,6 @@
"delta",
"page"
],
"legacy": false,
"context": {
"__exportFileFormat": "CSV",
"__resultFormat": "array",

View File

@ -824,7 +824,6 @@ public class Druids
private long limit;
private DimFilter dimFilter;
private List<String> columns = new ArrayList<>();
private Boolean legacy;
private ScanQuery.Order order;
private List<ScanQuery.OrderBy> orderBy;
private List<ColumnType> columnTypes = null;
@ -843,7 +842,6 @@ public class Druids
orderBy,
dimFilter,
columns,
legacy,
context,
columnTypes
);
@ -861,7 +859,6 @@ public class Druids
.limit(query.getScanRowsLimit())
.filters(query.getFilter())
.columns(query.getColumns())
.legacy(query.isLegacy())
.context(query.getContext())
.orderBy(query.getOrderBys())
.columnTypes(query.getColumnTypes());
@ -959,12 +956,6 @@ public class Druids
return this;
}
public ScanQueryBuilder legacy(Boolean legacy)
{
this.legacy = legacy;
return this;
}
public ScanQueryBuilder order(ScanQuery.Order order)
{
this.order = order;

View File

@ -187,7 +187,6 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
private final long scanRowsLimit;
private final DimFilter dimFilter;
private final List<String> columns;
private final Boolean legacy;
private final Order timeOrder;
private final List<OrderBy> orderBys;
private final Integer maxRowsQueuedForOrdering;
@ -207,7 +206,6 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
@JsonProperty("orderBy") List<OrderBy> orderBysFromUser,
@JsonProperty("filter") DimFilter dimFilter,
@JsonProperty("columns") List<String> columns,
@JsonProperty("legacy") Boolean legacy,
@JsonProperty("context") Map<String, Object> context,
@JsonProperty("columnTypes") List<ColumnType> columnTypes
)
@ -232,7 +230,6 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
);
this.dimFilter = dimFilter;
this.columns = columns;
this.legacy = legacy;
this.columnTypes = columnTypes;
if (columnTypes != null) {
@ -446,21 +443,6 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
return columnTypes;
}
/**
* Compatibility mode with the legacy scan-query extension.
*
* True, false, and null have different meanings: true/false mean "legacy" and "not legacy"; null means use the
* default set by {@link ScanQueryConfig#isLegacy()}. The method {@link #withNonNullLegacy} is provided to help
* with this.
*/
@Nullable
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_NULL)
public Boolean isLegacy()
{
return legacy;
}
@Override
public Ordering<ScanResultValue> getResultOrdering()
{
@ -507,11 +489,6 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
return Druids.ScanQueryBuilder.copy(this).limit(newLimit).build();
}
public ScanQuery withNonNullLegacy(final ScanQueryConfig scanQueryConfig)
{
return Druids.ScanQueryBuilder.copy(this).legacy(legacy != null ? legacy : scanQueryConfig.isLegacy()).build();
}
@Override
public ScanQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec)
{
@ -546,7 +523,6 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
return batchSize == scanQuery.batchSize &&
scanRowsOffset == scanQuery.scanRowsOffset &&
scanRowsLimit == scanQuery.scanRowsLimit &&
Objects.equals(legacy, scanQuery.legacy) &&
Objects.equals(virtualColumns, scanQuery.virtualColumns) &&
Objects.equals(resultFormat, scanQuery.resultFormat) &&
Objects.equals(dimFilter, scanQuery.dimFilter) &&
@ -566,8 +542,7 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
scanRowsLimit,
dimFilter,
columns,
orderBys,
legacy
orderBys
);
}
@ -585,7 +560,6 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
", dimFilter=" + dimFilter +
", columns=" + columns +
(orderBys.isEmpty() ? "" : ", orderBy=" + orderBys) +
(legacy == null ? "" : ", legacy=" + legacy) +
", context=" + getContext() +
'}';
}
@ -710,12 +684,6 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
*/
@Nullable
public RowSignature getRowSignature()
{
return getRowSignature(false);
}
@Nullable
public RowSignature getRowSignature(boolean defaultIsLegacy)
{
if (columns == null || columns.isEmpty()) {
// Note: if no specific list of columns is provided, then since we can't predict what columns will come back, we
@ -731,15 +699,7 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
}
return builder.build();
}
return guessRowSignature(defaultIsLegacy);
}
private RowSignature guessRowSignature(boolean defaultIsLegacy)
{
final RowSignature.Builder builder = RowSignature.builder();
if (Boolean.TRUE.equals(legacy) || (legacy == null && defaultIsLegacy)) {
builder.add(ScanQueryEngine.LEGACY_TIMESTAMP_KEY, null);
}
DataSource dataSource = getDataSource();
for (String columnName : columns) {
final ColumnType columnType = guessColumnType(columnName, virtualColumns, dataSource);

View File

@ -29,38 +29,23 @@ public class ScanQueryConfig
public static final String CTX_KEY_MAX_SEGMENT_PARTITIONS_FOR_ORDERING = "maxSegmentPartitionsOrderedInMemory";
@JsonProperty
private boolean legacy = false;
public boolean isLegacy()
{
return legacy;
}
@SuppressWarnings("unused") // Used by Jackson deserialization?
public ScanQueryConfig setLegacy(final boolean legacy)
{
this.legacy = legacy;
return this;
}
private int maxRowsQueuedForOrdering = 100000;
@JsonProperty
private int maxRowsQueuedForOrdering = 100000;
private int maxSegmentPartitionsOrderedInMemory = 50;
public int getMaxRowsQueuedForOrdering()
{
return maxRowsQueuedForOrdering;
}
@JsonProperty
private int maxSegmentPartitionsOrderedInMemory = 50;
public int getMaxSegmentPartitionsOrderedInMemory()
{
return maxSegmentPartitionsOrderedInMemory;
}
@Override
public boolean equals(final Object o)
public boolean equals(Object o)
{
if (this == o) {
return true;
@ -68,21 +53,23 @@ public class ScanQueryConfig
if (o == null || getClass() != o.getClass()) {
return false;
}
final ScanQueryConfig that = (ScanQueryConfig) o;
return legacy == that.legacy;
ScanQueryConfig that = (ScanQueryConfig) o;
return maxRowsQueuedForOrdering == that.maxRowsQueuedForOrdering
&& maxSegmentPartitionsOrderedInMemory == that.maxSegmentPartitionsOrderedInMemory;
}
@Override
public int hashCode()
{
return Objects.hash(legacy);
return Objects.hash(maxRowsQueuedForOrdering, maxSegmentPartitionsOrderedInMemory);
}
@Override
public String toString()
{
return "ScanQueryConfig{" +
"legacy=" + legacy +
"maxRowsQueuedForOrdering=" + maxRowsQueuedForOrdering +
", maxSegmentPartitionsOrderedInMemory=" + maxSegmentPartitionsOrderedInMemory +
'}';
}
}

View File

@ -23,7 +23,6 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
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.UOE;
@ -36,6 +35,7 @@ import org.apache.druid.query.QueryTimeoutException;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.VirtualColumn;
@ -59,8 +59,6 @@ import java.util.Set;
public class ScanQueryEngine
{
static final String LEGACY_TIMESTAMP_KEY = "timestamp";
public Sequence<ScanResultValue> process(
final ScanQuery query,
final Segment segment,
@ -68,10 +66,6 @@ public class ScanQueryEngine
@Nullable final QueryMetrics<?> queryMetrics
)
{
// "legacy" should be non-null due to toolChest.mergeResults
final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter");
final Long numScannedRows = responseContext.getRowScanCount();
if (numScannedRows != null && numScannedRows >= query.getScanRowsLimit() && query.getTimeOrder().equals(ScanQuery.Order.NONE)) {
return Sequences.empty();
@ -93,9 +87,6 @@ public class ScanQueryEngine
final List<String> allColumns = new ArrayList<>();
if (query.getColumns() != null && !query.getColumns().isEmpty()) {
if (legacy && !query.getColumns().contains(LEGACY_TIMESTAMP_KEY)) {
allColumns.add(LEGACY_TIMESTAMP_KEY);
}
// Unless we're in legacy mode, allColumns equals query.getColumns() exactly. This is nice since it makes
// the compactedList form easier to use.
@ -103,7 +94,7 @@ public class ScanQueryEngine
} else {
final Set<String> availableColumns = Sets.newLinkedHashSet(
Iterables.concat(
Collections.singleton(legacy ? LEGACY_TIMESTAMP_KEY : ColumnHolder.TIME_COLUMN_NAME),
Collections.singleton(ColumnHolder.TIME_COLUMN_NAME),
Iterables.transform(
Arrays.asList(query.getVirtualColumns().getVirtualColumns()),
VirtualColumn::getOutputName
@ -114,10 +105,6 @@ public class ScanQueryEngine
);
allColumns.addAll(availableColumns);
if (legacy) {
allColumns.remove(ColumnHolder.TIME_COLUMN_NAME);
}
}
final List<Interval> intervals = query.getQuerySegmentSpec().getIntervals();
@ -149,28 +136,15 @@ public class ScanQueryEngine
{
final List<BaseObjectColumnValueSelector> columnSelectors = new ArrayList<>(allColumns.size());
final RowSignature.Builder rowSignatureBuilder = RowSignature.builder();
final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory();
for (String column : allColumns) {
final BaseObjectColumnValueSelector selector;
if (legacy && LEGACY_TIMESTAMP_KEY.equals(column)) {
selector = cursor.getColumnSelectorFactory()
.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME);
ColumnCapabilities columnCapabilities = cursor.getColumnSelectorFactory()
.getColumnCapabilities(ColumnHolder.TIME_COLUMN_NAME);
rowSignatureBuilder.add(
column,
columnCapabilities == null ? null : columnCapabilities.toColumnType()
);
} else {
selector = cursor.getColumnSelectorFactory().makeColumnValueSelector(column);
ColumnCapabilities columnCapabilities = cursor.getColumnSelectorFactory()
.getColumnCapabilities(column);
rowSignatureBuilder.add(
column,
columnCapabilities == null ? null : columnCapabilities.toColumnType()
);
}
final BaseObjectColumnValueSelector selector = factory.makeColumnValueSelector(column);
ColumnCapabilities columnCapabilities = factory.getColumnCapabilities(column);
rowSignatureBuilder.add(
column,
columnCapabilities == null ? null : columnCapabilities.toColumnType()
);
columnSelectors.add(selector);
}
@ -246,14 +220,7 @@ public class ScanQueryEngine
private Object getColumnValue(int i)
{
final BaseObjectColumnValueSelector selector = columnSelectors.get(i);
final Object value;
if (legacy && allColumns.get(i).equals(LEGACY_TIMESTAMP_KEY)) {
value = DateTimes.utc((long) selector.getObject());
} else {
value = selector == null ? null : selector.getObject();
}
final Object value = selector == null ? null : selector.getObject();
return value;
}
};

View File

@ -50,16 +50,13 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
{
};
private final ScanQueryConfig scanQueryConfig;
private final GenericQueryMetricsFactory queryMetricsFactory;
@Inject
public ScanQueryQueryToolChest(
final ScanQueryConfig scanQueryConfig,
final GenericQueryMetricsFactory queryMetricsFactory
)
{
this.scanQueryConfig = scanQueryConfig;
this.queryMetricsFactory = queryMetricsFactory;
}
@ -86,10 +83,7 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
newLimit = originalQuery.getScanRowsLimit() + originalQuery.getScanRowsOffset();
}
// Ensure "legacy" is a non-null value, such that all other nodes this query is forwarded to will treat it
// the same way, even if they have different default legacy values.
final ScanQuery queryToRun = originalQuery.withNonNullLegacy(scanQueryConfig)
.withOffset(0)
final ScanQuery queryToRun = originalQuery.withOffset(0)
.withLimit(newLimit);
final Sequence<ScanResultValue> results;
@ -154,8 +148,7 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
@Override
public RowSignature resultArraySignature(final ScanQuery query)
{
boolean defaultIsLegacy = scanQueryConfig.isLegacy();
return query.getRowSignature(defaultIsLegacy);
return query.getRowSignature();
}
/**

View File

@ -91,7 +91,6 @@ public class DoubleStorageTest extends InitializedNullHandlingTest
);
private static final ScanQueryQueryToolChest SCAN_QUERY_QUERY_TOOL_CHEST = new ScanQueryQueryToolChest(
new ScanQueryConfig(),
DefaultGenericQueryMetricsFactory.instance()
);
@ -107,8 +106,7 @@ public class DoubleStorageTest extends InitializedNullHandlingTest
.dataSource(new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE))
.columns(Collections.emptyList())
.intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC)
.limit(Integer.MAX_VALUE)
.legacy(false);
.limit(Integer.MAX_VALUE);
}

View File

@ -278,7 +278,6 @@ public class AggregationTestHelper implements Closeable
ObjectMapper mapper = TestHelper.makeJsonMapper();
ScanQueryQueryToolChest toolchest = new ScanQueryQueryToolChest(
new ScanQueryConfig(),
DefaultGenericQueryMetricsFactory.instance()
);

View File

@ -69,7 +69,6 @@ import java.util.List;
public class MultiSegmentScanQueryTest extends InitializedNullHandlingTest
{
private static final ScanQueryQueryToolChest TOOL_CHEST = new ScanQueryQueryToolChest(
new ScanQueryConfig(),
DefaultGenericQueryMetricsFactory.instance()
);
@ -193,7 +192,6 @@ public class MultiSegmentScanQueryTest extends InitializedNullHandlingTest
.intervals(I_0112_0114_SPEC)
.batchSize(batchSize)
.columns(Collections.emptyList())
.legacy(false)
.limit(limit)
.offset(offset);
}

View File

@ -33,12 +33,10 @@ public class ScanQueryConfigTest
.<String, String>builder()
.put("maxSegmentPartitionsOrderedInMemory", "1")
.put("maxRowsQueuedForOrdering", "1")
.put("legacy", "true")
.build();
private final ImmutableMap<String, String> CONFIG_MAP2 = ImmutableMap
.<String, String>builder()
.put("legacy", "false")
.put("maxSegmentPartitionsOrderedInMemory", "42")
.build();
@ -52,16 +50,13 @@ public class ScanQueryConfigTest
final ScanQueryConfig config = MAPPER.convertValue(CONFIG_MAP, ScanQueryConfig.class);
Assert.assertEquals(1, config.getMaxRowsQueuedForOrdering());
Assert.assertEquals(1, config.getMaxSegmentPartitionsOrderedInMemory());
Assert.assertTrue(config.isLegacy());
final ScanQueryConfig config2 = MAPPER.convertValue(CONFIG_MAP2, ScanQueryConfig.class);
Assert.assertEquals(100000, config2.getMaxRowsQueuedForOrdering());
Assert.assertEquals(42, config2.getMaxSegmentPartitionsOrderedInMemory());
Assert.assertFalse(config2.isLegacy());
final ScanQueryConfig config3 = MAPPER.convertValue(CONFIG_MAP_EMPTY, ScanQueryConfig.class);
Assert.assertEquals(100000, config3.getMaxRowsQueuedForOrdering());
Assert.assertEquals(50, config3.getMaxSegmentPartitionsOrderedInMemory());
Assert.assertFalse(config3.isLegacy());
}
}

View File

@ -72,8 +72,7 @@ public class ScanQueryQueryToolChestTest
);
private final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest(
new ScanQueryConfig(),
new DefaultGenericQueryMetricsFactory()
DefaultGenericQueryMetricsFactory.instance()
);
@Test
@ -95,7 +94,6 @@ public class ScanQueryQueryToolChestTest
Druids.newScanQueryBuilder()
.dataSource("foo")
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000"))))
.legacy(true)
.build();
Assert.assertEquals(RowSignature.empty(), toolChest.resultArraySignature(scanQuery));
@ -117,23 +115,6 @@ public class ScanQueryQueryToolChestTest
);
}
@Test
public void test_resultArraySignature_columnsSpecifiedLegacyMode()
{
final ScanQuery scanQuery =
Druids.newScanQueryBuilder()
.dataSource("foo")
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000"))))
.columns("foo", "bar")
.legacy(true)
.build();
Assert.assertEquals(
RowSignature.builder().add("timestamp", null).add("foo", null).add("bar", null).build(),
toolChest.resultArraySignature(scanQuery)
);
}
@Test
public void test_resultsAsArrays_columnsNotSpecifiedListResults()
{

View File

@ -190,10 +190,7 @@ public class ScanQueryResultOrderingTest
public void setUp()
{
queryRunnerFactory = new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(
new ScanQueryConfig(),
new DefaultGenericQueryMetricsFactory()
),
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
);

View File

@ -74,10 +74,7 @@ public class ScanQueryRunnerFactoryTest
};
private static final ScanQueryRunnerFactory FACTORY = new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(
CONFIG,
DefaultGenericQueryMetricsFactory.instance()
),
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
CONFIG
);

View File

@ -62,7 +62,6 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -138,7 +137,6 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class);
private static final ScanQueryQueryToolChest TOOL_CHEST = new ScanQueryQueryToolChest(
new ScanQueryConfig(),
DefaultGenericQueryMetricsFactory.instance()
);
@ -148,28 +146,25 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
new ScanQueryConfig()
);
@Parameterized.Parameters(name = "{0}, legacy = {1}")
@Parameterized.Parameters(name = "{0}")
public static Iterable<Object[]> constructorFeeder()
{
return QueryRunnerTestHelper.cartesian(
return Iterables.transform(
QueryRunnerTestHelper.makeQueryRunners(
FACTORY
),
ImmutableList.of(false, true)
(runner) -> new Object[]{runner}
);
}
private final QueryRunner runner;
private final boolean legacy;
private final List<String> columns;
public ScanQueryRunnerTest(final QueryRunner runner, final boolean legacy)
public ScanQueryRunnerTest(final QueryRunner runner)
{
this.runner = runner;
this.legacy = legacy;
this.columns = Lists.newArrayList(
getTimestampName(),
ColumnHolder.TIME_COLUMN_NAME,
"expr",
"market",
"quality",
@ -200,8 +195,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
.dataSource(new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE))
.columns(Collections.emptyList())
.eternityInterval()
.limit(3)
.legacy(legacy);
.limit(3);
}
@Test
@ -270,7 +264,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
final List<List<Map<String, Object>>> expectedEvents = toEvents(
new String[]{
getTimestampName() + ":TIME",
ColumnHolder.TIME_COLUMN_NAME + ":TIME",
QueryRunnerTestHelper.MARKET_DIMENSION + ":STRING",
null,
null,
@ -284,24 +278,12 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
null,
QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE"
},
legacy,
V_0112_0114
);
// Add "__time" to all the expected events in legacy mode
if (legacy) {
for (List<Map<String, Object>> batch : expectedEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", ((DateTime) event.get("timestamp")).getMillis());
}
}
}
List<ScanResultValue> expectedResults = toExpected(
expectedEvents,
legacy
? Lists.newArrayList(getTimestampName(), "__time", "market", "index")
: Lists.newArrayList("__time", "market", "index"),
Lists.newArrayList("__time", "market", "index"),
0,
3
);
@ -321,7 +303,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
List<ScanResultValue> expectedResults = toExpected(
toEvents(
new String[]{
legacy ? getTimestampName() + ":TIME" : null,
null,
QueryRunnerTestHelper.MARKET_DIMENSION + ":STRING",
null,
null,
@ -335,10 +317,9 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
null,
QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE"
},
legacy,
V_0112_0114
),
legacy ? Lists.newArrayList(getTimestampName(), "market", "index") : Lists.newArrayList("market", "index"),
Lists.newArrayList("market", "index"),
0,
3
);
@ -359,7 +340,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
List<ScanResultValue> expectedResults = toExpected(
toEvents(
new String[]{
legacy ? getTimestampName() + ":TIME" : null,
null,
QueryRunnerTestHelper.MARKET_DIMENSION + ":STRING",
null,
null,
@ -373,10 +354,9 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
null,
QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE"
},
legacy,
V_0112_0114
),
legacy ? Lists.newArrayList(getTimestampName(), "market", "index") : Lists.newArrayList("market", "index"),
Lists.newArrayList("market", "index"),
0,
3
);
@ -399,14 +379,13 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
final List<List<Map<String, Object>>> events = toEvents(
new String[]{
legacy ? getTimestampName() + ":TIME" : null,
null,
null,
QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING",
null,
null,
QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE"
},
legacy,
// filtered values with day granularity
new String[]{
"2011-01-12T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t100.000000",
@ -434,7 +413,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
List<ScanResultValue> expectedResults = toExpected(
events,
legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"),
Lists.newArrayList("quality", "index"),
0,
limit
);
@ -463,14 +442,13 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
final List<List<Map<String, Object>>> events = toEvents(
new String[]{
legacy ? getTimestampName() + ":TIME" : null,
null,
null,
QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING",
null,
null,
QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE"
},
legacy,
// filtered values with day granularity
new String[]{
"2011-01-12T00:00:00.000Z\ttotal_market\tmezzanine\tpreferred\tmpreferred\t1000.000000",
@ -484,11 +462,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
List<ScanResultValue> expectedResults = toExpected(
events,
legacy ? Lists.newArrayList(
getTimestampName(),
QueryRunnerTestHelper.QUALITY_DIMENSION,
QueryRunnerTestHelper.INDEX_METRIC
) : Lists.newArrayList(
Lists.newArrayList(
QueryRunnerTestHelper.QUALITY_DIMENSION,
QueryRunnerTestHelper.INDEX_METRIC
),
@ -533,14 +507,13 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
final List<List<Map<String, Object>>> events = toEvents(
legacy ? new String[]{getTimestampName() + ":TIME"} : new String[0],
legacy,
new String[0],
V_0112_0114
);
List<ScanResultValue> expectedResults = toExpected(
events,
legacy ? Lists.newArrayList(getTimestampName(), "foo", "foo2") : Lists.newArrayList("foo", "foo2"),
Lists.newArrayList("foo", "foo2"),
0,
3
);
@ -591,40 +564,23 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
};
final List<List<Map<String, Object>>> ascendingEvents = toEvents(
new String[]{
legacy ? getTimestampName() + ":TIME" : ColumnHolder.TIME_COLUMN_NAME,
ColumnHolder.TIME_COLUMN_NAME,
null,
QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING",
null,
null,
QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE"
},
legacy,
(String[]) ArrayUtils.addAll(seg1Results, seg2Results)
);
if (legacy) {
for (List<Map<String, Object>> batch : ascendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", ((DateTime) event.get("timestamp")).getMillis());
}
}
} else {
for (List<Map<String, Object>> batch : ascendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis());
}
for (List<Map<String, Object>> batch : ascendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis());
}
}
List<ScanResultValue> ascendingExpectedResults = toExpected(
ascendingEvents,
legacy ?
Lists.newArrayList(
QueryRunnerTestHelper.TIME_DIMENSION,
getTimestampName(),
"quality",
"index"
) :
Lists.newArrayList(
QueryRunnerTestHelper.TIME_DIMENSION,
"quality",
@ -681,39 +637,23 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
ArrayUtils.reverse(expectedRet);
final List<List<Map<String, Object>>> descendingEvents = toEvents(
new String[]{
legacy ? getTimestampName() + ":TIME" : ColumnHolder.TIME_COLUMN_NAME,
ColumnHolder.TIME_COLUMN_NAME,
null,
QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING",
null,
null,
QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE"
},
legacy,
expectedRet
);
if (legacy) {
for (List<Map<String, Object>> batch : descendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", ((DateTime) event.get("timestamp")).getMillis());
}
}
} else {
for (List<Map<String, Object>> batch : descendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis());
}
for (List<Map<String, Object>> batch : descendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis());
}
}
List<ScanResultValue> descendingExpectedResults = toExpected(
descendingEvents,
legacy ?
Lists.newArrayList(
QueryRunnerTestHelper.TIME_DIMENSION,
getTimestampName(),
// getTimestampName() always returns the legacy timestamp when legacy is true
"quality",
"index"
) :
Lists.newArrayList(
QueryRunnerTestHelper.TIME_DIMENSION,
"quality",
@ -770,39 +710,22 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
Iterable<ScanResultValue> results = runner.run(QueryPlus.wrap(query)).toList();
final List<List<Map<String, Object>>> ascendingEvents = toEvents(
new String[]{
legacy ? getTimestampName() + ":TIME" : ColumnHolder.TIME_COLUMN_NAME,
ColumnHolder.TIME_COLUMN_NAME,
null,
QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING",
null,
null,
QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE"
},
legacy,
(String[]) ArrayUtils.addAll(seg1Results, seg2Results)
);
if (legacy) {
for (List<Map<String, Object>> batch : ascendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", ((DateTime) event.get("timestamp")).getMillis());
}
}
} else {
for (List<Map<String, Object>> batch : ascendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", ((DateTimes.of((String) event.get("__time"))).getMillis()));
}
for (List<Map<String, Object>> batch : ascendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", ((DateTimes.of((String) event.get("__time"))).getMillis()));
}
}
List<ScanResultValue> ascendingExpectedResults = toExpected(
ascendingEvents,
legacy ?
Lists.newArrayList(
QueryRunnerTestHelper.TIME_DIMENSION,
getTimestampName(),
// getTimestampName() always returns the legacy timestamp when legacy is true
"quality",
"index"
) :
Lists.newArrayList(
QueryRunnerTestHelper.TIME_DIMENSION,
"quality",
@ -863,39 +786,22 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
ArrayUtils.reverse(expectedRet);
final List<List<Map<String, Object>>> descendingEvents = toEvents(
new String[]{
legacy ? getTimestampName() + ":TIME" : ColumnHolder.TIME_COLUMN_NAME,
ColumnHolder.TIME_COLUMN_NAME,
null,
QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING",
null,
null,
QueryRunnerTestHelper.INDEX_METRIC + ":DOUBLE"
},
legacy,
expectedRet //segments in reverse order from above
);
if (legacy) {
for (List<Map<String, Object>> batch : descendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", ((DateTime) event.get("timestamp")).getMillis());
}
}
} else {
for (List<Map<String, Object>> batch : descendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", ((DateTimes.of((String) event.get("__time"))).getMillis()));
}
for (List<Map<String, Object>> batch : descendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", ((DateTimes.of((String) event.get("__time"))).getMillis()));
}
}
List<ScanResultValue> descendingExpectedResults = toExpected(
descendingEvents,
legacy ?
Lists.newArrayList(
QueryRunnerTestHelper.TIME_DIMENSION,
getTimestampName(),
// getTimestampName() always returns the legacy timestamp when legacy is true
"quality",
"index"
) :
Lists.newArrayList(
QueryRunnerTestHelper.TIME_DIMENSION,
"quality",
@ -996,7 +902,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
{
return toEvents(
new String[]{
getTimestampName() + ":TIME",
ColumnHolder.TIME_COLUMN_NAME + ":TIME",
QueryRunnerTestHelper.MARKET_DIMENSION + ":STRING",
QueryRunnerTestHelper.QUALITY_DIMENSION + ":STRING",
"qualityLong" + ":LONG",
@ -1018,12 +924,11 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
"indexMaxFloat",
"quality_uniques"
},
legacy,
valueSet
);
}
public static List<List<Map<String, Object>>> toEvents(final String[] dimSpecs, boolean legacy, final String[]... valueSet)
public static List<List<Map<String, Object>>> toEvents(final String[] dimSpecs, final String[]... valueSet)
{
List<String> values = new ArrayList<>();
for (String[] vSet : valueSet) {
@ -1085,7 +990,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
if (specs.length == 1 || specs[1].equals("STRING")) {
eventVal = values1[i];
} else if (specs[1].equals("TIME")) {
eventVal = toTimestamp(values1[i], legacy);
eventVal = DateTimes.of(values1[i]).getMillis();
} else if (specs[1].equals("FLOAT")) {
try {
eventVal = values1[i].isEmpty() ? NullHandling.defaultFloatValue() : Float.valueOf(values1[i]);
@ -1127,20 +1032,6 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest
return events;
}
private static Object toTimestamp(final String value, boolean legacy)
{
if (legacy) {
return DateTimes.of(value);
} else {
return DateTimes.of(value).getMillis();
}
}
private String getTimestampName()
{
return legacy ? "timestamp" : ColumnHolder.TIME_COLUMN_NAME;
}
private List<ScanResultValue> toExpected(
List<List<Map<String, Object>>> targets,
List<String> columns,

View File

@ -67,7 +67,6 @@ public class ScanQuerySpecTest
null,
Arrays.asList("market", "quality", "index"),
null,
null,
null
);
@ -102,7 +101,6 @@ public class ScanQuerySpecTest
null,
Arrays.asList("market", "quality", "index", "__time"),
null,
null,
null
);
@ -141,7 +139,6 @@ public class ScanQuerySpecTest
null,
Arrays.asList("market", "quality", "index", "__time"),
null,
null,
null
);
@ -171,7 +168,6 @@ public class ScanQuerySpecTest
null,
Arrays.asList("market", "quality", "index"),
null,
null,
null
);

View File

@ -20,7 +20,6 @@
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.query.DefaultGenericQueryMetricsFactory;
import org.apache.druid.query.Druids;
@ -39,23 +38,17 @@ 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.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@RunWith(Parameterized.class)
public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
{
public static final QuerySegmentSpec I_0112_0114 = ScanQueryRunnerTest.I_0112_0114;
private static final ScanQueryQueryToolChest TOOL_CHEST = new ScanQueryQueryToolChest(
new ScanQueryConfig(),
DefaultGenericQueryMetricsFactory.instance()
);
private static final ScanQueryRunnerFactory FACTORY = new ScanQueryRunnerFactory(
@ -63,25 +56,7 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
new ScanQueryEngine(),
new ScanQueryConfig()
);
private final IncrementalIndex index;
private final boolean legacy;
public UnnestScanQueryRunnerTest(final IncrementalIndex index, final boolean legacy)
{
this.index = index;
this.legacy = legacy;
}
@Parameterized.Parameters(name = "{0}")
public static Iterable<Object[]> constructorFeeder()
{
NullHandling.initializeForTests();
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex();
final List<Object[]> constructors = new ArrayList<>();
constructors.add(new Object[]{rtIndex, true});
constructors.add(new Object[]{rtIndex, false});
return constructors;
}
private final IncrementalIndex index = TestIndex.getIncrementalTestIndex();
private Druids.ScanQueryBuilder newTestUnnestQuery()
{
@ -89,8 +64,7 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
.dataSource(QueryRunnerTestHelper.UNNEST_DATA_SOURCE)
.columns(Collections.emptyList())
.eternityInterval()
.limit(3)
.legacy(legacy);
.limit(3);
}
private Druids.ScanQueryBuilder newTestUnnestQueryWithFilterDataSource()
@ -99,8 +73,7 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
.dataSource(QueryRunnerTestHelper.UNNEST_FILTER_DATA_SOURCE)
.columns(Collections.emptyList())
.eternityInterval()
.limit(3)
.legacy(legacy);
.limit(3);
}
@Test
@ -123,38 +96,19 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
);
Iterable<ScanResultValue> results = queryRunner.run(QueryPlus.wrap(query)).toList();
String[] columnNames;
if (legacy) {
columnNames = new String[]{
getTimestampName() + ":TIME",
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
} else {
columnNames = new String[]{
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
}
String[] values;
if (legacy) {
values = new String[]{
"2011-01-12T00:00:00.000Z\ta",
"2011-01-12T00:00:00.000Z\tpreferred",
"2011-01-12T00:00:00.000Z\tb"
};
} else {
values = new String[]{
"a",
"preferred",
"b"
};
}
String[] columnNames = new String[]{
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
String[] values = new String[]{
"a",
"preferred",
"b"
};
final List<List<Map<String, Object>>> events = ScanQueryRunnerTest.toEvents(columnNames, legacy, values);
final List<List<Map<String, Object>>> events = ScanQueryRunnerTest.toEvents(columnNames, values);
List<ScanResultValue> expectedResults = toExpected(
events,
legacy
? Lists.newArrayList(getTimestampName(), QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST)
: Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST),
Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST),
0,
3
);
@ -181,38 +135,19 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
);
Iterable<ScanResultValue> results = queryRunner.run(QueryPlus.wrap(query)).toList();
String[] columnNames;
if (legacy) {
columnNames = new String[]{
getTimestampName() + ":TIME",
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
} else {
columnNames = new String[]{
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
}
String[] values;
if (legacy) {
values = new String[]{
"2011-01-12T00:00:00.000Z\ta",
"2011-01-12T00:00:00.000Z\tpreferred",
"2011-01-12T00:00:00.000Z\tb"
};
} else {
values = new String[]{
"a",
"preferred",
"b"
};
}
String[] columnNames = new String[]{
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
String[] values = new String[]{
"a",
"preferred",
"b"
};
final List<List<Map<String, Object>>> events = ScanQueryRunnerTest.toEvents(columnNames, legacy, values);
final List<List<Map<String, Object>>> events = ScanQueryRunnerTest.toEvents(columnNames, values);
List<ScanResultValue> expectedResults = toExpected(
events,
legacy
? Lists.newArrayList(getTimestampName(), QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST)
: Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST),
Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST),
0,
3
);
@ -237,7 +172,6 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
))
.columns(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST)
.eternityInterval()
.legacy(legacy)
.limit(3)
.build();
@ -251,38 +185,19 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
"rtIndexvc"
);
Iterable<ScanResultValue> results = vcrunner.run(QueryPlus.wrap(query)).toList();
String[] columnNames;
if (legacy) {
columnNames = new String[]{
getTimestampName() + ":TIME",
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
} else {
columnNames = new String[]{
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
}
String[] values;
if (legacy) {
values = new String[]{
"2011-01-12T00:00:00.000Z\ta",
"2011-01-12T00:00:00.000Z\tpreferred",
"2011-01-12T00:00:00.000Z\tb"
};
} else {
values = new String[]{
"a",
"preferred",
"b"
};
}
String[] columnNames = new String[]{
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
String[] values = new String[]{
"a",
"preferred",
"b"
};
final List<List<Map<String, Object>>> events = ScanQueryRunnerTest.toEvents(columnNames, legacy, values);
final List<List<Map<String, Object>>> events = ScanQueryRunnerTest.toEvents(columnNames, values);
List<ScanResultValue> expectedResults = toExpected(
events,
legacy
? Lists.newArrayList(getTimestampName(), QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST)
: Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST),
Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST),
0,
3
);
@ -307,7 +222,6 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
))
.columns(QueryRunnerTestHelper.MARKET_DIMENSION, QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST)
.eternityInterval()
.legacy(legacy)
.limit(4)
.build();
@ -322,46 +236,21 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
);
Iterable<ScanResultValue> results = vcrunner.run(QueryPlus.wrap(query)).toList();
String[] columnNames;
if (legacy) {
columnNames = new String[]{
getTimestampName() + ":TIME",
QueryRunnerTestHelper.MARKET_DIMENSION,
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
} else {
columnNames = new String[]{
QueryRunnerTestHelper.MARKET_DIMENSION,
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
}
String[] values;
if (legacy) {
values = new String[]{
"2011-01-12T00:00:00.000Z\tspot\tspot",
"2011-01-12T00:00:00.000Z\tspot\tautomotive",
"2011-01-12T00:00:00.000Z\tspot\tspot",
"2011-01-12T00:00:00.000Z\tspot\tbusiness",
};
} else {
values = new String[]{
"spot\tspot",
"spot\tautomotive",
"spot\tspot",
"spot\tbusiness"
};
}
String[] columnNames = new String[]{
QueryRunnerTestHelper.MARKET_DIMENSION,
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
String[] values = new String[]{
"spot\tspot",
"spot\tautomotive",
"spot\tspot",
"spot\tbusiness"
};
final List<List<Map<String, Object>>> events = ScanQueryRunnerTest.toEvents(columnNames, legacy, values);
final List<List<Map<String, Object>>> events = ScanQueryRunnerTest.toEvents(columnNames, values);
List<ScanResultValue> expectedResults = toExpected(
events,
legacy
? Lists.newArrayList(
getTimestampName(),
QueryRunnerTestHelper.MARKET_DIMENSION,
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
)
: Lists.newArrayList(
Lists.newArrayList(
QueryRunnerTestHelper.MARKET_DIMENSION,
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
),
@ -392,38 +281,19 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
);
Iterable<ScanResultValue> results = queryRunner.run(QueryPlus.wrap(query)).toList();
String[] columnNames;
if (legacy) {
columnNames = new String[]{
getTimestampName() + ":TIME",
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
} else {
columnNames = new String[]{
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
}
String[] values;
if (legacy) {
values = new String[]{
"2011-01-12T00:00:00.000Z\ta",
"2011-01-12T00:00:00.000Z\tpreferred",
"2011-01-12T00:00:00.000Z\tb"
};
} else {
values = new String[]{
"a",
"preferred",
"b"
};
}
String[] columnNames = new String[]{
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
String[] values = new String[]{
"a",
"preferred",
"b"
};
final List<List<Map<String, Object>>> events = ScanQueryRunnerTest.toEvents(columnNames, legacy, values);
final List<List<Map<String, Object>>> events = ScanQueryRunnerTest.toEvents(columnNames, values);
List<ScanResultValue> expectedResults = toExpected(
events,
legacy
? Lists.newArrayList(getTimestampName(), QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST)
: Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST),
Collections.singletonList(QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST),
0,
3
);
@ -453,47 +323,25 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
);
Iterable<ScanResultValue> results = queryRunner.run(QueryPlus.wrap(query)).toList();
String[] columnNames;
if (legacy) {
columnNames = new String[]{
getTimestampName() + ":TIME",
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
} else {
columnNames = new String[]{
ColumnHolder.TIME_COLUMN_NAME,
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
}
String[] values;
values = new String[]{
String[] columnNames = new String[]{
ColumnHolder.TIME_COLUMN_NAME,
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
};
String[] values = new String[]{
"2011-01-12T00:00:00.000Z\ta",
"2011-01-12T00:00:00.000Z\tpreferred",
"2011-01-12T00:00:00.000Z\tb"
};
final List<List<Map<String, Object>>> ascendingEvents = ScanQueryRunnerTest.toEvents(columnNames, legacy, values);
if (legacy) {
for (List<Map<String, Object>> batch : ascendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", ((DateTime) event.get("timestamp")).getMillis());
}
}
} else {
for (List<Map<String, Object>> batch : ascendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis());
}
final List<List<Map<String, Object>>> ascendingEvents = ScanQueryRunnerTest.toEvents(columnNames, values);
for (List<Map<String, Object>> batch : ascendingEvents) {
for (Map<String, Object> event : batch) {
event.put("__time", (DateTimes.of((String) event.get("__time"))).getMillis());
}
}
List<ScanResultValue> ascendingExpectedResults = toExpected(
ascendingEvents,
legacy ?
Lists.newArrayList(
QueryRunnerTestHelper.TIME_DIMENSION,
getTimestampName(),
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
) :
Lists.newArrayList(
QueryRunnerTestHelper.TIME_DIMENSION,
QueryRunnerTestHelper.PLACEMENTISH_DIMENSION_UNNEST
@ -505,12 +353,6 @@ public class UnnestScanQueryRunnerTest extends InitializedNullHandlingTest
ScanQueryRunnerTest.verify(ascendingExpectedResults, results);
}
private String getTimestampName()
{
return legacy ? "timestamp" : ColumnHolder.TIME_COLUMN_NAME;
}
private List<ScanResultValue> toExpected(
List<List<Map<String, Object>>> targets,
List<String> columns,

View File

@ -222,10 +222,7 @@ public class StreamAppenderatorTester implements AutoCloseable
QueryRunnerTestHelper.NOOP_QUERYWATCHER
),
ScanQuery.class, new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(
new ScanQueryConfig(),
new DefaultGenericQueryMetricsFactory()
),
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)
@ -269,10 +266,7 @@ public class StreamAppenderatorTester implements AutoCloseable
QueryRunnerTestHelper.NOOP_QUERYWATCHER
),
ScanQuery.class, new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(
new ScanQueryConfig(),
new DefaultGenericQueryMetricsFactory()
),
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)

View File

@ -685,7 +685,6 @@ public class ClientQuerySegmentWalkerTest
ScanQuery subquery = new Druids.ScanQueryBuilder().dataSource(MULTI)
.columns("s", "n")
.eternityInterval()
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build();
final GroupByQuery query =
@ -736,7 +735,6 @@ public class ClientQuerySegmentWalkerTest
ScanQuery subquery = new Druids.ScanQueryBuilder().dataSource(MULTI)
.columns("s", "n")
.eternityInterval()
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build();
final TopNQuery query =
@ -858,7 +856,6 @@ public class ClientQuerySegmentWalkerTest
.dataSource(FOO)
.intervals(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY))
.columns("s")
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build()
.withId(DUMMY_QUERY_ID);
@ -876,7 +873,6 @@ public class ClientQuerySegmentWalkerTest
)
)
.columns("v")
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build()
.withId(DUMMY_QUERY_ID);

View File

@ -343,10 +343,7 @@ public class QueryStackTests
.put(
ScanQuery.class,
new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(
new ScanQueryConfig(),
new DefaultGenericQueryMetricsFactory()
),
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)

View File

@ -1678,7 +1678,6 @@ public class DruidQuery
orderByColumns,
filtration.getDimFilter(),
scanColumnsList,
false,
withScanSignatureIfNeeded(
virtualColumns,
scanColumnsList,

View File

@ -608,8 +608,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
public static Druids.ScanQueryBuilder newScanQueryBuilder()
{
return new Druids.ScanQueryBuilder().resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false);
return new Druids.ScanQueryBuilder().resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST);
}
protected static DruidExceptionMatcher invalidSqlIs(String s)

View File

@ -1490,7 +1490,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.columns(ImmutableList.of("v0"))
.context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -1522,7 +1521,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.columns("v0", "v1", "v2")
.context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -3493,7 +3491,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.columns("dim4", "j0.a0", "v0")
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
@ -3840,7 +3837,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.columns("dim1", "dim2")
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
@ -3950,7 +3946,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of(
"EXPR$0"
@ -4010,7 +4005,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -4054,7 +4048,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -4148,7 +4141,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -4194,7 +4186,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -4247,7 +4238,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -4296,7 +4286,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -4349,7 +4338,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -4431,7 +4419,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("_j0.unnest", "dim1", "j0.unnest", "v0", "v1"))
.build()
@ -4495,7 +4482,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("_j0.unnest", "arrayLongNulls", "arrayStringNulls", "j0.unnest"))
.build()
@ -4595,7 +4581,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("dim1", "j0.unnest", "v0", "v1", "v2"))
.build()
@ -4656,7 +4641,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.virtualColumns(expressionVirtualColumn(
"v0",
@ -4758,7 +4742,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
"'Baz'",
ColumnType.STRING
))
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "dimZipf", "v0"))
.build()
@ -4826,7 +4809,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
expressionVirtualColumn("v0", "array('a','b')", ColumnType.STRING_ARRAY),
expressionVirtualColumn("v1", "1", ColumnType.LONG)
)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "v0", "v1"))
.build()
@ -4897,7 +4879,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "dimZipf", "j0.unnest"))
.build()
@ -4974,7 +4955,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.virtualColumns(
expressionVirtualColumn("v0", "array('a','b')", ColumnType.STRING_ARRAY)
)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "j0.unnest", "v0"))
.build()
@ -5204,7 +5184,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.limit(3)
@ -5233,7 +5212,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -5280,7 +5258,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.filters(or(
NullHandling.sqlCompatible()
? range("m1", ColumnType.LONG, null, "10", false, false)
@ -5320,7 +5297,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false) // (j0.unnest = a || (m1 = 2 && j0.unnest = b))
// (j0.unnest = a || (m1 = 2 && j0.unnest = b))
.filters(or(
equality("j0.unnest", "a", ColumnType.STRING),
and(
@ -5357,7 +5334,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -5393,7 +5369,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
ColumnType.LONG
))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest", "v0"))
.build()
@ -5427,7 +5402,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
"timestamp_floor(\"__time\",'PT1H',null,'UTC')",
ColumnType.LONG))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest", "v0"))
.build()
@ -5456,7 +5430,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.filters(equality("dim2", "a", ColumnType.STRING))
.columns("dim3")
.limit(2)
@ -5468,7 +5441,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -5511,7 +5483,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -5551,7 +5522,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -5592,7 +5562,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -5622,7 +5591,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -5652,7 +5620,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -5747,7 +5714,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.filters(in("dim2", ImmutableList.of("a", "b", "ab", "abc")))
.columns("dim2")
.context(QUERY_CONTEXT_UNNEST)
@ -5762,7 +5728,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("_j0.unnest"))
.build()
@ -5818,7 +5783,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.EXPR$0"))
.build()
@ -5862,7 +5826,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.virtualColumns(expressionVirtualColumn("v0", "strlen(\"j0.unnest\")", ColumnType.LONG))
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("v0"))
.build()
@ -5909,7 +5872,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -5937,7 +5899,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -5968,7 +5929,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -5992,7 +5952,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6032,7 +5991,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6061,7 +6019,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6088,7 +6045,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6124,7 +6080,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6151,7 +6106,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.filters(
or(
@ -6185,7 +6139,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6213,7 +6166,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6244,7 +6196,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6292,7 +6243,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6319,7 +6269,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.filters(
or(
@ -6361,7 +6310,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
equality("dim3", "d", ColumnType.STRING)
)
)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6395,7 +6343,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
equality("j0.unnest", "a", ColumnType.STRING)
)
)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("dim3", "j0.unnest"))
.build()
@ -6686,8 +6633,8 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
? not(istrue(expressionFilter("(\"j0.unnest\" == \"m2\")")))
: not(expressionFilter("(\"j0.unnest\" == \"m2\")"))
)
))
.legacy(false)
)
)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest", "m2"))
.build()
@ -6728,7 +6675,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.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()
@ -6760,7 +6706,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Intervals.of("2023-01-02T00:00:00.000Z/2023-01-03T00:10:00.001Z")))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6803,7 +6748,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.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()
@ -6838,7 +6782,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -6874,7 +6817,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.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()
@ -6918,7 +6860,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
))
.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()
@ -6966,7 +6907,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.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)
@ -6980,7 +6920,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
: in("m1", ImmutableList.of("1", "2"))
)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -7018,7 +6957,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Intervals.of(
"2023-01-02T00:00:00.000Z/2023-01-03T00:10:00.001Z")))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.columns("arrayLongNulls", "j0.unnest")
.limit(2)
.context(QUERY_CONTEXT_UNNEST)
@ -7030,7 +6968,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
new ArrayContainsElementFilter("arrayLongNulls", ColumnType.LONG, 2L, null)
)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("j0.unnest"))
.build()
@ -7110,7 +7047,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
.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"))
@ -7142,7 +7078,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.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()
@ -7200,7 +7135,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
)
.eternityInterval()
.columns("d0", "j0.unnest")
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
@ -7322,7 +7256,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.virtualColumns(expressionVirtualColumn("v0", "substring(\"j0.unnest\", 0, -1)", ColumnType.STRING))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("v0"))
.build()
@ -7362,7 +7295,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
.virtualColumns(expressionVirtualColumn("v0", "substring(\"j0.unnest\", 0, -1)", ColumnType.STRING))
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_UNNEST)
.columns(ImmutableList.of("v0"))
.build()

View File

@ -131,12 +131,12 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
?
"DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n"
+ " DruidJoinQueryRel(condition=[=(SUBSTRING($2, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n"
:
"DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n"
+ " DruidJoinQueryRel(condition=[=(SUBSTRING($2, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"null\",\"column\":\"dim1\"}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n";
final String explanation = NullHandling.replaceWithDefault() ?
"["
@ -184,14 +184,13 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
// Skip vectorization since otherwise the "context" will change for each subtest.
skipVectorize();
String legacyExplanationWithContext = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":false,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
String legacyExplanationWithContext = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":false,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
String explanation = "[{"
+ "\"query\":{\"queryType\":\"scan\","
+ "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},"
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"resultFormat\":\"compactedList\","
+ "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],"
+ "\"legacy\":false,"
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
+ "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}],"
@ -204,7 +203,6 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"resultFormat\":\"compactedList\","
+ "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],"
+ "\"legacy\":false,"
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
+ "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}],"
@ -256,33 +254,33 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
+ "UNION ALL SELECT dim1 FROM druid.foo WHERE dim1 = '44')";
final String legacyExplanation = NullHandling.replaceWithDefault()
? "DruidUnionRel(limit=[-1])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
+ " DruidUnionRel(limit=[-1])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
: "DruidUnionRel(limit=[-1])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
+ " DruidUnionRel(limit=[-1])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n";
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n";
final String explanation = NullHandling.replaceWithDefault()
? "["
+ "{"
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],"
+ "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]"
+ "},"
+ "{"
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],"
+ "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]"
+ "},"
+ "{"
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],"
+ "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]"
+ "}]"
: "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]}]";
: "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]}]";
final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
final String attributes = "{\"statementType\":\"SELECT\"}";
testQuery(
@ -331,7 +329,7 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
+ "],"
+ "\"resultFormat\":\"compactedList\","
+ "\"columns\":[\"v0\",\"v1\"],"
+ "\"legacy\":false,"
+ ""
+ "\"context\":{\"defaultTimeout\":300000,\"forceExpressionVirtualColumns\":true,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
+ "\"columnTypes\":[\"STRING\",\"STRING\"],"
+ "\"granularity\":{\"type\":\"all\"}},"
@ -358,7 +356,7 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
+ "],"
+ "\"resultFormat\":\"compactedList\","
+ "\"columns\":[\"v0\",\"v1\"],"
+ "\"legacy\":false,"
+ ""
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
+ "\"columnTypes\":[\"STRING\",\"STRING\"],"
+ "\"granularity\":{\"type\":\"all\"}},"
@ -398,7 +396,7 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
+ "],"
+ "\"resultFormat\":\"compactedList\","
+ "\"columns\":[\"v0\"],"
+ "\"legacy\":false,"
+ ""
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
+ "\"columnTypes\":[\"LONG\"],"
+ "\"granularity\":{\"type\":\"all\"}},"

View File

@ -123,7 +123,6 @@ public class CalciteExportTest extends CalciteIngestionDmlTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
.expectResources(dataSourceRead("foo"), externalWrite(LocalFileExportStorageProvider.TYPE_NAME))
@ -198,7 +197,6 @@ public class CalciteExportTest extends CalciteIngestionDmlTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
.expectResources(dataSourceRead("foo"), externalWrite(LocalFileStorageConnectorProvider.TYPE_NAME))
@ -224,7 +222,6 @@ public class CalciteExportTest extends CalciteIngestionDmlTest
.filters(equality("dim2", "val", ColumnType.STRING))
.columns("dim2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
.expectResources(dataSourceRead("foo"), externalWrite(LocalFileStorageConnectorProvider.TYPE_NAME))
@ -251,7 +248,6 @@ public class CalciteExportTest extends CalciteIngestionDmlTest
.filters(equality("dim2", "val", ColumnType.STRING))
.columns("dim2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
.expectResources(dataSourceRead("foo"), externalWrite(LocalFileStorageConnectorProvider.TYPE_NAME))
@ -306,7 +302,6 @@ public class CalciteExportTest extends CalciteIngestionDmlTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
.expectResources(dataSourceRead("foo"), dataSourceWrite("csv"))

View File

@ -777,7 +777,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
+ "{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}],"
+ "\"resultFormat\":\"compactedList\","
+ "\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"},"
+ "{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],\"legacy\":false,"
+ "{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],"
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"FLOAT\",\"DOUBLE\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"},"
+ "{\"name\":\"v1\",\"type\":\"DOUBLE\"}],"
@ -843,7 +843,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]},"
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"namespace\",\"order\":\"ascending\"},{\"columnName\":\"country\",\"order\":\"ascending\"}],"
+ "\"columns\":[\"__time\",\"country\",\"namespace\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\","
+ "\"columns\":[\"__time\",\"country\",\"namespace\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\","
+ "\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}],"
+ " signature=[{__time:LONG, namespace:STRING, country:STRING}])\n";
@ -875,7 +875,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]},"
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"namespace\",\"order\":\"ascending\"},{\"columnName\":\"country\",\"order\":\"ascending\"}],"
+ "\"columns\":[\"__time\",\"country\",\"namespace\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\","
+ "\"columns\":[\"__time\",\"country\",\"namespace\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\","
+ "\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}],"
+ "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"namespace\",\"outputColumn\":\"namespace\"},"
@ -943,15 +943,15 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
final String legacyExplanation = "DruidJoinQueryRel(condition=[=($3, $6)], joinType=[left], query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},"
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\","
+ "\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"},"
+ "{\"columnName\":\"Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"Capital\",\"isRobot\",\"regionName\",\"v0\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\","
+ "{\"columnName\":\"Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"Capital\",\"isRobot\",\"regionName\",\"v0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\","
+ "\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"}}], signature=[{v0:LONG, isRobot:STRING, Capital:STRING, regionName:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://boo.gz\"]},\"inputFormat\":{\"type\":\"json\"},"
+ "\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}]},"
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"cityName\",\"countryIsoCode\",\"isRobot\",\"regionName\",\"timestamp\"],\"legacy\":false,"
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"cityName\",\"countryIsoCode\",\"isRobot\",\"regionName\",\"timestamp\"],"
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{isRobot:STRING, timestamp:STRING, cityName:STRING, countryIsoCode:STRING, regionName:STRING}])\n"
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true},"
+ "\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{Capital:STRING, ISO2:STRING}])\n";
+ "\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{Capital:STRING, ISO2:STRING}])\n";
// Use testQuery for EXPLAIN (not testIngestionQuery).
testQuery(
PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN,
@ -977,10 +977,10 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
+ ",\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},"
+ "{\"name\":\"regionName\",\"type\":\"STRING\"}]},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true},"
+ "\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\","
+ "\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\","
+ "\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\","
+ "\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}},\"rightPrefix\":\"j0.\",\"condition\":\"(\\\"countryIsoCode\\\" == \\\"j0.ISO2\\\")\",\"joinType\":\"LEFT\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"},"
+ "{\"columnName\":\"j0.Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"isRobot\",\"j0.Capital\",\"regionName\",\"v0\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\","
+ "{\"columnName\":\"j0.Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"isRobot\",\"j0.Capital\",\"regionName\",\"v0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\","
+ "\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"j0.Capital\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"__time\"},"
+ "{\"queryColumn\":\"isRobot\",\"outputColumn\":\"isRobotAlias\"},{\"queryColumn\":\"j0.Capital\",\"outputColumn\":\"countryCapital\"},{\"queryColumn\":\"regionName\",\"outputColumn\":\"regionName\"}]}]";
@ -1310,7 +1310,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
+ "\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},"
+ "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]},"
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"legacy\":false,"
+ "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],"
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
+ "\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}],"
+ "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]"
@ -1411,7 +1411,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
+ "{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}],"
+ "\"resultFormat\":\"compactedList\","
+ "\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"},"
+ "{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],\"legacy\":false,"
+ "{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],"
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"FLOAT\",\"DOUBLE\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"},"
+ "{\"name\":\"v1\",\"type\":\"DOUBLE\"}],"

View File

@ -3515,7 +3515,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.filters(equality("m2", "1000", ColumnType.DOUBLE))
.columns("dim2")
.legacy(false)
.build()
);
@ -3545,7 +3544,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
false
))
.columns("dim2")
.legacy(false)
.build()
),
rightTable,
@ -5633,7 +5631,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
.columns("l1")
.intervals(querySegmentSpec(Filtration.eternity()))
.context(queryContext)
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build()
),
@ -6173,7 +6170,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.dataSource(CalciteTests.DATASOURCE3)
.columns("dim2")
.legacy(false)
.context(context)
.build()
),
@ -6232,7 +6228,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.dataSource(CalciteTests.DATASOURCE3)
.columns("dim2")
.legacy(false)
.context(context)
.build()
),
@ -6287,7 +6282,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.dataSource(CalciteTests.DATASOURCE1)
.filters(in("dim2", ImmutableList.of("a", "b", "ab", "abc")))
.legacy(false)
.context(context)
.columns("dim2")
.build()
@ -6304,7 +6298,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.dataSource(CalciteTests.DATASOURCE3)
.columns("dim2")
.legacy(false)
.context(context)
.build()
),
@ -6379,7 +6372,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
null
))
.columns("dim2", "j0.unnest")
.legacy(false)
.context(context)
.build()
),
@ -6451,7 +6443,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
ImmutableList.of("a", "ab", "abc", "b")
)
)
.legacy(false)
.context(context)
.build()
),
@ -6471,7 +6462,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.dataSource(CalciteTests.DATASOURCE3)
.columns("dim2")
.legacy(false)
.context(context)
.build()
),

View File

@ -1748,7 +1748,6 @@ public class CalciteLookupFunctionQueryTest extends BaseCalciteQueryTest
expressionVirtualColumn("v0", "null", ColumnType.STRING)
)
.columns("v0")
.legacy(false)
.filters(isNull("dim2"))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
@ -1785,7 +1784,6 @@ public class CalciteLookupFunctionQueryTest extends BaseCalciteQueryTest
expressionVirtualColumn("v0", "null", ColumnType.STRING)
)
.columns("v0")
.legacy(false)
.filters(isNull("dim2"))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
@ -2140,7 +2138,6 @@ public class CalciteLookupFunctionQueryTest extends BaseCalciteQueryTest
.columns("$f1", "EXPR$0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT)
.legacy(false)
.build()
);
}

View File

@ -195,7 +195,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
.columns(ImmutableList.of("v0"))
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -223,7 +222,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
.columns(ImmutableList.of("v0"))
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -252,7 +250,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
.columns(ImmutableList.of("v0"))
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -456,7 +453,6 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
.columns(ImmutableList.of("v0"))
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(

View File

@ -1182,7 +1182,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
.columns("v0", "v1", "v2", "v3")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
)
@ -1270,7 +1269,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("j0.unnest")
.context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY)
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build()
)
@ -1330,7 +1328,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("j0.unnest")
.context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY)
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build()
)
@ -1387,7 +1384,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("j0.unnest")
.context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY)
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build()
)
@ -4858,7 +4854,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
.columns("v0", "v1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -4908,7 +4903,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
.columns("v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -4956,7 +4950,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
.columns("v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -5007,7 +5000,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
.columns("string", "v0", "v1", "v2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -5103,7 +5095,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
.columns("v0", "v1", "v2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -5140,7 +5131,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
.columns("v0", "v1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -5179,7 +5169,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
.columns("v0", "v1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -5308,7 +5297,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.filters(isNull("v0"))
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -5512,7 +5500,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.filters(notNull("v0"))
.columns("v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
expectedResults,
@ -6018,7 +6005,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
"variantWithArrays"
)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
useDefault ?
@ -6612,7 +6598,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.columns("nest")
.filters(notNull("nest"))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
NullHandling.replaceWithDefault()
@ -6642,7 +6627,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.columns("nest", "nester")
.filters(isNull("nest"))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
// selector filter is wrong
@ -6698,7 +6682,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("j0.unnest", "long", "v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
)
@ -6746,7 +6729,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("long")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
) :
@ -6761,7 +6743,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.virtualColumns(expressionVirtualColumn("v0", "nvl(\"long\",\"j0.unnest\")", ColumnType.LONG))
.columns("v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -6847,7 +6828,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
.columns("v0", "v1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -6888,7 +6868,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
.columns("v0")
.context(QUERY_CONTEXT_DEFAULT)
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build()
)
@ -6943,7 +6922,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.columns("v0")
.limit(1)
.context(QUERY_CONTEXT_DEFAULT)
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build()
)
@ -6983,7 +6961,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("j0.unnest")
.context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY)
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.build()
)
@ -7165,7 +7142,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
)
)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(

View File

@ -75,7 +75,6 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0")
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -685,7 +684,6 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.columns("EXPR$0")
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),

View File

@ -1351,7 +1351,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.columns("EXPR$0", "dim2")
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of()
@ -1376,7 +1375,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.columns("EXPR$0", "dim2")
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of()
@ -2528,7 +2526,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.limit(1)
.columns(ImmutableList.of("v0"))
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -3470,7 +3467,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0")
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
: ImmutableList.of(
@ -3508,7 +3504,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0")
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
: ImmutableList.of(
@ -3546,7 +3541,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0")
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
: ImmutableList.of(
@ -3965,7 +3959,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0")
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(new Object[]{0L})
@ -4243,7 +4236,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0", "EXPR$1")
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -4316,7 +4308,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0", "EXPR$1")
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of()
@ -6389,7 +6380,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0")
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -6777,7 +6767,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.virtualColumns(new ExpressionVirtualColumn("v0", ExprEval.of(1L).toExpr(), ColumnType.LONG))
.columns("v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_LOS_ANGELES)
.build()
),
@ -12709,7 +12698,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim1", "dim2", "dimX")
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
@ -13851,7 +13839,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
expressionVirtualColumn("v0", "round(\"f1\")", ColumnType.FLOAT)
)
.columns("f1", "v0")
.legacy(false)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
@ -14932,7 +14919,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.columns("l1")
.intervals(querySegmentSpec(Filtration.eternity()))
.context(QUERY_CONTEXT_DEFAULT)
.legacy(false)
.filters(
in("l1", ColumnType.LONG, ImmutableList.of(4842L, 4844L, 4845L, 14905L, 4853L, 29064L))
)
@ -15178,7 +15164,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
)
)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.limit(10)
.build()
),
@ -15818,7 +15803,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.columns("a0", "w0")
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.limit(10)
.build()
)
@ -15993,7 +15977,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
)
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.columns(ImmutableList.of(
"EXPR$0"
))
@ -16078,7 +16061,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.columns("EXPR$0")
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
) :
ImmutableList.of(
@ -16094,7 +16076,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.virtualColumns(expressionVirtualColumn("v0", "0", ColumnType.LONG))
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(NullHandling.sqlCompatible() ? new Object[]{null} : new Object[]{0})
@ -16205,7 +16186,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.filters(isNull("dim3"))
.limit(1)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_LOS_ANGELES)
.build()
)

View File

@ -705,7 +705,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest
+ "\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},"
+ "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]},"
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"legacy\":false,"
+ "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],"
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\","
+ "\"sqlReplaceTimeChunks\":\"all\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
+ "\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],"
@ -790,7 +790,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest
+ "], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"dim1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]";
final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"dim1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]";
final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"dim1\"],\"replaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\"}";
@ -883,7 +883,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest
+ "\"orderBy\":[{\"columnName\":\"__time\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"},"
+ "{\"columnName\":\"dim3\",\"order\":\"ascending\"},{\"columnName\":\"dim2\",\"order\":\"ascending\"}],"
+ "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],"
+ "\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\","
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\","
+ "\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\","
+ "\"vectorizeVirtualColumns\":\"false\"},"
+ "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],"

View File

@ -92,7 +92,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
)
.columns("v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -126,7 +125,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
)
.columns("v0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -176,7 +174,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0", "EXPR$1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -204,7 +201,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0", "EXPR$1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -232,7 +228,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0", "EXPR$1")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -270,7 +265,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0", "EXPR$1", "EXPR$2", "EXPR$3", "EXPR$4")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of(
@ -497,7 +491,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
)
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
expected
@ -533,7 +526,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
expressionVirtualColumn("v4", "5", ColumnType.LONG)
)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(context)
.build()
),
@ -591,7 +583,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
)
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
expected
@ -610,13 +601,12 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"resultFormat\":\"compactedList\","
+ "\"columns\":[\"EXPR$0\"],"
+ "\"legacy\":false,"
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
+ "\"columnTypes\":[\"LONG\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"signature\":[{\"name\":\"EXPR$0\",\"type\":\"LONG\"}],"
+ "\"columnMappings\":[{\"queryColumn\":\"EXPR$0\",\"outputColumn\":\"EXPR$0\"}]"
+ "}]";
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"EXPR$0\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"EXPR$0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
+ "\"columnTypes\":[\"LONG\"],\"granularity\":{\"type\":\"all\"}}], signature=[{EXPR$0:LONG}])\n";
final String resources = "[]";
final String attributes = "{\"statementType\":\"SELECT\"}";
@ -896,7 +886,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -922,7 +911,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of()
@ -947,7 +935,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim2")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
),
ImmutableList.of()
@ -1012,7 +999,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("CURRENT_DATE", "CURRENT_TIMESTAMP", "EXPR$1", "EXPR$2", "EXPR$3", "EXPR$4", "EXPR$6")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(context)
.build()
),
@ -1344,14 +1330,13 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
skipVectorize();
final String query = "EXPLAIN PLAN FOR SELECT * FROM druid.foo";
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
final String explanation = "[{"
+ "\"query\":{\"queryType\":\"scan\","
+ "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},"
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
+ "\"resultFormat\":\"compactedList\","
+ "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],"
+ "\"legacy\":false,"
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
+ "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX<hyperUnique>\"],\"granularity\":{\"type\":\"all\"}},"
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}],"

View File

@ -731,7 +731,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest
.limit(10)
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.legacy(false)
.build()
),
ImmutableList.of()
@ -952,7 +951,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest
.intervals(querySegmentSpec(Filtration.eternity()))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.columns("__time")
.legacy(false)
.context(queryContext)
.build()),
"j0.",
@ -1307,7 +1305,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest
.limit(1L)
.order(ScanQuery.Order.DESCENDING)
.columns("__time", "channel")
.legacy(false)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),

View File

@ -49,7 +49,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest
.virtualColumns(
expressionVirtualColumn("v0", "null", null)
)
.legacy(false)
.build(),
Druids.newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
@ -57,7 +56,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest
.columns("dim1", "dim4")
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
)
@ -94,7 +92,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest
.columns("d1", "dim1", "dim4", "f1")
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
.expectedResults(
@ -137,7 +134,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.filters(equality("dim1", "2", ColumnType.STRING))
.legacy(false)
.build()
)
.expectedResults(
@ -165,7 +161,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest
.columns("dim1")
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
.expectedResults(
@ -196,7 +191,6 @@ public class CalciteTableAppendTest extends BaseCalciteQueryTest
.columns("dim3")
.context(QUERY_CONTEXT_DEFAULT)
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.build()
)
.expectedResults(

View File

@ -319,7 +319,7 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest
"\"inputSource\":{\"type\":\"http\",\"uris\":[\"http://foo.com/bar.csv\"],\"httpAuthenticationUsername\":\"bob\",\"httpAuthenticationPassword\":{\"type\":\"default\",\"password\":\"secret\"}}," +
"\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]}," +
"\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," +
"\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"legacy\":false," +
"\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"]," +
"\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"," +
"\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\"," +
"\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," +

View File

@ -101,7 +101,6 @@ select v.*,e.* from v inner join e on (e.cityName = v.cityName);
},
"resultFormat" : "compactedList",
"columns" : [ "a0", "d0", "j0.a0", "j0.d0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING", "LONG", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -39,7 +39,6 @@ select cityName, countryName from wikipedia where cityName='New York' limit 1;
"matchValue" : "New York"
},
"columns" : [ "cityName", "countryName" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -82,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -83,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -82,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -83,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -79,7 +79,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -105,7 +104,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -80,7 +80,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -107,7 +106,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -82,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -83,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -82,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -83,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -82,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -83,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -82,7 +82,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -108,7 +107,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -83,7 +83,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -110,7 +109,6 @@ DruidAggregate(group=[{0}], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -57,7 +57,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -78,7 +77,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -57,7 +57,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -78,7 +77,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "m1", "v0" ],
"legacy" : false,
"columnTypes" : [ "FLOAT", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "j0.k", "j0.v", "m1" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "FLOAT" ],
"granularity" : {
"type" : "all"

View File

@ -63,7 +63,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
@ -84,7 +83,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -63,7 +63,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
@ -84,7 +83,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -60,7 +60,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
@ -81,7 +80,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -63,7 +63,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
@ -84,7 +83,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -63,7 +63,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
@ -84,7 +83,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -63,7 +63,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
@ -84,7 +83,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -63,7 +63,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "v0" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"
@ -84,7 +83,6 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical])
},
"resultFormat" : "compactedList",
"columns" : [ "dim1", "dim2", "j0.k", "j0.v" ],
"legacy" : false,
"columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -75,7 +75,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -76,7 +76,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -75,7 +75,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -76,7 +76,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -72,7 +72,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -98,7 +97,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
@ -121,7 +119,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -73,7 +73,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -100,7 +99,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
@ -123,7 +121,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -75,7 +75,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"value" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -101,7 +100,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"value" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
@ -124,7 +122,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"

View File

@ -76,7 +76,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "__time", "v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"
@ -103,7 +102,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
"matchValue" : "10.1"
},
"columns" : [ "dim1" ],
"legacy" : false,
"columnTypes" : [ "STRING" ],
"granularity" : {
"type" : "all"
@ -126,7 +124,6 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical])
} ],
"resultFormat" : "compactedList",
"columns" : [ "__time", "_v0" ],
"legacy" : false,
"columnTypes" : [ "LONG", "STRING" ],
"granularity" : {
"type" : "all"

Some files were not shown because too many files have changed in this diff Show More