Add more sql tests for groupby queries (#11454)

* Add more sql tests for simple groupby queries

* unused import

* fix tests

* javadocs

* unused import
This commit is contained in:
Jihoon Son 2021-07-20 21:05:11 -07:00 committed by GitHub
parent a2538d264d
commit 84c957f541
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 751 additions and 15 deletions

View File

@ -30,6 +30,7 @@ import org.apache.druid.hll.VersionOneHyperLogLogCollector;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.math.expr.ExprMacroTable;
@ -53,6 +54,7 @@ import org.apache.druid.query.filter.InDimFilter;
import org.apache.druid.query.filter.NotDimFilter;
import org.apache.druid.query.filter.OrDimFilter;
import org.apache.druid.query.filter.SelectorDimFilter;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.having.DimFilterHavingSpec;
import org.apache.druid.query.ordering.StringComparator;
import org.apache.druid.query.ordering.StringComparators;
@ -105,6 +107,10 @@ import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
/**
* A base class for SQL query testing. It sets up query execution environment, provides useful helper methods,
* and populates data using {@link CalciteTests#createMockWalker}.
*/
public class BaseCalciteQueryTest extends CalciteTestBase
{
public static String NULL_STRING;
@ -973,4 +979,18 @@ public class BaseCalciteQueryTest extends CalciteTestBase
);
walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder());
}
protected Map<String, Object> withTimestampResultContext(
Map<String, Object> input,
String timestampResultField,
int timestampResultFieldIndex,
Granularity granularity
)
{
Map<String, Object> output = new HashMap<>(input);
output.put(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD, timestampResultField);
output.put(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_GRANULARITY, granularity);
output.put(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_INDEX, timestampResultFieldIndex);
return output;
}
}

View File

@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.math.expr.ExprMacroTable;
@ -13387,6 +13386,108 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim() throws Exception
{
testQuery(
"SELECT dim2, time_floor(gran, 'P1M') gran, sum(s)\n"
+ "FROM (SELECT time_floor(__time, 'P1D') AS gran, dim2, sum(m1) as s FROM druid.foo GROUP BY 1, 2 HAVING sum(m1) > 1) AS x\n"
+ "GROUP BY 1, 2\n"
+ "ORDER BY dim2, gran desc",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1D',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("v0", "d0", ValueType.LONG),
new DefaultDimensionSpec("dim2", "d1")
)
)
.setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1")))
.setContext(
withTimestampResultContext(
QUERY_CONTEXT_DEFAULT,
"d0",
0,
Granularities.DAY
)
)
.build()
)
)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"d0\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("d1", "_d0"),
new DefaultDimensionSpec("v0", "_d1", ValueType.LONG)
)
)
.setDimFilter(
new BoundDimFilter(
"a0",
"1",
null,
true,
null,
null,
null,
StringComparators.NUMERIC
)
)
.setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("_a0", "a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec("_d0", OrderByColumnSpec.Direction.ASCENDING),
new OrderByColumnSpec(
"_d1",
Direction.DESCENDING,
StringComparators.NUMERIC
)
),
Integer.MAX_VALUE
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
NullHandling.replaceWithDefault() ?
ImmutableList.of(
new Object[]{"", timestamp("2001-01-01"), 6.0},
new Object[]{"", timestamp("2000-01-01"), 5.0},
new Object[]{"a", timestamp("2001-01-01"), 4.0},
new Object[]{"abc", timestamp("2001-01-01"), 5.0}
) :
ImmutableList.of(
new Object[]{null, timestamp("2001-01-01"), 6.0},
new Object[]{null, timestamp("2000-01-01"), 2.0},
new Object[]{"", timestamp("2000-01-01"), 3.0},
new Object[]{"a", timestamp("2001-01-01"), 4.0},
new Object[]{"abc", timestamp("2001-01-01"), 5.0}
)
);
}
@Test
public void testGroupingSets() throws Exception
{
@ -17931,18 +18032,4 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
)
);
}
private Map<String, Object> withTimestampResultContext(
Map<String, Object> input,
String timestampResultField,
int timestampResultFieldIndex,
Granularity granularity
)
{
Map<String, Object> output = new HashMap<>(input);
output.put(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD, timestampResultField);
output.put(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_GRANULARITY, granularity);
output.put(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_INDEX, timestampResultFieldIndex);
return output;
}
}

View File

@ -0,0 +1,629 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.sql.calcite;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.filter.LikeDimFilter;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec.Direction;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.junit.Test;
/**
* This class tests simple aggregation SQL queries, i.e., no joins and no nested queries.
*/
public class CalciteSimpleQueryTest extends BaseCalciteQueryTest
{
@Test
public void testGroupByTimeAndDim() throws Exception
{
testQuery(
"SELECT FLOOR(__time TO MONTH), dim2, SUM(cnt)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1, 2",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("v0", "d0", ValueType.LONG),
new DefaultDimensionSpec("dim2", "d1")
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d0", 0, Granularities.MONTH))
.build()
),
NullHandling.replaceWithDefault()
? ImmutableList.of(
new Object[]{timestamp("2000-01-01"), "", 2L},
new Object[]{timestamp("2000-01-01"), "a", 1L},
new Object[]{timestamp("2001-01-01"), "", 1L},
new Object[]{timestamp("2001-01-01"), "a", 1L},
new Object[]{timestamp("2001-01-01"), "abc", 1L}
)
: ImmutableList.of(
new Object[]{timestamp("2000-01-01"), null, 1L},
new Object[]{timestamp("2000-01-01"), "", 1L},
new Object[]{timestamp("2000-01-01"), "a", 1L},
new Object[]{timestamp("2001-01-01"), null, 1L},
new Object[]{timestamp("2001-01-01"), "a", 1L},
new Object[]{timestamp("2001-01-01"), "abc", 1L}
)
);
}
@Test
public void testGroupByDimAndTime() throws Exception
{
testQuery(
"SELECT dim2, FLOOR(__time TO MONTH), SUM(cnt)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1, 2",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("v0", "d1", ValueType.LONG)
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d1", 1, Granularities.MONTH))
.build()
),
NullHandling.replaceWithDefault()
? ImmutableList.of(
new Object[]{"", timestamp("2000-01-01"), 2L},
new Object[]{"", timestamp("2001-01-01"), 1L},
new Object[]{"a", timestamp("2000-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L}
)
: ImmutableList.of(
new Object[]{null, timestamp("2000-01-01"), 1L},
new Object[]{null, timestamp("2001-01-01"), 1L},
new Object[]{"", timestamp("2000-01-01"), 1L},
new Object[]{"a", timestamp("2000-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L}
)
);
}
@Test
public void testGroupByDimAndTimeWhereOnTime() throws Exception
{
testQuery(
"SELECT dim2, FLOOR(__time TO MONTH), SUM(cnt)\n"
+ "FROM druid.foo\n"
+ "WHERE FLOOR(__time TO MONTH) = TIMESTAMP '2001-01-01'\n"
+ "GROUP BY 1, 2",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Intervals.of("2001-01-01/P1M")))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("v0", "d1", ValueType.LONG)
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d1", 1, Granularities.MONTH))
.build()
),
NullHandling.replaceWithDefault()
? ImmutableList.of(
new Object[]{"", timestamp("2001-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L}
)
: ImmutableList.of(
new Object[]{null, timestamp("2001-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L}
)
);
}
@Test
public void testGroupByDimAndTimeOnDim() throws Exception
{
testQuery(
"SELECT dim2, FLOOR(__time TO MONTH), SUM(cnt)\n"
+ "FROM druid.foo\n"
+ "WHERE dim2 LIKE 'a%'\n"
+ "GROUP BY 1, 2",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("v0", "d1", ValueType.LONG)
)
)
.setDimFilter(new LikeDimFilter("dim2", "a%", null, null))
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d1", 1, Granularities.MONTH))
.build()
),
NullHandling.replaceWithDefault()
? ImmutableList.of(
new Object[]{"a", timestamp("2000-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L}
)
: ImmutableList.of(
new Object[]{"a", timestamp("2000-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L}
)
);
}
@Test
public void testGroupByTimeAndDimOrderByDim() throws Exception
{
testQuery(
"SELECT FLOOR(__time TO MONTH), dim2, SUM(cnt)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1, 2\n"
+ "ORDER BY dim2",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("v0", "d0", ValueType.LONG),
new DefaultDimensionSpec("dim2", "d1")
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(new OrderByColumnSpec("d1", Direction.ASCENDING)),
null
)
)
.setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d0", 0, Granularities.MONTH))
.build()
),
NullHandling.replaceWithDefault()
? ImmutableList.of(
new Object[]{timestamp("2000-01-01"), "", 2L},
new Object[]{timestamp("2001-01-01"), "", 1L},
new Object[]{timestamp("2000-01-01"), "a", 1L},
new Object[]{timestamp("2001-01-01"), "a", 1L},
new Object[]{timestamp("2001-01-01"), "abc", 1L}
)
: ImmutableList.of(
new Object[]{timestamp("2000-01-01"), null, 1L},
new Object[]{timestamp("2001-01-01"), null, 1L},
new Object[]{timestamp("2000-01-01"), "", 1L},
new Object[]{timestamp("2000-01-01"), "a", 1L},
new Object[]{timestamp("2001-01-01"), "a", 1L},
new Object[]{timestamp("2001-01-01"), "abc", 1L}
)
);
}
@Test
public void testGroupByTimeAndDimOrderByDimDesc() throws Exception
{
testQuery(
"SELECT FLOOR(__time TO MONTH), dim2, SUM(cnt)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1, 2\n"
+ "ORDER BY dim2 DESC",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("v0", "d0", ValueType.LONG),
new DefaultDimensionSpec("dim2", "d1")
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(new OrderByColumnSpec("d1", Direction.DESCENDING)),
null
)
)
.setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d0", 0, Granularities.MONTH))
.build()
),
NullHandling.replaceWithDefault()
? ImmutableList.of(
new Object[]{timestamp("2001-01-01"), "abc", 1L},
new Object[]{timestamp("2000-01-01"), "a", 1L},
new Object[]{timestamp("2001-01-01"), "a", 1L},
new Object[]{timestamp("2000-01-01"), "", 2L},
new Object[]{timestamp("2001-01-01"), "", 1L}
)
: ImmutableList.of(
new Object[]{timestamp("2001-01-01"), "abc", 1L},
new Object[]{timestamp("2000-01-01"), "a", 1L},
new Object[]{timestamp("2001-01-01"), "a", 1L},
new Object[]{timestamp("2000-01-01"), "", 1L},
new Object[]{timestamp("2000-01-01"), null, 1L},
new Object[]{timestamp("2001-01-01"), null, 1L}
)
);
}
@Test
public void testGroupByDimAndTimeOrderByTime() throws Exception
{
testQuery(
"SELECT dim2, FLOOR(__time TO MONTH), SUM(cnt)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1, 2\n"
+ "ORDER BY FLOOR(__time TO MONTH)",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("v0", "d1", ValueType.LONG)
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec("d1", Direction.ASCENDING, StringComparators.NUMERIC)
),
null
)
)
.setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d1", 1, Granularities.MONTH))
.build()
),
NullHandling.replaceWithDefault()
? ImmutableList.of(
new Object[]{"", timestamp("2000-01-01"), 2L},
new Object[]{"a", timestamp("2000-01-01"), 1L},
new Object[]{"", timestamp("2001-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L}
)
: ImmutableList.of(
new Object[]{null, timestamp("2000-01-01"), 1L},
new Object[]{"", timestamp("2000-01-01"), 1L},
new Object[]{"a", timestamp("2000-01-01"), 1L},
new Object[]{null, timestamp("2001-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L}
)
);
}
@Test
public void testGroupByDimAndTimeOrderByTimeDesc() throws Exception
{
testQuery(
"SELECT dim2, FLOOR(__time TO MONTH), SUM(cnt)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1, 2\n"
+ "ORDER BY FLOOR(__time TO MONTH) DESC",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("v0", "d1", ValueType.LONG)
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec("d1", Direction.DESCENDING, StringComparators.NUMERIC)
),
null
)
)
.setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d1", 1, Granularities.MONTH))
.build()
),
NullHandling.replaceWithDefault()
? ImmutableList.of(
new Object[]{"", timestamp("2001-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L},
new Object[]{"", timestamp("2000-01-01"), 2L},
new Object[]{"a", timestamp("2000-01-01"), 1L}
)
: ImmutableList.of(
new Object[]{null, timestamp("2001-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L},
new Object[]{null, timestamp("2000-01-01"), 1L},
new Object[]{"", timestamp("2000-01-01"), 1L},
new Object[]{"a", timestamp("2000-01-01"), 1L}
)
);
}
@Test
public void testGroupByDimAndTimeOrderByTimeAndDim() throws Exception
{
testQuery(
"SELECT dim2, FLOOR(__time TO MONTH), SUM(cnt)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1, 2\n"
+ "ORDER BY FLOOR(__time TO MONTH), dim2 DESC",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("v0", "d1", ValueType.LONG)
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec("d1", Direction.ASCENDING, StringComparators.NUMERIC),
new OrderByColumnSpec("d0", Direction.DESCENDING)
),
null
)
)
.setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d1", 1, Granularities.MONTH))
.build()
),
NullHandling.replaceWithDefault()
? ImmutableList.of(
new Object[]{"a", timestamp("2000-01-01"), 1L},
new Object[]{"", timestamp("2000-01-01"), 2L},
new Object[]{"abc", timestamp("2001-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"", timestamp("2001-01-01"), 1L}
)
: ImmutableList.of(
new Object[]{"a", timestamp("2000-01-01"), 1L},
new Object[]{"", timestamp("2000-01-01"), 1L},
new Object[]{null, timestamp("2000-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{null, timestamp("2001-01-01"), 1L}
)
);
}
@Test
public void testGroupByDimAndTimeOrderByDimAndTime() throws Exception
{
testQuery(
"SELECT dim2, FLOOR(__time TO MONTH), SUM(cnt)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1, 2\n"
+ "ORDER BY dim2, FLOOR(__time TO MONTH) DESC",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("v0", "d1", ValueType.LONG)
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec("d0", Direction.ASCENDING),
new OrderByColumnSpec("d1", Direction.DESCENDING, StringComparators.NUMERIC)
),
null
)
)
.setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d1", 1, Granularities.MONTH))
.build()
),
NullHandling.replaceWithDefault()
? ImmutableList.of(
new Object[]{"", timestamp("2001-01-01"), 1L},
new Object[]{"", timestamp("2000-01-01"), 2L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"a", timestamp("2000-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L}
)
: ImmutableList.of(
new Object[]{null, timestamp("2001-01-01"), 1L},
new Object[]{null, timestamp("2000-01-01"), 1L},
new Object[]{"", timestamp("2000-01-01"), 1L},
new Object[]{"a", timestamp("2001-01-01"), 1L},
new Object[]{"a", timestamp("2000-01-01"), 1L},
new Object[]{"abc", timestamp("2001-01-01"), 1L}
)
);
}
@Test
public void testGroupByDimAndTimeAndDimOrderByDimAndTimeDim() throws Exception
{
testQuery(
"SELECT dim2, FLOOR(__time TO MONTH), dim1, SUM(cnt)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1, 2, 3\n"
+ "ORDER BY dim2 DESC, FLOOR(__time TO MONTH) DESC, dim1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
dimensions(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("v0", "d1", ValueType.LONG),
new DefaultDimensionSpec("dim1", "d2")
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
new OrderByColumnSpec("d0", Direction.DESCENDING),
new OrderByColumnSpec("d1", Direction.DESCENDING, StringComparators.NUMERIC),
new OrderByColumnSpec("d2", Direction.ASCENDING)
),
null
)
)
.setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d1", 1, Granularities.MONTH))
.build()
),
NullHandling.replaceWithDefault()
? ImmutableList.of(
new Object[]{"abc", timestamp("2001-01-01"), "def", 1L},
new Object[]{"a", timestamp("2001-01-01"), "1", 1L},
new Object[]{"a", timestamp("2000-01-01"), "", 1L},
new Object[]{"", timestamp("2001-01-01"), "abc", 1L},
new Object[]{"", timestamp("2000-01-01"), "10.1", 1L},
new Object[]{"", timestamp("2000-01-01"), "2", 1L}
)
: ImmutableList.of(
new Object[]{"abc", timestamp("2001-01-01"), "def", 1L},
new Object[]{"a", timestamp("2001-01-01"), "1", 1L},
new Object[]{"a", timestamp("2000-01-01"), "", 1L},
new Object[]{"", timestamp("2000-01-01"), "2", 1L},
new Object[]{null, timestamp("2001-01-01"), "abc", 1L},
new Object[]{null, timestamp("2000-01-01"), "10.1", 1L}
)
);
}
}