You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by cw...@apache.org on 2019/02/24 04:10:38 UTC
[incubator-druid] branch master updated: Added checkstyle for
"Methods starting with Capital Letters" (#7118)
This is an automated email from the ASF dual-hosted git repository.
cwylie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git
The following commit(s) were added to refs/heads/master by this push:
new 8b803cb Added checkstyle for "Methods starting with Capital Letters" (#7118)
8b803cb is described below
commit 8b803cbc22b15799fd0526de8d6d0eea155ad733
Author: Himanshu Pandey <hp...@pivotal.io>
AuthorDate: Sat Feb 23 20:10:31 2019 -0800
Added checkstyle for "Methods starting with Capital Letters" (#7118)
* Added checkstyle for "Methods starting with Capital Letters" and changed the method names violating this.
* Un-abbreviate the method names in the calcite tests
* Fixed checkstyle errors
* Changed asserts position in the code
---
codestyle/checkstyle.xml | 5 +
.../hll/sql/HllSketchSqlAggregatorTest.java | 2 +-
.../quantiles/DoublesSketchAggregatorTest.java | 4 +-
.../theta/sql/ThetaSketchSqlAggregatorTest.java | 2 +-
.../query/filter/sql/BloomDimFilterSqlTest.java | 16 +-
.../kafka/KafkaDataSourceMetadataTest.java | 30 +-
.../druid/indexing/kafka/KafkaIndexTaskTest.java | 168 +-
.../indexing/kafka/KafkaRecordSupplierTest.java | 26 +-
.../kinesis/KinesisDataSourceMetadataTest.java | 30 +-
.../indexing/kinesis/KinesisIndexTaskTest.java | 132 +-
.../kinesis/KinesisRecordSupplierTest.java | 20 +-
.../kinesis/supervisor/KinesisSupervisorTest.java | 2 +-
.../IngestSegmentFirehoseFactoryTimelineTest.java | 40 +-
.../druid/indexing/overlord/TaskLifecycleTest.java | 16 +-
...GroupByLimitPushDownInsufficientBufferTest.java | 6 +-
.../GroupByLimitPushDownMultiNodeMergeTest.java | 6 +-
.../query/groupby/GroupByMultiSegmentTest.java | 4 +-
.../druid/query/lookup/LookupConfigTest.java | 2 +-
.../druid/segment/filter/ExpressionFilterTest.java | 126 +-
.../appenderator/AppenderatorPlumberTest.java | 4 +-
.../realtime/appenderator/AppenderatorTest.java | 102 +-
.../DefaultOfflineAppenderatorFactoryTest.java | 4 +-
.../druid/sql/avatica/DruidAvaticaHandlerTest.java | 50 +-
.../druid/sql/calcite/BaseCalciteQueryTest.java | 44 +-
.../apache/druid/sql/calcite/CalciteQueryTest.java | 1752 ++++++++++----------
25 files changed, 1302 insertions(+), 1291 deletions(-)
diff --git a/codestyle/checkstyle.xml b/codestyle/checkstyle.xml
index d42b705..04e18b5 100644
--- a/codestyle/checkstyle.xml
+++ b/codestyle/checkstyle.xml
@@ -291,5 +291,10 @@ codestyle/checkstyle.xml. "/>
<property name="illegalPattern" value="true"/>
<property name="message" value="Duplicate line"/>
</module>
+
+ <!-- Added as per the issue #6936 - Prohibit method names starting with capital letters -->
+ <module name="MethodName">
+ <property name = "format" value = "^[a-z_]*[a-z0-9][a-zA-Z0-9_]*$"/>
+ </module>
</module>
</module>
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
index 1690ef9..da70643 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
@@ -274,7 +274,7 @@ public class HllSketchSqlAggregatorTest extends CalciteTestBase
null,
null
),
- BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null))
+ BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
),
new HllSketchBuildAggregatorFactory(
"a3",
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java
index 65d5717..a29f649 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java
@@ -342,7 +342,7 @@ public class DoublesSketchAggregatorTest
}
@Test
- public void QueryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Exception
+ public void queryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Exception
{
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),
@@ -418,7 +418,7 @@ public class DoublesSketchAggregatorTest
}
@Test
- public void TimeSeriesQueryInputAsFloat() throws Exception
+ public void timeSeriesQueryInputAsFloat() throws Exception
{
Sequence<Row> seq = timeSeriesHelper.createIndexAndRunQueryOnSegment(
new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()),
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
index 919a596..939e396 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
@@ -278,7 +278,7 @@ public class ThetaSketchSqlAggregatorTest extends CalciteTestBase
null,
null
),
- BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null))
+ BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null))
),
new SketchMergeAggregatorFactory(
"a3",
diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
index 0b44cf1..335599a 100644
--- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
+++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
@@ -114,12 +114,12 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
new BloomDimFilter("dim1", BloomKFilterHolder.fromBloomKFilter(filter), null)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -146,7 +146,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.virtualColumns()
.filters(
@@ -155,7 +155,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
createExprMacroTable()
)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -178,7 +178,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.virtualColumns()
.filters(
@@ -187,7 +187,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
createExprMacroTable()
)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -214,7 +214,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
new OrDimFilter(
@@ -222,7 +222,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
new BloomDimFilter("dim2", BloomKFilterHolder.fromBloomKFilter(filter2), null)
)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java
index 89f5ce8..5b60913 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java
@@ -28,10 +28,10 @@ import java.util.Map;
public class KafkaDataSourceMetadataTest
{
- private static final KafkaDataSourceMetadata KM0 = KM("foo", ImmutableMap.of());
- private static final KafkaDataSourceMetadata KM1 = KM("foo", ImmutableMap.of(0, 2L, 1, 3L));
- private static final KafkaDataSourceMetadata KM2 = KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L));
- private static final KafkaDataSourceMetadata KM3 = KM("foo", ImmutableMap.of(0, 2L, 2, 5L));
+ private static final KafkaDataSourceMetadata KM0 = km("foo", ImmutableMap.of());
+ private static final KafkaDataSourceMetadata KM1 = km("foo", ImmutableMap.of(0, 2L, 1, 3L));
+ private static final KafkaDataSourceMetadata KM2 = km("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L));
+ private static final KafkaDataSourceMetadata KM3 = km("foo", ImmutableMap.of(0, 2L, 2, 5L));
@Test
public void testMatches()
@@ -70,27 +70,27 @@ public class KafkaDataSourceMetadataTest
public void testPlus()
{
Assert.assertEquals(
- KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)),
+ km("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)),
KM1.plus(KM3)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
+ km("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
KM0.plus(KM2)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
+ km("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
KM1.plus(KM2)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)),
+ km("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)),
KM2.plus(KM1)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
+ km("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)),
KM2.plus(KM2)
);
}
@@ -99,32 +99,32 @@ public class KafkaDataSourceMetadataTest
public void testMinus()
{
Assert.assertEquals(
- KM("foo", ImmutableMap.of(1, 3L)),
+ km("foo", ImmutableMap.of(1, 3L)),
KM1.minus(KM3)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of()),
+ km("foo", ImmutableMap.of()),
KM0.minus(KM2)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of()),
+ km("foo", ImmutableMap.of()),
KM1.minus(KM2)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of(2, 5L)),
+ km("foo", ImmutableMap.of(2, 5L)),
KM2.minus(KM1)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of()),
+ km("foo", ImmutableMap.of()),
KM2.minus(KM2)
);
}
- private static KafkaDataSourceMetadata KM(String topic, Map<Integer, Long> offsets)
+ private static KafkaDataSourceMetadata km(String topic, Map<Integer, Long> offsets)
{
return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, offsets));
}
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
index 892349f..b19bf8a 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
@@ -277,21 +277,21 @@ public class KafkaIndexTaskTest
private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic)
{
return ImmutableList.of(
- new ProducerRecord<>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2011", "e", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2011", "e", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable")),
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")),
new ProducerRecord<>(topic, 0, null, null),
- new ProducerRecord<>(topic, 0, null, JB("2013", "f", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "notanumber", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "10", "notanumber", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "10", "20.0", "notanumber")),
- new ProducerRecord<>(topic, 1, null, JB("2012", "g", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 1, null, JB("2011", "h", "y", "10", "20.0", "1.0"))
+ new ProducerRecord<>(topic, 0, null, jb("2013", "f", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "notanumber", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "10", "notanumber", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "10", "20.0", "notanumber")),
+ new ProducerRecord<>(topic, 1, null, jb("2012", "g", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 1, null, jb("2011", "h", "y", "10", "20.0", "1.0"))
);
}
@@ -411,8 +411,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@@ -461,8 +461,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@@ -570,13 +570,13 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc4 = SD(task, "2011/P1D", 0);
- SegmentDescriptor desc5 = SD(task, "2011/P1D", 1);
- SegmentDescriptor desc6 = SD(task, "2012/P1D", 0);
- SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
+ SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
+ SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
+ SegmentDescriptor desc6 = sd(task, "2012/P1D", 0);
+ SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))),
@@ -696,6 +696,7 @@ public class KafkaIndexTaskTest
}
final Map<Integer, Long> nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets());
+
Assert.assertTrue(checkpoint2.getPartitionSequenceNumberMap().equals(nextOffsets));
task.getRunner().setEndOffsets(nextOffsets, false);
@@ -727,15 +728,20 @@ public class KafkaIndexTaskTest
Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed());
Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable());
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
-
+
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc4 = SD(task, "2011/P1D", 0);
- SegmentDescriptor desc5 = SD(task, "2011/P1D", 1);
- SegmentDescriptor desc6 = SD(task, "2012/P1D", 0);
- SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
+ SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
+ SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
+ SegmentDescriptor desc6 = sd(task, "2012/P1D", 0);
+ SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
+ Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
+ Assert.assertEquals(
+ new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))),
+ metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
+
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))),
@@ -844,8 +850,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L))),
@@ -865,13 +871,13 @@ public class KafkaIndexTaskTest
}
List<ProducerRecord<byte[], byte[]>> records = ImmutableList.of(
- new ProducerRecord<>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2011", "D", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2012", "e", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2009", "B", "y", "10", "20.0", "1.0"))
+ new ProducerRecord<>(topic, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2011", "D", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2012", "e", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2009", "B", "y", "10", "20.0", "1.0"))
);
final String baseSequenceName = "sequence0";
@@ -973,8 +979,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@@ -1023,9 +1029,9 @@ public class KafkaIndexTaskTest
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
+ SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@@ -1083,7 +1089,7 @@ public class KafkaIndexTaskTest
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2009/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2009/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@@ -1164,8 +1170,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@@ -1212,8 +1218,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@@ -1307,10 +1313,10 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
- SegmentDescriptor desc3 = SD(task, "2013/P1D", 0);
- SegmentDescriptor desc4 = SD(task, "2049/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
+ SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
+ SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L))),
@@ -1463,8 +1469,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@@ -1528,8 +1534,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata, should all be from the first task
- SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@@ -1581,8 +1587,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
// Check published segments & metadata
- SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
@@ -1599,8 +1605,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1);
- SegmentDescriptor desc4 = SD(task2, "2013/P1D", 0);
+ SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
+ SegmentDescriptor desc4 = sd(task2, "2013/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
@@ -1643,11 +1649,11 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
// desc3 will not be created in KafkaIndexTask (0.12.x) as it does not create per Kafka partition Druid segments
- SegmentDescriptor desc3 = SD(task, "2011/P1D", 1);
- SegmentDescriptor desc4 = SD(task, "2012/P1D", 0);
+ SegmentDescriptor desc3 = sd(task, "2011/P1D", 1);
+ SegmentDescriptor desc4 = sd(task, "2012/P1D", 0);
Assert.assertEquals(isIncrementalHandoffSupported
? ImmutableSet.of(desc1, desc2, desc4)
: ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
@@ -1722,9 +1728,9 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
- SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0);
+ SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
+ SegmentDescriptor desc3 = sd(task2, "2012/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 1L))),
@@ -1820,8 +1826,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 6L))),
@@ -1909,8 +1915,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 6L))),
@@ -2037,8 +2043,8 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))),
@@ -2178,10 +2184,10 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = SD(task, "2013/P1D", 0);
- SegmentDescriptor desc4 = SD(task, "2049/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
+ SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
+ SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L))),
@@ -2669,7 +2675,7 @@ public class KafkaIndexTaskTest
return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows"));
}
- private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
+ private static byte[] jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
{
try {
return new ObjectMapper().writeValueAsBytes(
@@ -2688,7 +2694,7 @@ public class KafkaIndexTaskTest
}
}
- private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum)
+ private SegmentDescriptor sd(final Task task, final String intervalString, final int partitionNum)
{
final Interval interval = Intervals.of(intervalString);
return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum);
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java
index f944bf0..a5e75c9 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java
@@ -66,25 +66,25 @@ public class KafkaRecordSupplierTest
private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic)
{
return ImmutableList.of(
- new ProducerRecord<>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2011", "e", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2011", "e", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable")),
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")),
new ProducerRecord<>(topic, 0, null, null),
- new ProducerRecord<>(topic, 0, null, JB("2013", "f", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "notanumber", "20.0", "1.0")),
- new ProducerRecord<>(topic, 1, null, JB("2049", "f", "y", "10", "notanumber", "1.0")),
- new ProducerRecord<>(topic, 1, null, JB("2049", "f", "y", "10", "20.0", "notanumber")),
- new ProducerRecord<>(topic, 1, null, JB("2012", "g", "y", "10", "20.0", "1.0")),
- new ProducerRecord<>(topic, 1, null, JB("2011", "h", "y", "10", "20.0", "1.0"))
+ new ProducerRecord<>(topic, 0, null, jb("2013", "f", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "notanumber", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 1, null, jb("2049", "f", "y", "10", "notanumber", "1.0")),
+ new ProducerRecord<>(topic, 1, null, jb("2049", "f", "y", "10", "20.0", "notanumber")),
+ new ProducerRecord<>(topic, 1, null, jb("2012", "g", "y", "10", "20.0", "1.0")),
+ new ProducerRecord<>(topic, 1, null, jb("2011", "h", "y", "10", "20.0", "1.0"))
);
}
- private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
+ private static byte[] jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
{
try {
return new ObjectMapper().writeValueAsBytes(
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java
index f1e3b0f..2c5bce1 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java
@@ -29,10 +29,10 @@ import java.util.Map;
public class KinesisDataSourceMetadataTest
{
- private static final KinesisDataSourceMetadata KM0 = KM("foo", ImmutableMap.of());
- private static final KinesisDataSourceMetadata KM1 = KM("foo", ImmutableMap.of("0", "2L", "1", "3L"));
- private static final KinesisDataSourceMetadata KM2 = KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L"));
- private static final KinesisDataSourceMetadata KM3 = KM("foo", ImmutableMap.of("0", "2L", "2", "5L"));
+ private static final KinesisDataSourceMetadata KM0 = km("foo", ImmutableMap.of());
+ private static final KinesisDataSourceMetadata KM1 = km("foo", ImmutableMap.of("0", "2L", "1", "3L"));
+ private static final KinesisDataSourceMetadata KM2 = km("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L"));
+ private static final KinesisDataSourceMetadata KM3 = km("foo", ImmutableMap.of("0", "2L", "2", "5L"));
@Test
public void testMatches()
@@ -71,27 +71,27 @@ public class KinesisDataSourceMetadataTest
public void testPlus()
{
Assert.assertEquals(
- KM("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
+ km("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
KM1.plus(KM3)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
+ km("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
KM0.plus(KM2)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
+ km("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
KM1.plus(KM2)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
+ km("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")),
KM2.plus(KM1)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
+ km("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")),
KM2.plus(KM2)
);
}
@@ -100,32 +100,32 @@ public class KinesisDataSourceMetadataTest
public void testMinus()
{
Assert.assertEquals(
- KM("foo", ImmutableMap.of("1", "3L")),
+ km("foo", ImmutableMap.of("1", "3L")),
KM1.minus(KM3)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of()),
+ km("foo", ImmutableMap.of()),
KM0.minus(KM2)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of()),
+ km("foo", ImmutableMap.of()),
KM1.minus(KM2)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of("2", "5L")),
+ km("foo", ImmutableMap.of("2", "5L")),
KM2.minus(KM1)
);
Assert.assertEquals(
- KM("foo", ImmutableMap.of()),
+ km("foo", ImmutableMap.of()),
KM2.minus(KM2)
);
}
- private static KinesisDataSourceMetadata KM(String stream, Map<String, String> sequences)
+ private static KinesisDataSourceMetadata km(String stream, Map<String, String> sequences)
{
return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, sequences));
}
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
index 5294116..78ce481 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
@@ -198,26 +198,26 @@ public class KinesisIndexTaskTest extends EasyMockSupport
private static String shardId0 = "0";
private static KinesisRecordSupplier recordSupplier;
private static List<OrderedPartitionableRecord<String, String>> records = ImmutableList.of(
- new OrderedPartitionableRecord<>(stream, "1", "0", JB("2008", "a", "y", "10", "20.0", "1.0")),
- new OrderedPartitionableRecord<>(stream, "1", "1", JB("2009", "b", "y", "10", "20.0", "1.0")),
- new OrderedPartitionableRecord<>(stream, "1", "2", JB("2010", "c", "y", "10", "20.0", "1.0")),
- new OrderedPartitionableRecord<>(stream, "1", "3", JB("2011", "d", "y", "10", "20.0", "1.0")),
- new OrderedPartitionableRecord<>(stream, "1", "4", JB("2011", "e", "y", "10", "20.0", "1.0")),
+ new OrderedPartitionableRecord<>(stream, "1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")),
+ new OrderedPartitionableRecord<>(stream, "1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")),
+ new OrderedPartitionableRecord<>(stream, "1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")),
+ new OrderedPartitionableRecord<>(stream, "1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")),
+ new OrderedPartitionableRecord<>(stream, "1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")),
new OrderedPartitionableRecord<>(
stream,
"1",
"5",
- JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")
+ jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")
),
new OrderedPartitionableRecord<>(stream, "1", "6", Collections.singletonList(StringUtils.toUtf8("unparseable"))),
new OrderedPartitionableRecord<>(stream, "1", "7", Collections.singletonList(StringUtils.toUtf8("unparseable2"))),
new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(StringUtils.toUtf8("{}"))),
- new OrderedPartitionableRecord<>(stream, "1", "9", JB("2013", "f", "y", "10", "20.0", "1.0")),
- new OrderedPartitionableRecord<>(stream, "1", "10", JB("2049", "f", "y", "notanumber", "20.0", "1.0")),
- new OrderedPartitionableRecord<>(stream, "1", "11", JB("2049", "f", "y", "10", "notanumber", "1.0")),
- new OrderedPartitionableRecord<>(stream, "1", "12", JB("2049", "f", "y", "10", "20.0", "notanumber")),
- new OrderedPartitionableRecord<>(stream, "0", "0", JB("2012", "g", "y", "10", "20.0", "1.0")),
- new OrderedPartitionableRecord<>(stream, "0", "1", JB("2011", "h", "y", "10", "20.0", "1.0"))
+ new OrderedPartitionableRecord<>(stream, "1", "9", jb("2013", "f", "y", "10", "20.0", "1.0")),
+ new OrderedPartitionableRecord<>(stream, "1", "10", jb("2049", "f", "y", "notanumber", "20.0", "1.0")),
+ new OrderedPartitionableRecord<>(stream, "1", "11", jb("2049", "f", "y", "10", "notanumber", "1.0")),
+ new OrderedPartitionableRecord<>(stream, "1", "12", jb("2049", "f", "y", "10", "20.0", "notanumber")),
+ new OrderedPartitionableRecord<>(stream, "0", "0", jb("2012", "g", "y", "10", "20.0", "1.0")),
+ new OrderedPartitionableRecord<>(stream, "0", "1", jb("2011", "h", "y", "10", "20.0", "1.0"))
);
private static ServiceEmitter emitter;
@@ -404,8 +404,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
@@ -484,8 +484,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2011/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2012/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2011/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2012/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
@@ -609,13 +609,13 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc4 = SD(task, "2011/P1D", 0);
- SegmentDescriptor desc5 = SD(task, "2011/P1D", 1);
- SegmentDescriptor desc6 = SD(task, "2012/P1D", 0);
- SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
+ SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
+ SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
+ SegmentDescriptor desc6 = sd(task, "2012/P1D", 0);
+ SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@@ -770,12 +770,12 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc4 = SD(task, "2011/P1D", 0);
- SegmentDescriptor desc5 = SD(task, "2049/P1D", 0);
- SegmentDescriptor desc7 = SD(task, "2013/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
+ SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
+ SegmentDescriptor desc5 = sd(task, "2049/P1D", 0);
+ SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc7), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@@ -857,8 +857,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
@@ -940,9 +940,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = SD(task, "2010/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
+ SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
@@ -1033,7 +1033,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2009/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2009/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
@@ -1171,8 +1171,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@@ -1248,8 +1248,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@@ -1395,10 +1395,10 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
- SegmentDescriptor desc3 = SD(task, "2013/P1D", 0);
- SegmentDescriptor desc4 = SD(task, "2049/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
+ SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
+ SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
@@ -1620,8 +1620,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
@@ -1731,8 +1731,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata, should all be from the first task
- SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
@@ -1829,8 +1829,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
// Check published segments & metadata
- SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
@@ -1849,8 +1849,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1);
- SegmentDescriptor desc4 = SD(task2, "2013/P1D", 0);
+ SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
+ SegmentDescriptor desc4 = sd(task2, "2013/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
@@ -1928,9 +1928,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
- SegmentDescriptor desc4 = SD(task, "2012/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
+ SegmentDescriptor desc4 = sd(task, "2012/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc4), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@@ -2043,10 +2043,10 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
- SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1);
- SegmentDescriptor desc4 = SD(task2, "2012/P1D", 0);
+ SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
+ SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
+ SegmentDescriptor desc4 = sd(task2, "2012/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertEquals(
@@ -2196,8 +2196,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
@@ -2317,8 +2317,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(
@@ -2407,8 +2407,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = SD(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = SD(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
+ SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of(
@@ -2877,7 +2877,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows"));
}
- private static List<byte[]> JB(
+ private static List<byte[]> jb(
String timestamp,
String dim1,
String dim2,
@@ -2903,7 +2903,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
}
}
- private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum)
+ private SegmentDescriptor sd(final Task task, final String intervalString, final int partitionNum)
{
final Interval interval = Intervals.of(intervalString);
return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum);
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java
index 166678c..dd99920 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java
@@ -76,20 +76,20 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
private static Shard shard1;
private static KinesisRecordSupplier recordSupplier;
private static List<Record> shard1Records = ImmutableList.of(
- new Record().withData(JB("2011", "d", "y", "10", "20.0", "1.0")).withSequenceNumber("0"),
- new Record().withData(JB("2011", "e", "y", "10", "20.0", "1.0")).withSequenceNumber("1"),
- new Record().withData(JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")).withSequenceNumber("2"),
+ new Record().withData(jb("2011", "d", "y", "10", "20.0", "1.0")).withSequenceNumber("0"),
+ new Record().withData(jb("2011", "e", "y", "10", "20.0", "1.0")).withSequenceNumber("1"),
+ new Record().withData(jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")).withSequenceNumber("2"),
new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable"))).withSequenceNumber("3"),
new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable2"))).withSequenceNumber("4"),
new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("{}"))).withSequenceNumber("5"),
- new Record().withData(JB("2013", "f", "y", "10", "20.0", "1.0")).withSequenceNumber("6"),
- new Record().withData(JB("2049", "f", "y", "notanumber", "20.0", "1.0")).withSequenceNumber("7"),
- new Record().withData(JB("2012", "g", "y", "10", "20.0", "1.0")).withSequenceNumber("8"),
- new Record().withData(JB("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9")
+ new Record().withData(jb("2013", "f", "y", "10", "20.0", "1.0")).withSequenceNumber("6"),
+ new Record().withData(jb("2049", "f", "y", "notanumber", "20.0", "1.0")).withSequenceNumber("7"),
+ new Record().withData(jb("2012", "g", "y", "10", "20.0", "1.0")).withSequenceNumber("8"),
+ new Record().withData(jb("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9")
);
private static List<Record> shard0Records = ImmutableList.of(
- new Record().withData(JB("2008", "a", "y", "10", "20.0", "1.0")).withSequenceNumber("0"),
- new Record().withData(JB("2009", "b", "y", "10", "20.0", "1.0")).withSequenceNumber("1")
+ new Record().withData(jb("2008", "a", "y", "10", "20.0", "1.0")).withSequenceNumber("0"),
+ new Record().withData(jb("2009", "b", "y", "10", "20.0", "1.0")).withSequenceNumber("1")
);
private static List<Object> allRecords = ImmutableList.builder()
.addAll(shard0Records.stream()
@@ -120,7 +120,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
.toList()))
.build();
- private static ByteBuffer JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
+ private static ByteBuffer jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
{
try {
return ByteBuffer.wrap(new ObjectMapper().writeValueAsBytes(
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
index 750b924..2ceadb5 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
@@ -3560,7 +3560,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
);
}
- private static List<byte[]> JB(
+ private static List<byte[]> jb(
String timestamp,
String dim1,
String dim2,
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java
index 2cc0948..8ca24d9 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java
@@ -152,7 +152,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
FileUtils.deleteDirectory(tmpDir);
}
- private static TestCase TC(
+ private static TestCase tc(
String intervalString,
int expectedCount,
long expectedSum,
@@ -174,7 +174,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
);
}
- private static DataSegmentMaker DS(
+ private static DataSegmentMaker ds(
String intervalString,
String version,
int partitionNum,
@@ -184,7 +184,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
return new DataSegmentMaker(Intervals.of(intervalString), version, partitionNum, Arrays.asList(rows));
}
- private static InputRow IR(String timeString, long metricValue)
+ private static InputRow ir(String timeString, long metricValue)
{
return new MapBasedInputRow(
DateTimes.of(timeString).getMillis(),
@@ -236,34 +236,34 @@ public class IngestSegmentFirehoseFactoryTimelineTest
public static Collection<Object[]> constructorFeeder()
{
final List<TestCase> testCases = ImmutableList.of(
- TC(
+ tc(
"2000/2000T02", 3, 7,
- DS("2000/2000T01", "v1", 0, IR("2000", 1), IR("2000T00:01", 2)),
- DS("2000T01/2000T02", "v1", 0, IR("2000T01", 4))
+ ds("2000/2000T01", "v1", 0, ir("2000", 1), ir("2000T00:01", 2)),
+ ds("2000T01/2000T02", "v1", 0, ir("2000T01", 4))
) /* Adjacent segments */,
- TC(
+ tc(
"2000/2000T02", 3, 7,
- DS("2000/2000T02", "v1", 0, IR("2000", 1), IR("2000T00:01", 2), IR("2000T01", 8)),
- DS("2000T01/2000T02", "v2", 0, IR("2000T01:01", 4))
+ ds("2000/2000T02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8)),
+ ds("2000T01/2000T02", "v2", 0, ir("2000T01:01", 4))
) /* 1H segment overlaid on top of 2H segment */,
- TC(
+ tc(
"2000/2000-01-02", 4, 23,
- DS("2000/2000-01-02", "v1", 0, IR("2000", 1), IR("2000T00:01", 2), IR("2000T01", 8), IR("2000T02", 16)),
- DS("2000T01/2000T02", "v2", 0, IR("2000T01:01", 4))
+ ds("2000/2000-01-02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8), ir("2000T02", 16)),
+ ds("2000T01/2000T02", "v2", 0, ir("2000T01:01", 4))
) /* 1H segment overlaid on top of 1D segment */,
- TC(
+ tc(
"2000/2000T02", 4, 15,
- DS("2000/2000T02", "v1", 0, IR("2000", 1), IR("2000T00:01", 2), IR("2000T01", 8)),
- DS("2000/2000T02", "v1", 1, IR("2000T01:01", 4))
+ ds("2000/2000T02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8)),
+ ds("2000/2000T02", "v1", 1, ir("2000T01:01", 4))
) /* Segment set with two segments for the same interval */,
- TC(
+ tc(
"2000T01/2000T02", 1, 2,
- DS("2000/2000T03", "v1", 0, IR("2000", 1), IR("2000T01", 2), IR("2000T02", 4))
+ ds("2000/2000T03", "v1", 0, ir("2000", 1), ir("2000T01", 2), ir("2000T02", 4))
) /* Segment wider than desired interval */,
- TC(
+ tc(
"2000T02/2000T04", 2, 12,
- DS("2000/2000T03", "v1", 0, IR("2000", 1), IR("2000T01", 2), IR("2000T02", 4)),
- DS("2000T03/2000T04", "v1", 0, IR("2000T03", 8))
+ ds("2000/2000T03", "v1", 0, ir("2000", 1), ir("2000T01", 2), ir("2000T02", 4)),
+ ds("2000T03/2000T04", "v1", 0, ir("2000T03", 8))
) /* Segment intersecting desired interval */
);
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
index 455c3f0..d7e51fa 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
@@ -194,16 +194,16 @@ public class TaskLifecycleTest
private static DateTime now = DateTimes.nowUtc();
private static final Iterable<InputRow> realtimeIdxTaskInputRows = ImmutableList.of(
- IR(now.toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 1.0f),
- IR(now.plus(new Period(Hours.ONE)).toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 2.0f),
- IR(now.plus(new Period(Hours.TWO)).toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 3.0f)
+ ir(now.toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 1.0f),
+ ir(now.plus(new Period(Hours.ONE)).toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 2.0f),
+ ir(now.plus(new Period(Hours.TWO)).toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 3.0f)
);
private static final Iterable<InputRow> IdxTaskInputRows = ImmutableList.of(
- IR("2010-01-01T01", "x", "y", 1),
- IR("2010-01-01T01", "x", "z", 1),
- IR("2010-01-02T01", "a", "b", 2),
- IR("2010-01-02T01", "a", "c", 1)
+ ir("2010-01-01T01", "x", "y", 1),
+ ir("2010-01-01T01", "x", "z", 1),
+ ir("2010-01-02T01", "a", "b", 2),
+ ir("2010-01-02T01", "a", "c", 1)
);
@Rule
@@ -240,7 +240,7 @@ public class TaskLifecycleTest
return new NoopServiceEmitter();
}
- private static InputRow IR(String dt, String dim1, String dim2, float met)
+ private static InputRow ir(String dt, String dim1, String dim2, float met)
{
return new MapBasedInputRow(
DateTimes.of(dt).getMillis(),
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
index d6ac1bf..5d8be6d 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java
@@ -394,7 +394,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
strategySelector,
new GroupByQueryQueryToolChest(
strategySelector,
- NoopIntervalChunkingQueryRunnerDecorator()
+ noopIntervalChunkingQueryRunnerDecorator()
)
);
@@ -402,7 +402,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
tooSmallStrategySelector,
new GroupByQueryQueryToolChest(
tooSmallStrategySelector,
- NoopIntervalChunkingQueryRunnerDecorator()
+ noopIntervalChunkingQueryRunnerDecorator()
)
);
}
@@ -679,7 +679,7 @@ public class GroupByLimitPushDownInsufficientBufferTest
}
};
- public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator()
+ public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
{
return new IntervalChunkingQueryRunnerDecorator(null, null, null)
{
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
index a177398..581e80b 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java
@@ -426,7 +426,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
strategySelector,
new GroupByQueryQueryToolChest(
strategySelector,
- NoopIntervalChunkingQueryRunnerDecorator()
+ noopIntervalChunkingQueryRunnerDecorator()
)
);
@@ -434,7 +434,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
strategySelector2,
new GroupByQueryQueryToolChest(
strategySelector2,
- NoopIntervalChunkingQueryRunnerDecorator()
+ noopIntervalChunkingQueryRunnerDecorator()
)
);
}
@@ -780,7 +780,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest
}
};
- public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator()
+ public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
{
return new IntervalChunkingQueryRunnerDecorator(null, null, null)
{
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java
index 27e41ba..62e7a40 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java
@@ -287,7 +287,7 @@ public class GroupByMultiSegmentTest
strategySelector,
new GroupByQueryQueryToolChest(
strategySelector,
- NoopIntervalChunkingQueryRunnerDecorator()
+ noopIntervalChunkingQueryRunnerDecorator()
)
);
}
@@ -419,7 +419,7 @@ public class GroupByMultiSegmentTest
}
};
- public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator()
+ public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
{
return new IntervalChunkingQueryRunnerDecorator(null, null, null) {
@Override
diff --git a/processing/src/test/java/org/apache/druid/query/lookup/LookupConfigTest.java b/processing/src/test/java/org/apache/druid/query/lookup/LookupConfigTest.java
index ca18913..b8f295f 100644
--- a/processing/src/test/java/org/apache/druid/query/lookup/LookupConfigTest.java
+++ b/processing/src/test/java/org/apache/druid/query/lookup/LookupConfigTest.java
@@ -36,7 +36,7 @@ public class LookupConfigTest
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Test
- public void TestSerDesr() throws IOException
+ public void testSerDesr() throws IOException
{
LookupConfig lookupConfig = new LookupConfig(temporaryFolder.newFile().getAbsolutePath());
Assert.assertEquals(
diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java
index 3d3d50c..f9d4a91 100644
--- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java
@@ -117,22 +117,22 @@ public class ExpressionFilterTest extends BaseFilterTest
@Test
public void testOneSingleValuedStringColumn()
{
- assertFilterMatches(EDF("dim3 == ''"), ImmutableList.of("0"));
- assertFilterMatches(EDF("dim3 == '1'"), ImmutableList.of("3", "4", "6"));
- assertFilterMatches(EDF("dim3 == 'a'"), ImmutableList.of("7"));
- assertFilterMatches(EDF("dim3 == 1"), ImmutableList.of("3", "4", "6"));
- assertFilterMatches(EDF("dim3 == 1.0"), ImmutableList.of("3", "4", "6"));
- assertFilterMatches(EDF("dim3 == 1.234"), ImmutableList.of("9"));
- assertFilterMatches(EDF("dim3 < '2'"), ImmutableList.of("0", "1", "3", "4", "6", "9"));
+ assertFilterMatches(edf("dim3 == ''"), ImmutableList.of("0"));
+ assertFilterMatches(edf("dim3 == '1'"), ImmutableList.of("3", "4", "6"));
+ assertFilterMatches(edf("dim3 == 'a'"), ImmutableList.of("7"));
+ assertFilterMatches(edf("dim3 == 1"), ImmutableList.of("3", "4", "6"));
+ assertFilterMatches(edf("dim3 == 1.0"), ImmutableList.of("3", "4", "6"));
+ assertFilterMatches(edf("dim3 == 1.234"), ImmutableList.of("9"));
+ assertFilterMatches(edf("dim3 < '2'"), ImmutableList.of("0", "1", "3", "4", "6", "9"));
if (NullHandling.replaceWithDefault()) {
- assertFilterMatches(EDF("dim3 < 2"), ImmutableList.of("0", "3", "4", "6", "7", "9"));
- assertFilterMatches(EDF("dim3 < 2.0"), ImmutableList.of("0", "3", "4", "6", "7", "9"));
+ assertFilterMatches(edf("dim3 < 2"), ImmutableList.of("0", "3", "4", "6", "7", "9"));
+ assertFilterMatches(edf("dim3 < 2.0"), ImmutableList.of("0", "3", "4", "6", "7", "9"));
} else {
// Empty String and "a" will not match
- assertFilterMatches(EDF("dim3 < 2"), ImmutableList.of("3", "4", "6", "9"));
- assertFilterMatches(EDF("dim3 < 2.0"), ImmutableList.of("3", "4", "6", "9"));
+ assertFilterMatches(edf("dim3 < 2"), ImmutableList.of("3", "4", "6", "9"));
+ assertFilterMatches(edf("dim3 < 2.0"), ImmutableList.of("3", "4", "6", "9"));
}
- assertFilterMatches(EDF("like(dim3, '1%')"), ImmutableList.of("1", "3", "4", "6", "9"));
+ assertFilterMatches(edf("like(dim3, '1%')"), ImmutableList.of("1", "3", "4", "6", "9"));
}
@Test
@@ -141,124 +141,124 @@ public class ExpressionFilterTest extends BaseFilterTest
// Expressions currently treat multi-valued arrays as nulls.
// This test is just documenting the current behavior, not necessarily saying it makes sense.
if (NullHandling.replaceWithDefault()) {
- assertFilterMatches(EDF("dim4 == ''"), ImmutableList.of("0", "1", "2", "4", "5", "6", "7", "8"));
+ assertFilterMatches(edf("dim4 == ''"), ImmutableList.of("0", "1", "2", "4", "5", "6", "7", "8"));
} else {
- assertFilterMatches(EDF("dim4 == ''"), ImmutableList.of("2"));
+ assertFilterMatches(edf("dim4 == ''"), ImmutableList.of("2"));
// AS per SQL standard null == null returns false.
- assertFilterMatches(EDF("dim4 == null"), ImmutableList.of());
+ assertFilterMatches(edf("dim4 == null"), ImmutableList.of());
}
- assertFilterMatches(EDF("dim4 == '1'"), ImmutableList.of());
- assertFilterMatches(EDF("dim4 == '3'"), ImmutableList.of("3"));
+ assertFilterMatches(edf("dim4 == '1'"), ImmutableList.of());
+ assertFilterMatches(edf("dim4 == '3'"), ImmutableList.of("3"));
}
@Test
public void testOneLongColumn()
{
if (NullHandling.replaceWithDefault()) {
- assertFilterMatches(EDF("dim1 == ''"), ImmutableList.of("0"));
+ assertFilterMatches(edf("dim1 == ''"), ImmutableList.of("0"));
} else {
// A long does not match empty string
- assertFilterMatches(EDF("dim1 == ''"), ImmutableList.of());
+ assertFilterMatches(edf("dim1 == ''"), ImmutableList.of());
}
- assertFilterMatches(EDF("dim1 == '1'"), ImmutableList.of("1"));
- assertFilterMatches(EDF("dim1 == 2"), ImmutableList.of("2"));
- assertFilterMatches(EDF("dim1 < '2'"), ImmutableList.of("0", "1"));
- assertFilterMatches(EDF("dim1 < 2"), ImmutableList.of("0", "1"));
- assertFilterMatches(EDF("dim1 < 2.0"), ImmutableList.of("0", "1"));
- assertFilterMatches(EDF("like(dim1, '1%')"), ImmutableList.of("1"));
+ assertFilterMatches(edf("dim1 == '1'"), ImmutableList.of("1"));
+ assertFilterMatches(edf("dim1 == 2"), ImmutableList.of("2"));
+ assertFilterMatches(edf("dim1 < '2'"), ImmutableList.of("0", "1"));
+ assertFilterMatches(edf("dim1 < 2"), ImmutableList.of("0", "1"));
+ assertFilterMatches(edf("dim1 < 2.0"), ImmutableList.of("0", "1"));
+ assertFilterMatches(edf("like(dim1, '1%')"), ImmutableList.of("1"));
}
@Test
public void testOneFloatColumn()
{
if (NullHandling.replaceWithDefault()) {
- assertFilterMatches(EDF("dim2 == ''"), ImmutableList.of("0"));
+ assertFilterMatches(edf("dim2 == ''"), ImmutableList.of("0"));
} else {
// A float does not match empty string
- assertFilterMatches(EDF("dim2 == ''"), ImmutableList.of());
+ assertFilterMatches(edf("dim2 == ''"), ImmutableList.of());
}
- assertFilterMatches(EDF("dim2 == '1'"), ImmutableList.of("1"));
- assertFilterMatches(EDF("dim2 == 2"), ImmutableList.of("2"));
- assertFilterMatches(EDF("dim2 < '2'"), ImmutableList.of("0", "1"));
- assertFilterMatches(EDF("dim2 < 2"), ImmutableList.of("0", "1"));
- assertFilterMatches(EDF("dim2 < 2.0"), ImmutableList.of("0", "1"));
- assertFilterMatches(EDF("like(dim2, '1%')"), ImmutableList.of("1"));
+ assertFilterMatches(edf("dim2 == '1'"), ImmutableList.of("1"));
+ assertFilterMatches(edf("dim2 == 2"), ImmutableList.of("2"));
+ assertFilterMatches(edf("dim2 < '2'"), ImmutableList.of("0", "1"));
+ assertFilterMatches(edf("dim2 < 2"), ImmutableList.of("0", "1"));
+ assertFilterMatches(edf("dim2 < 2.0"), ImmutableList.of("0", "1"));
+ assertFilterMatches(edf("like(dim2, '1%')"), ImmutableList.of("1"));
}
@Test
public void testConstantExpression()
{
- assertFilterMatches(EDF("1 + 1"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
- assertFilterMatches(EDF("0 + 0"), ImmutableList.of());
+ assertFilterMatches(edf("1 + 1"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
+ assertFilterMatches(edf("0 + 0"), ImmutableList.of());
}
@Test
public void testCompareColumns()
{
// String vs string
- assertFilterMatches(EDF("dim0 == dim3"), ImmutableList.of("2", "5", "8"));
+ assertFilterMatches(edf("dim0 == dim3"), ImmutableList.of("2", "5", "8"));
if (NullHandling.replaceWithDefault()) {
// String vs long
- assertFilterMatches(EDF("dim1 == dim3"), ImmutableList.of("0", "2", "5", "8"));
+ assertFilterMatches(edf("dim1 == dim3"), ImmutableList.of("0", "2", "5", "8"));
// String vs float
- assertFilterMatches(EDF("dim2 == dim3"), ImmutableList.of("0", "2", "5", "8"));
+ assertFilterMatches(edf("dim2 == dim3"), ImmutableList.of("0", "2", "5", "8"));
} else {
// String vs long
- assertFilterMatches(EDF("dim1 == dim3"), ImmutableList.of("2", "5", "8"));
+ assertFilterMatches(edf("dim1 == dim3"), ImmutableList.of("2", "5", "8"));
// String vs float
- assertFilterMatches(EDF("dim2 == dim3"), ImmutableList.of("2", "5", "8"));
+ assertFilterMatches(edf("dim2 == dim3"), ImmutableList.of("2", "5", "8"));
}
// String vs. multi-value string
// Expressions currently treat multi-valued arrays as nulls.
// This test is just documenting the current behavior, not necessarily saying it makes sense.
- assertFilterMatches(EDF("dim0 == dim4"), ImmutableList.of("3"));
+ assertFilterMatches(edf("dim0 == dim4"), ImmutableList.of("3"));
}
@Test
public void testMissingColumn()
{
if (NullHandling.replaceWithDefault()) {
- assertFilterMatches(EDF("missing == ''"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
+ assertFilterMatches(edf("missing == ''"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
} else {
// AS per SQL standard null == null returns false.
- assertFilterMatches(EDF("missing == null"), ImmutableList.of());
+ assertFilterMatches(edf("missing == null"), ImmutableList.of());
}
- assertFilterMatches(EDF("missing == '1'"), ImmutableList.of());
- assertFilterMatches(EDF("missing == 2"), ImmutableList.of());
+ assertFilterMatches(edf("missing == '1'"), ImmutableList.of());
+ assertFilterMatches(edf("missing == 2"), ImmutableList.of());
if (NullHandling.replaceWithDefault()) {
// missing equivaluent to 0
- assertFilterMatches(EDF("missing < '2'"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
- assertFilterMatches(EDF("missing < 2"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
- assertFilterMatches(EDF("missing < 2.0"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
+ assertFilterMatches(edf("missing < '2'"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
+ assertFilterMatches(edf("missing < 2"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
+ assertFilterMatches(edf("missing < 2.0"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
} else {
// missing equivalent to null
- assertFilterMatches(EDF("missing < '2'"), ImmutableList.of());
- assertFilterMatches(EDF("missing < 2"), ImmutableList.of());
- assertFilterMatches(EDF("missing < 2.0"), ImmutableList.of());
+ assertFilterMatches(edf("missing < '2'"), ImmutableList.of());
+ assertFilterMatches(edf("missing < 2"), ImmutableList.of());
+ assertFilterMatches(edf("missing < 2.0"), ImmutableList.of());
}
- assertFilterMatches(EDF("missing > '2'"), ImmutableList.of());
- assertFilterMatches(EDF("missing > 2"), ImmutableList.of());
- assertFilterMatches(EDF("missing > 2.0"), ImmutableList.of());
- assertFilterMatches(EDF("like(missing, '1%')"), ImmutableList.of());
+ assertFilterMatches(edf("missing > '2'"), ImmutableList.of());
+ assertFilterMatches(edf("missing > 2"), ImmutableList.of());
+ assertFilterMatches(edf("missing > 2.0"), ImmutableList.of());
+ assertFilterMatches(edf("like(missing, '1%')"), ImmutableList.of());
}
@Test
public void testGetRequiredColumn()
{
- Assert.assertEquals(EDF("like(dim1, '1%')").getRequiredColumns(), Sets.newHashSet("dim1"));
- Assert.assertEquals(EDF("dim2 == '1'").getRequiredColumns(), Sets.newHashSet("dim2"));
- Assert.assertEquals(EDF("dim3 < '2'").getRequiredColumns(), Sets.newHashSet("dim3"));
- Assert.assertEquals(EDF("dim4 == ''").getRequiredColumns(), Sets.newHashSet("dim4"));
- Assert.assertEquals(EDF("1 + 1").getRequiredColumns(), new HashSet<>());
- Assert.assertEquals(EDF("dim0 == dim3").getRequiredColumns(), Sets.newHashSet("dim0", "dim3"));
- Assert.assertEquals(EDF("missing == ''").getRequiredColumns(), Sets.newHashSet("missing"));
+ Assert.assertEquals(edf("like(dim1, '1%')").getRequiredColumns(), Sets.newHashSet("dim1"));
+ Assert.assertEquals(edf("dim2 == '1'").getRequiredColumns(), Sets.newHashSet("dim2"));
+ Assert.assertEquals(edf("dim3 < '2'").getRequiredColumns(), Sets.newHashSet("dim3"));
+ Assert.assertEquals(edf("dim4 == ''").getRequiredColumns(), Sets.newHashSet("dim4"));
+ Assert.assertEquals(edf("1 + 1").getRequiredColumns(), new HashSet<>());
+ Assert.assertEquals(edf("dim0 == dim3").getRequiredColumns(), Sets.newHashSet("dim0", "dim3"));
+ Assert.assertEquals(edf("missing == ''").getRequiredColumns(), Sets.newHashSet("missing"));
}
- private static ExpressionDimFilter EDF(final String expression)
+ private static ExpressionDimFilter edf(final String expression)
{
return new ExpressionDimFilter(expression, TestExprMacroTable.INSTANCE);
}
diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java
index b860b00..15a650b 100644
--- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java
+++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java
@@ -100,8 +100,8 @@ public class AppenderatorPlumberTest
// getDataSource
Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource());
- InputRow[] rows = new InputRow[] {AppenderatorTest.IR("2000", "foo", 1),
- AppenderatorTest.IR("2000", "bar", 2), AppenderatorTest.IR("2000", "qux", 4)};
+ InputRow[] rows = new InputRow[] {AppenderatorTest.ir("2000", "foo", 1),
+ AppenderatorTest.ir("2000", "bar", 2), AppenderatorTest.ir("2000", "qux", 4)};
// add
Assert.assertEquals(1, plumber.add(rows[0], null).getRowCount());
diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTest.java
index e0f67dc..334214d 100644
--- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTest.java
+++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTest.java
@@ -58,9 +58,9 @@ import java.util.concurrent.atomic.AtomicInteger;
public class AppenderatorTest
{
private static final List<SegmentIdWithShardSpec> IDENTIFIERS = ImmutableList.of(
- SI("2000/2001", "A", 0),
- SI("2000/2001", "A", 1),
- SI("2001/2002", "A", 0)
+ si("2000/2001", "A", 0),
+ si("2000/2001", "A", 1),
+ si("2001/2002", "A", 0)
);
@Test
@@ -83,21 +83,21 @@ public class AppenderatorTest
commitMetadata.put("x", "1");
Assert.assertEquals(
1,
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier)
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier)
.getNumRowsInSegment()
);
commitMetadata.put("x", "2");
Assert.assertEquals(
2,
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "bar", 2), committerSupplier)
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar", 2), committerSupplier)
.getNumRowsInSegment()
);
commitMetadata.put("x", "3");
Assert.assertEquals(
1,
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "qux", 4), committerSupplier)
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 4), committerSupplier)
.getNumRowsInSegment()
);
@@ -173,14 +173,14 @@ public class AppenderatorTest
};
appenderator.startJob();
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
//expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 10 (dimsKeySize) = 138 + 1 byte when null handling is enabled
int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0;
Assert.assertEquals(
138 + nullHandlingOverhead,
((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0))
);
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals(
138 + nullHandlingOverhead,
((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1))
@@ -216,11 +216,11 @@ public class AppenderatorTest
};
appenderator.startJob();
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
//expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 10 (dimsKeySize) = 138
int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0;
Assert.assertEquals(138 + nullHandlingOverhead, ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory());
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals(
276 + 2 * nullHandlingOverhead,
((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()
@@ -258,7 +258,7 @@ public class AppenderatorTest
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.startJob();
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
//we still calculate the size even when ignoring it to make persist decision
int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0;
Assert.assertEquals(
@@ -266,7 +266,7 @@ public class AppenderatorTest
((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0))
);
Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals(
276 + 2 * nullHandlingOverhead,
((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()
@@ -310,17 +310,17 @@ public class AppenderatorTest
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.startJob();
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "baz", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "baz", 1), committerSupplier);
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "qux", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 1), committerSupplier);
Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "bob", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 1), committerSupplier);
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.persistAll(committerSupplier.get());
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
@@ -356,17 +356,17 @@ public class AppenderatorTest
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.startJob();
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier, false);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier, false);
Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier, false);
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier, false);
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier, false);
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier, false);
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "baz", 1), committerSupplier, false);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "baz", 1), committerSupplier, false);
Assert.assertEquals(3, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "qux", 1), committerSupplier, false);
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 1), committerSupplier, false);
Assert.assertEquals(4, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "bob", 1), committerSupplier, false);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 1), committerSupplier, false);
Assert.assertEquals(5, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.persistAll(committerSupplier.get());
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
@@ -409,15 +409,15 @@ public class AppenderatorTest
appenderator.startJob();
eventCount.incrementAndGet();
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
eventCount.incrementAndGet();
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "bar", 2), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar", 2), committerSupplier);
eventCount.incrementAndGet();
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "baz", 3), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "baz", 3), committerSupplier);
eventCount.incrementAndGet();
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "qux", 4), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "qux", 4), committerSupplier);
eventCount.incrementAndGet();
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "bob", 5), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 5), committerSupplier);
appenderator.close();
try (final AppenderatorTester tester2 = new AppenderatorTester(
@@ -445,9 +445,9 @@ public class AppenderatorTest
Assert.assertEquals(0, appenderator.getTotalRowCount());
appenderator.startJob();
Assert.assertEquals(0, appenderator.getTotalRowCount());
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier);
Assert.assertEquals(1, appenderator.getTotalRowCount());
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier);
Assert.assertEquals(2, appenderator.getTotalRowCount());
appenderator.persistAll(committerSupplier.get()).get();
@@ -457,13 +457,13 @@ public class AppenderatorTest
appenderator.drop(IDENTIFIERS.get(1)).get();
Assert.assertEquals(0, appenderator.getTotalRowCount());
- appenderator.add(IDENTIFIERS.get(2), IR("2001", "bar", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(2), ir("2001", "bar", 1), committerSupplier);
Assert.assertEquals(1, appenderator.getTotalRowCount());
- appenderator.add(IDENTIFIERS.get(2), IR("2001", "baz", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(2), ir("2001", "baz", 1), committerSupplier);
Assert.assertEquals(2, appenderator.getTotalRowCount());
- appenderator.add(IDENTIFIERS.get(2), IR("2001", "qux", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(2), ir("2001", "qux", 1), committerSupplier);
Assert.assertEquals(3, appenderator.getTotalRowCount());
- appenderator.add(IDENTIFIERS.get(2), IR("2001", "bob", 1), committerSupplier);
+ appenderator.add(IDENTIFIERS.get(2), ir("2001", "bob", 1), committerSupplier);
Assert.assertEquals(4, appenderator.getTotalRowCount());
appenderator.persistAll(committerSupplier.get()).get();
@@ -483,13 +483,13 @@ public class AppenderatorTest
final Appenderator appenderator = tester.getAppenderator();
appenderator.startJob();
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 2), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "foo", 4), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(2), IR("2001", "foo", 8), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(2), IR("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(2), IR("2001T02", "foo", 32), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(2), IR("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 2), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "foo", 4), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(2), ir("2001", "foo", 8), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(2), ir("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(2), ir("2001T02", "foo", 32), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(2), ir("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil()));
// Query1: 2000/2001
final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder()
@@ -619,13 +619,13 @@ public class AppenderatorTest
final Appenderator appenderator = tester.getAppenderator();
appenderator.startJob();
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 2), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(1), IR("2000", "foo", 4), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(2), IR("2001", "foo", 8), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(2), IR("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(2), IR("2001T02", "foo", 32), Suppliers.ofInstance(Committers.nil()));
- appenderator.add(IDENTIFIERS.get(2), IR("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 2), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(1), ir("2000", "foo", 4), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(2), ir("2001", "foo", 8), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(2), ir("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(2), ir("2001T02", "foo", 32), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(IDENTIFIERS.get(2), ir("2001T03", "foo", 64), Suppliers.ofInstance(Committers.nil()));
// Query1: segment #2
final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder()
@@ -742,7 +742,7 @@ public class AppenderatorTest
}
}
- private static SegmentIdWithShardSpec SI(String interval, String version, int partitionNum)
+ private static SegmentIdWithShardSpec si(String interval, String version, int partitionNum)
{
return new SegmentIdWithShardSpec(
AppenderatorTester.DATASOURCE,
@@ -752,7 +752,7 @@ public class AppenderatorTest
);
}
- static InputRow IR(String ts, String dim, long met)
+ static InputRow ir(String ts, String dim, long met)
{
return new MapBasedInputRow(
DateTimes.of(ts).getMillis(),
diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java
index fbf85b4..e02ca6d 100644
--- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java
+++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java
@@ -167,9 +167,9 @@ public class DefaultOfflineAppenderatorFactoryTest
new LinearShardSpec(0)
);
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(identifier, AppenderatorTest.IR("2000", "bar", 1), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(identifier, AppenderatorTest.ir("2000", "bar", 1), Suppliers.ofInstance(Committers.nil()));
Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory());
- appenderator.add(identifier, AppenderatorTest.IR("2000", "baz", 1), Suppliers.ofInstance(Committers.nil()));
+ appenderator.add(identifier, AppenderatorTest.ir("2000", "baz", 1), Suppliers.ofInstance(Committers.nil()));
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
appenderator.close();
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java
index 4f25a66..425632d 100644
--- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java
@@ -362,7 +362,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = client.getMetaData();
Assert.assertEquals(
ImmutableList.of(
- ROW(Pair.of("TABLE_CAT", "druid"))
+ row(Pair.of("TABLE_CAT", "druid"))
),
getRows(metaData.getCatalogs())
);
@@ -374,7 +374,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = client.getMetaData();
Assert.assertEquals(
ImmutableList.of(
- ROW(Pair.of("TABLE_CATALOG", "druid"), Pair.of("TABLE_SCHEM", "druid"))
+ row(Pair.of("TABLE_CATALOG", "druid"), Pair.of("TABLE_SCHEM", "druid"))
),
getRows(metaData.getSchemas(null, "druid"))
);
@@ -386,19 +386,19 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = client.getMetaData();
Assert.assertEquals(
ImmutableList.of(
- ROW(
+ row(
Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE1),
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_TYPE", "TABLE")
),
- ROW(
+ row(
Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE2),
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_TYPE", "TABLE")
),
- ROW(
+ row(
Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE3),
Pair.of("TABLE_SCHEM", "druid"),
@@ -418,25 +418,25 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = superuserClient.getMetaData();
Assert.assertEquals(
ImmutableList.of(
- ROW(
+ row(
Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE1),
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_TYPE", "TABLE")
),
- ROW(
+ row(
Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE2),
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_TYPE", "TABLE")
),
- ROW(
+ row(
Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_TYPE", "TABLE")
),
- ROW(
+ row(
Pair.of("TABLE_CAT", "druid"),
Pair.of("TABLE_NAME", CalciteTests.DATASOURCE3),
Pair.of("TABLE_SCHEM", "druid"),
@@ -456,7 +456,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = client.getMetaData();
Assert.assertEquals(
ImmutableList.of(
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "__time"),
@@ -464,7 +464,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "TIMESTAMP"),
Pair.of("IS_NULLABLE", "NO")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "cnt"),
@@ -472,7 +472,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "BIGINT"),
Pair.of("IS_NULLABLE", "NO")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "dim1"),
@@ -480,7 +480,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "VARCHAR"),
Pair.of("IS_NULLABLE", "YES")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "dim2"),
@@ -488,7 +488,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "VARCHAR"),
Pair.of("IS_NULLABLE", "YES")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "dim3"),
@@ -496,7 +496,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "VARCHAR"),
Pair.of("IS_NULLABLE", "YES")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "m1"),
@@ -504,7 +504,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "FLOAT"),
Pair.of("IS_NULLABLE", "NO")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "m2"),
@@ -512,7 +512,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "DOUBLE"),
Pair.of("IS_NULLABLE", "NO")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", "foo"),
Pair.of("COLUMN_NAME", "unique_dim1"),
@@ -547,7 +547,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final DatabaseMetaData metaData = superuserClient.getMetaData();
Assert.assertEquals(
ImmutableList.of(
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "__time"),
@@ -555,7 +555,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "TIMESTAMP"),
Pair.of("IS_NULLABLE", "NO")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "cnt"),
@@ -563,7 +563,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "BIGINT"),
Pair.of("IS_NULLABLE", "NO")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "dim1"),
@@ -571,7 +571,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "VARCHAR"),
Pair.of("IS_NULLABLE", "YES")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "dim2"),
@@ -579,7 +579,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "VARCHAR"),
Pair.of("IS_NULLABLE", "YES")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "m1"),
@@ -587,7 +587,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "FLOAT"),
Pair.of("IS_NULLABLE", "NO")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "m2"),
@@ -595,7 +595,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
Pair.of("TYPE_NAME", "DOUBLE"),
Pair.of("IS_NULLABLE", "NO")
),
- ROW(
+ row(
Pair.of("TABLE_SCHEM", "druid"),
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
Pair.of("COLUMN_NAME", "unique_dim1"),
@@ -928,7 +928,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
}
}
- private static Map<String, Object> ROW(final Pair<String, ?>... entries)
+ private static Map<String, Object> row(final Pair<String, ?>... entries)
{
final Map<String, Object> m = new HashMap<>();
for (Pair<String, ?> entry : entries) {
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
index e963517..4ee3b1f 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
@@ -243,70 +243,70 @@ public class BaseCalciteQueryTest extends CalciteTestBase
}
// Generate timestamps for expected results
- public static long T(final String timeString)
+ public static long t(final String timeString)
{
return Calcites.jodaToCalciteTimestamp(DateTimes.of(timeString), DateTimeZone.UTC);
}
// Generate timestamps for expected results
- public static long T(final String timeString, final String timeZoneString)
+ public static long t(final String timeString, final String timeZoneString)
{
final DateTimeZone timeZone = DateTimes.inferTzFromString(timeZoneString);
return Calcites.jodaToCalciteTimestamp(new DateTime(timeString, timeZone), timeZone);
}
// Generate day numbers for expected results
- public static int D(final String dayString)
+ public static int d(final String dayString)
{
- return (int) (Intervals.utc(T("1970"), T(dayString)).toDurationMillis() / (86400L * 1000L));
+ return (int) (Intervals.utc(t("1970"), t(dayString)).toDurationMillis() / (86400L * 1000L));
}
- public static QuerySegmentSpec QSS(final Interval... intervals)
+ public static QuerySegmentSpec querySegmentSpec(final Interval... intervals)
{
return new MultipleIntervalSegmentSpec(Arrays.asList(intervals));
}
- public static AndDimFilter AND(DimFilter... filters)
+ public static AndDimFilter and(DimFilter... filters)
{
return new AndDimFilter(Arrays.asList(filters));
}
- public static OrDimFilter OR(DimFilter... filters)
+ public static OrDimFilter or(DimFilter... filters)
{
return new OrDimFilter(Arrays.asList(filters));
}
- public static NotDimFilter NOT(DimFilter filter)
+ public static NotDimFilter not(DimFilter filter)
{
return new NotDimFilter(filter);
}
- public static InDimFilter IN(String dimension, List<String> values, ExtractionFn extractionFn)
+ public static InDimFilter in(String dimension, List<String> values, ExtractionFn extractionFn)
{
return new InDimFilter(dimension, values, extractionFn);
}
- public static SelectorDimFilter SELECTOR(final String fieldName, final String value, final ExtractionFn extractionFn)
+ public static SelectorDimFilter selector(final String fieldName, final String value, final ExtractionFn extractionFn)
{
return new SelectorDimFilter(fieldName, value, extractionFn);
}
- public static ExpressionDimFilter EXPRESSION_FILTER(final String expression)
+ public static ExpressionDimFilter expressionFilter(final String expression)
{
return new ExpressionDimFilter(expression, CalciteTests.createExprMacroTable());
}
- public static DimFilter NUMERIC_SELECTOR(
+ public static DimFilter numeric_Selector(
final String fieldName,
final String value,
final ExtractionFn extractionFn
)
{
// We use Bound filters for numeric equality to achieve "10.0" = "10"
- return BOUND(fieldName, value, value, false, false, extractionFn, StringComparators.NUMERIC);
+ return bound(fieldName, value, value, false, false, extractionFn, StringComparators.NUMERIC);
}
- public static BoundDimFilter BOUND(
+ public static BoundDimFilter bound(
final String fieldName,
final String lower,
final String upper,
@@ -319,7 +319,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
return new BoundDimFilter(fieldName, lower, upper, lowerStrict, upperStrict, null, extractionFn, comparator);
}
- public static BoundDimFilter TIME_BOUND(final Object intervalObj)
+ public static BoundDimFilter timeBound(final Object intervalObj)
{
final Interval interval = new Interval(intervalObj, ISOChronology.getInstanceUTC());
return new BoundDimFilter(
@@ -329,32 +329,32 @@ public class BaseCalciteQueryTest extends CalciteTestBase
false,
true,
null,
- null,
+ null,
StringComparators.NUMERIC
);
}
- public static CascadeExtractionFn CASCADE(final ExtractionFn... fns)
+ public static CascadeExtractionFn cascade(final ExtractionFn... fns)
{
return new CascadeExtractionFn(fns);
}
- public static List<DimensionSpec> DIMS(final DimensionSpec... dimensionSpecs)
+ public static List<DimensionSpec> dimensionSpec(final DimensionSpec... dimensionSpecs)
{
return Arrays.asList(dimensionSpecs);
}
- public static List<AggregatorFactory> AGGS(final AggregatorFactory... aggregators)
+ public static List<AggregatorFactory> aggregators(final AggregatorFactory... aggregators)
{
return Arrays.asList(aggregators);
}
- public static DimFilterHavingSpec HAVING(final DimFilter filter)
+ public static DimFilterHavingSpec having(final DimFilter filter)
{
return new DimFilterHavingSpec(filter, true);
}
- public static ExpressionVirtualColumn EXPRESSION_VIRTUAL_COLUMN(
+ public static ExpressionVirtualColumn expression_Virtual_Column(
final String name,
final String expression,
final ValueType outputType
@@ -363,7 +363,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
return new ExpressionVirtualColumn(name, expression, outputType, CalciteTests.createExprMacroTable());
}
- public static ExpressionPostAggregator EXPRESSION_POST_AGG(final String name, final String expression)
+ public static ExpressionPostAggregator expresionPostAgg(final String name, final String expression)
{
return new ExpressionPostAggregator(name, expression, null, CalciteTests.createExprMacroTable());
}
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
index b89ae45..ad6030f 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
@@ -108,8 +108,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .virtualColumns(EXPRESSION_VIRTUAL_COLUMN("v0", "2", ValueType.LONG))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .virtualColumns(expression_Virtual_Column("v0", "2", ValueType.LONG))
.columns("dim1", "v0")
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
.limit(1)
@@ -133,15 +133,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .filters(SELECTOR("dim2", "0", null))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .filters(selector("dim2", "0", null))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new CountAggregatorFactory("a0"),
new DoubleSumAggregatorFactory("a1", "m2")
))
.postAggregators(
- EXPRESSION_POST_AGG("p0", "(exp(\"a0\") + 10)")
+ expresionPostAgg("p0", "(exp(\"a0\") + 10)")
)
.context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS)
.build()),
@@ -157,14 +157,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z")))
+ .intervals(querySegmentSpec(Intervals.of("2999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z")))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new CountAggregatorFactory("a0"),
new DoubleSumAggregatorFactory("a1", "m2")
))
.postAggregators(
- EXPRESSION_POST_AGG("p0", "(exp(\"a0\") + 10)")
+ expresionPostAgg("p0", "(exp(\"a0\") + 10)")
)
.context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS)
.build()),
@@ -177,10 +177,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
"SELECT COUNT(*) FROM foo WHERE dim1 = 'nonexistent' GROUP BY FLOOR(__time TO DAY)",
ImmutableList.of(Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .filters(SELECTOR("dim1", "nonexistent", null))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .filters(selector("dim1", "nonexistent", null))
.granularity(Granularities.DAY)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new CountAggregatorFactory("a0")
))
.context(TIMESERIES_CONTEXT_DEFAULT)
@@ -212,21 +212,21 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.postAggregators(
- EXPRESSION_POST_AGG("p0", "'foo'"),
- EXPRESSION_POST_AGG("p1", "'xfoo'"),
- EXPRESSION_POST_AGG("p2", "'foo'"),
- EXPRESSION_POST_AGG("p3", "' foo'"),
- EXPRESSION_POST_AGG("p4", "'foo'"),
- EXPRESSION_POST_AGG("p5", "'foo'"),
- EXPRESSION_POST_AGG("p6", "'foo'"),
- EXPRESSION_POST_AGG("p7", "'foo '"),
- EXPRESSION_POST_AGG("p8", "'foox'"),
- EXPRESSION_POST_AGG("p9", "' foo'"),
- EXPRESSION_POST_AGG("p10", "'xfoo'")
+ expresionPostAgg("p0", "'foo'"),
+ expresionPostAgg("p1", "'xfoo'"),
+ expresionPostAgg("p2", "'foo'"),
+ expresionPostAgg("p3", "' foo'"),
+ expresionPostAgg("p4", "'foo'"),
+ expresionPostAgg("p5", "'foo'"),
+ expresionPostAgg("p6", "'foo'"),
+ expresionPostAgg("p7", "'foo '"),
+ expresionPostAgg("p8", "'foox'"),
+ expresionPostAgg("p9", "' foo'"),
+ expresionPostAgg("p10", "'xfoo'")
)
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
@@ -429,21 +429,21 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1")
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{T("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1f, 1.0, hyperLogLogCollectorClassName},
+ new Object[]{t("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1f, 1.0, hyperLogLogCollectorClassName},
new Object[]{
- T("2000-01-02"), 1L, "10.1", NULL_VALUE, "[\"b\",\"c\"]", 2f, 2.0, hyperLogLogCollectorClassName
+ t("2000-01-02"), 1L, "10.1", NULL_VALUE, "[\"b\",\"c\"]", 2f, 2.0, hyperLogLogCollectorClassName
},
- new Object[]{T("2000-01-03"), 1L, "2", "", "d", 3f, 3.0, hyperLogLogCollectorClassName},
- new Object[]{T("2001-01-01"), 1L, "1", "a", "", 4f, 4.0, hyperLogLogCollectorClassName},
- new Object[]{T("2001-01-02"), 1L, "def", "abc", NULL_VALUE, 5f, 5.0, hyperLogLogCollectorClassName},
- new Object[]{T("2001-01-03"), 1L, "abc", NULL_VALUE, NULL_VALUE, 6f, 6.0, hyperLogLogCollectorClassName}
+ new Object[]{t("2000-01-03"), 1L, "2", "", "d", 3f, 3.0, hyperLogLogCollectorClassName},
+ new Object[]{t("2001-01-01"), 1L, "1", "a", "", 4f, 4.0, hyperLogLogCollectorClassName},
+ new Object[]{t("2001-01-02"), 1L, "def", "abc", NULL_VALUE, 5f, 5.0, hyperLogLogCollectorClassName},
+ new Object[]{t("2001-01-03"), 1L, "abc", NULL_VALUE, NULL_VALUE, 6f, 6.0, hyperLogLogCollectorClassName}
)
);
}
@@ -463,7 +463,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.FORBIDDEN_DATASOURCE)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1")
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
@@ -471,7 +471,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
),
ImmutableList.of(
new Object[]{
- T("2000-01-01"),
+ t("2000-01-01"),
1L,
"forbidden",
"abcd",
@@ -491,9 +491,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -528,7 +528,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1")
.limit(2)
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
@@ -536,8 +536,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build()
),
ImmutableList.of(
- new Object[]{T("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1.0f, 1.0, HLLC_STRING},
- new Object[]{T("2000-01-02"), 1L, "10.1", NULL_VALUE, "[\"b\",\"c\"]", 2.0f, 2.0, HLLC_STRING}
+ new Object[]{t("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1.0f, 1.0, HLLC_STRING},
+ new Object[]{t("2000-01-02"), 1L, "10.1", NULL_VALUE, "[\"b\",\"c\"]", 2.0f, 2.0, HLLC_STRING}
)
);
}
@@ -550,9 +550,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
- EXPRESSION_VIRTUAL_COLUMN("v0", "substring(\"dim2\", 0, 1)", ValueType.STRING)
+ expression_Virtual_Column("v0", "substring(\"dim2\", 0, 1)", ValueType.STRING)
)
.columns("v0")
.limit(2)
@@ -578,7 +578,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newSelectQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1"))
@@ -588,8 +588,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build()
),
ImmutableList.of(
- new Object[]{T("2001-01-03"), 1L, "abc", NULL_VALUE, NULL_VALUE, 6f, 6d, HLLC_STRING},
- new Object[]{T("2001-01-02"), 1L, "def", "abc", NULL_VALUE, 5f, 5d, HLLC_STRING}
+ new Object[]{t("2001-01-03"), 1L, "abc", NULL_VALUE, NULL_VALUE, 6f, 6d, HLLC_STRING},
+ new Object[]{t("2001-01-02"), 1L, "def", "abc", NULL_VALUE, 5f, 5d, HLLC_STRING}
)
);
}
@@ -605,7 +605,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newSelectQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1"))
@@ -615,7 +615,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build(),
Druids.newSelectQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimensions(ImmutableList.of("dummy"))
.metrics(ImmutableList.of("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1"))
@@ -631,12 +631,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build()
),
ImmutableList.of(
- new Object[]{T("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1f, 1.0, HLLC_STRING},
- new Object[]{T("2000-01-02"), 1L, "10.1", NULL_VALUE, "[\"b\",\"c\"]", 2f, 2.0, HLLC_STRING},
- new Object[]{T("2000-01-03"), 1L, "2", "", "d", 3f, 3.0, HLLC_STRING},
- new Object[]{T("2001-01-01"), 1L, "1", "a", "", 4f, 4.0, HLLC_STRING},
- new Object[]{T("2001-01-02"), 1L, "def", "abc", NULL_VALUE, 5f, 5.0, HLLC_STRING},
- new Object[]{T("2001-01-03"), 1L, "abc", NULL_VALUE, NULL_VALUE, 6f, 6.0, HLLC_STRING}
+ new Object[]{t("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1f, 1.0, HLLC_STRING},
+ new Object[]{t("2000-01-02"), 1L, "10.1", NULL_VALUE, "[\"b\",\"c\"]", 2f, 2.0, HLLC_STRING},
+ new Object[]{t("2000-01-03"), 1L, "2", "", "d", 3f, 3.0, HLLC_STRING},
+ new Object[]{t("2001-01-01"), 1L, "1", "a", "", 4f, 4.0, HLLC_STRING},
+ new Object[]{t("2001-01-02"), 1L, "def", "abc", NULL_VALUE, 5f, 5.0, HLLC_STRING},
+ new Object[]{t("2001-01-03"), 1L, "abc", NULL_VALUE, NULL_VALUE, 6f, 6.0, HLLC_STRING}
)
);
}
@@ -649,7 +649,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim2")
.limit(2)
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
@@ -671,8 +671,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newSelectQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .dimensionSpecs(DIMS(new DefaultDimensionSpec("dim1", "d1")))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .dimensionSpecs(dimensionSpec(new DefaultDimensionSpec("dim1", "d1")))
.granularity(Granularities.ALL)
.descending(true)
.dimensions(ImmutableList.of("dummy"))
@@ -698,8 +698,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new GroupByQuery.Builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setGranularity(Granularities.ALL)
.setLimitSpec(
new DefaultLimitSpec(
@@ -741,15 +741,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim1")
- .filters(NOT(SELECTOR("dim1", "", null)))
+ .filters(not(selector("dim1", "", null)))
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build(),
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim1", "dim2")
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
@@ -796,10 +796,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -817,10 +817,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -839,10 +839,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -876,15 +876,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.MONTH)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{T("2000-01-01"), 3L},
- new Object[]{T("2001-01-01"), 3L}
+ new Object[]{t("2000-01-01"), 3L},
+ new Object[]{t("2001-01-01"), 3L}
)
);
}
@@ -897,10 +897,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -930,10 +930,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("m1", "d0", ValueType.FLOAT)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("m1", "d0", ValueType.FLOAT)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -956,10 +956,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("m2", "d0", ValueType.DOUBLE)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("m2", "d0", ValueType.DOUBLE)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -982,10 +982,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
- .filters(SELECTOR("m1", "1.0", null))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
+ .filters(selector("m1", "1.0", null))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -1003,10 +1003,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
- .filters(SELECTOR("m2", "1.0", null))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
+ .filters(selector("m2", "1.0", null))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -1024,10 +1024,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(new DoubleSumAggregatorFactory("a0", "m1")))
- .setHavingSpec(HAVING(NUMERIC_SELECTOR("a0", "21", null)))
+ .setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1")))
+ .setHavingSpec(having(numeric_Selector("a0", "21", null)))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -1045,12 +1045,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
- .setAggregatorSpecs(AGGS(new DoubleSumAggregatorFactory("a0", "m1")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
+ .setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1")))
.setHavingSpec(
- HAVING(
+ having(
new BoundDimFilter(
"a0",
"1",
@@ -1084,11 +1084,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
.setAggregatorSpecs(
- AGGS(
+ aggregators(
new CardinalityAggregatorFactory(
"a0",
null,
@@ -1101,8 +1101,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
)
)
.setHavingSpec(
- HAVING(
- BOUND(
+ having(
+ bound(
"a0",
"1",
null,
@@ -1141,10 +1141,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
- DIMS(
+ dimensionSpec(
new DefaultDimensionSpec("dim2", "d0", ValueType.STRING),
new DefaultDimensionSpec("m1", "d1", ValueType.FLOAT)
)
@@ -1153,13 +1153,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0", "_d0", ValueType.STRING)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0", "_d0", ValueType.STRING)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setHavingSpec(
- HAVING(
- BOUND(
+ having(
+ bound(
"a0",
"1",
null,
@@ -1190,17 +1190,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
{
testQuery(
PLANNER_CONFIG_FALLBACK,
- "SELECT dim1, CAST(SUM(m1) AS FLOAT) AS m1_sum FROM druid.foo GROUP BY dim1 HAVING CAST(SUM(m1) AS FLOAT) > 1",
+ "SELECT dim1, CASt(SUM(m1) AS FLOAT) AS m1_sum FROM druid.foo GROUP BY dim1 HAVING CAST(SUM(m1) AS FLOAT) > 1",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
- .setAggregatorSpecs(AGGS(new DoubleSumAggregatorFactory("a0", "m1")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
+ .setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1")))
.setHavingSpec(
- HAVING(
+ having(
new BoundDimFilter(
"a0",
"1",
@@ -1234,14 +1234,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimFilter(EXPRESSION_FILTER("((\"m1\" - 1) == \"dim1\")"))
- .setDimensions(DIMS(
+ .setDimFilter(expressionFilter("((\"m1\" - 1) == \"dim1\")"))
+ .setDimensions(dimensionSpec(
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("m1", "d1", ValueType.FLOAT)
))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -1271,20 +1271,20 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
- .setAggregatorSpecs(AGGS(
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
+ .setAggregatorSpecs(aggregators(
new FilteredAggregatorFactory(
new CountAggregatorFactory("a0"),
- NOT(SELECTOR("dim2", "a", null))
+ not(selector("dim2", "a", null))
),
new CountAggregatorFactory("a1")
))
.setPostAggregatorSpecs(ImmutableList.of(
- EXPRESSION_POST_AGG("p0", "(\"a0\" / \"a1\")")
+ expresionPostAgg("p0", "(\"a0\" / \"a1\")")
))
- .setHavingSpec(HAVING(EXPRESSION_FILTER("((\"a0\" / \"a1\") == 1)")))
+ .setHavingSpec(having(expressionFilter("((\"a0\" / \"a1\") == 1)")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -1309,11 +1309,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setPostAggregatorSpecs(ImmutableList.of(
- EXPRESSION_POST_AGG("p0", "substring(\"d0\", 1, -1)")
+ expresionPostAgg("p0", "substring(\"d0\", 1, -1)")
))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
@@ -1342,12 +1342,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setPostAggregatorSpecs(ImmutableList.of(
- EXPRESSION_POST_AGG("p0", "substring(\"d0\", 1, -1)"),
- EXPRESSION_POST_AGG("p1", "strlen(\"d0\")")
+ expresionPostAgg("p0", "substring(\"d0\", 1, -1)"),
+ expresionPostAgg("p1", "strlen(\"d0\")")
))
.setLimitSpec(new DefaultLimitSpec(
ImmutableList.of(
@@ -1391,11 +1391,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim1", "d0"))
.postAggregators(ImmutableList.of(
- EXPRESSION_POST_AGG("p0", "substring(\"d0\", 1, -1)")
+ expresionPostAgg("p0", "substring(\"d0\", 1, -1)")
))
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
.threshold(10)
@@ -1428,12 +1428,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim1", "d0"))
.postAggregators(ImmutableList.of(
- EXPRESSION_POST_AGG("p0", "substring(\"d0\", 1, -1)"),
- EXPRESSION_POST_AGG("p1", "strlen(\"d0\")")
+ expresionPostAgg("p0", "substring(\"d0\", 1, -1)"),
+ expresionPostAgg("p1", "strlen(\"d0\")")
))
.metric(new NumericTopNMetricSpec("p1"))
.threshold(10)
@@ -1459,23 +1459,23 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build(),
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build(),
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -1493,16 +1493,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build(),
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -1526,9 +1526,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new DoubleSumAggregatorFactory("a0", "m1")))
+ .aggregators(aggregators(new DoubleSumAggregatorFactory("a0", "m1")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -1552,10 +1552,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new DoubleSumAggregatorFactory("a0", "m1")))
- .postAggregators(ImmutableList.of(EXPRESSION_POST_AGG("p0", "(\"a0\" / 10)")))
+ .aggregators(aggregators(new DoubleSumAggregatorFactory("a0", "m1")))
+ .postAggregators(ImmutableList.of(expresionPostAgg("p0", "(\"a0\" / 10)")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -1580,10 +1580,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(new DoubleSumAggregatorFactory("a0", "m1")))
- .setHavingSpec(HAVING(NUMERIC_SELECTOR("a0", "21", null)))
+ .setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1")))
+ .setHavingSpec(having(numeric_Selector("a0", "21", null)))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -1612,10 +1612,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"case_searched("
+ "(CAST(timestamp_extract(\"__time\",'DAY','UTC'), 'DOUBLE') == \"m1\"),"
@@ -1628,8 +1628,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ValueType.STRING
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0")))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0")))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -1653,17 +1653,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"case_searched(((\"m1\" > 1) && (\"m1\" < 5) && (\"cnt\" == 1)),'x',null)",
ValueType.STRING
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0")))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0")))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -1684,10 +1684,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .filters(EXPRESSION_FILTER("case_searched((\"dim2\" == 'a'),1,isnull(\"dim2\"))"))
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .filters(expressionFilter("case_searched((\"dim2\" == 'a'),1,isnull(\"dim2\"))"))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -1711,12 +1711,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .filters(EXPRESSION_FILTER("case_searched((\"dim2\" == 'a'),"
+ .filters(expressionFilter("case_searched((\"dim2\" == 'a'),"
+ (NullHandling.replaceWithDefault() ? "1" : "0")
+ ",(\"dim2\" == ''))"))
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -1740,12 +1740,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .filters(EXPRESSION_FILTER("case_searched((\"dim2\" == 'a'),"
+ .filters(expressionFilter("case_searched((\"dim2\" == 'a'),"
+ (NullHandling.replaceWithDefault() ? "1" : "0")
+ ",(\"dim2\" == null))"))
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -1769,17 +1769,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"case_searched(notnull(\"dim2\"),\"dim2\",\"dim1\")",
ValueType.STRING
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.STRING)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.STRING)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -1810,10 +1810,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .filters(SELECTOR("dim2", null, null))
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .filters(selector("dim2", null, null))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -1871,11 +1871,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.filters(
- OR(
- BOUND("dim1", "d", null, true, false, null, StringComparators.LEXICOGRAPHIC),
- SELECTOR("dim2", "a", null)
+ or(
+ bound("dim1", "d", null, true, false, null, StringComparators.LEXICOGRAPHIC),
+ selector("dim2", "a", null)
)
)
.columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1")
@@ -1884,9 +1884,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build()
),
ImmutableList.of(
- new Object[]{T("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1.0f, 1.0d, HLLC_STRING},
- new Object[]{T("2001-01-01"), 1L, "1", "a", "", 4.0f, 4.0d, HLLC_STRING},
- new Object[]{T("2001-01-02"), 1L, "def", "abc", NULL_VALUE, 5.0f, 5.0d, HLLC_STRING}
+ new Object[]{t("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1.0f, 1.0d, HLLC_STRING},
+ new Object[]{t("2001-01-01"), 1L, "1", "a", "", 4.0f, 4.0d, HLLC_STRING},
+ new Object[]{t("2001-01-02"), 1L, "def", "abc", NULL_VALUE, 5.0f, 5.0d, HLLC_STRING}
)
);
}
@@ -1924,10 +1924,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .filters(SELECTOR("dim1", "foobar", null))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .filters(selector("dim1", "foobar", null))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new CountAggregatorFactory("a0"),
new LongMaxAggregatorFactory("a1", "cnt")
))
@@ -1946,10 +1946,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .filters(SELECTOR("dim1", "foobar", null))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .filters(selector("dim1", "foobar", null))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new CountAggregatorFactory("a0"),
new LongMaxAggregatorFactory("a1", "cnt")
))
@@ -1968,9 +1968,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -1988,12 +1988,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new FilteredAggregatorFactory(
new CountAggregatorFactory("a0"),
- NOT(SELECTOR("dim2", null, null))
+ not(selector("dim2", null, null))
)
))
.context(TIMESERIES_CONTEXT_DEFAULT)
@@ -2017,12 +2017,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new FilteredAggregatorFactory(
new CountAggregatorFactory("a0"),
- EXPRESSION_FILTER(
+ expressionFilter(
"notnull(case_searched((\"dim2\" == 'abc'),'yes',(\"dim2\" == 'def'),'yes',"
+ DruidExpression.nullLiteral()
+ "))"
@@ -2046,9 +2046,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2067,13 +2067,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .filters(AND(
- SELECTOR("dim2", "a", null),
- NOT(SELECTOR("dim1", "z", new SubstringDimExtractionFn(0, 1)))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .filters(and(
+ selector("dim2", "a", null),
+ not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1)))
))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2091,13 +2091,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .filters(AND(
- SELECTOR("dim2", "a", null),
- NOT(SELECTOR("dim1", "z", new SubstringDimExtractionFn(0, 1)))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .filters(and(
+ selector("dim2", "a", null),
+ not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1)))
))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2142,15 +2142,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
- OR(
+ or(
new LikeDimFilter("dim1", "a%", null, null),
new LikeDimFilter("dim2", "%xb%", "x", null)
)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2168,15 +2168,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
- OR(
- BOUND("cnt", "3", null, false, false, null, StringComparators.NUMERIC),
- SELECTOR("cnt", "1", null)
+ or(
+ bound("cnt", "3", null, false, false, null, StringComparators.NUMERIC),
+ selector("cnt", "1", null)
)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2194,12 +2194,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
- BOUND("cnt", "1.1", "100000001.0", true, true, null, StringComparators.NUMERIC)
+ bound("cnt", "1.1", "100000001.0", true, true, null, StringComparators.NUMERIC)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2211,12 +2211,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
- SELECTOR("cnt", "1.0", null)
+ selector("cnt", "1.0", null)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2230,12 +2230,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
- SELECTOR("cnt", "100000001.0", null)
+ selector("cnt", "100000001.0", null)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2247,12 +2247,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
- IN("cnt", ImmutableList.of("1.0", "100000001.0"), null)
+ in("cnt", ImmutableList.of("1.0", "100000001.0"), null)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2270,10 +2270,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .filters(IN("cnt", ImmutableList.of("1", "2"), null))
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .filters(in("cnt", ImmutableList.of("1", "2"), null))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2293,15 +2293,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setDimFilter(
- OR(
- SELECTOR("dim1", "10", null),
- AND(
- EXPRESSION_FILTER("(floor(CAST(\"dim1\", 'DOUBLE')) == 10.00)"),
- BOUND("dim1", "9", "10.5", true, false, null, StringComparators.NUMERIC)
+ or(
+ selector("dim1", "10", null),
+ and(
+ expressionFilter("(floor(CAST(\"dim1\", 'DOUBLE')) == 10.00)"),
+ bound("dim1", "9", "10.5", true, false, null, StringComparators.NUMERIC)
)
)
)
@@ -2322,14 +2322,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(
- AGGS(
+ aggregators(
new CountAggregatorFactory("a0"),
new FilteredAggregatorFactory(
new CountAggregatorFactory("a1"),
- NOT(SELECTOR("dim1", null, null))
+ not(selector("dim1", null, null))
),
new LongSumAggregatorFactory("a2:sum", "cnt"),
new CountAggregatorFactory("a2:count"),
@@ -2338,7 +2338,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new LongMaxAggregatorFactory("a5", "cnt"),
new FilteredAggregatorFactory(
new CountAggregatorFactory("a6"),
- NOT(SELECTOR("dim2", null, null))
+ not(selector("dim2", null, null))
)
)
)
@@ -2351,7 +2351,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new FieldAccessPostAggregator(null, "a2:count")
)
),
- EXPRESSION_POST_AGG("p0", "((\"a3\" + \"a4\") + \"a5\")")
+ expresionPostAgg("p0", "((\"a3\" + \"a4\") + \"a5\")")
)
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
@@ -2376,17 +2376,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim1", "d0"))
.metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec("p0")))
- .aggregators(AGGS(
+ .aggregators(aggregators(
new FloatMinAggregatorFactory("a0", "m1"),
new FloatMaxAggregatorFactory("a1", "m1")
))
.postAggregators(
ImmutableList.of(
- EXPRESSION_POST_AGG("p0", "(\"a0\" + \"a1\")")
+ expresionPostAgg("p0", "(\"a0\" + \"a1\")")
)
)
.threshold(3)
@@ -2413,14 +2413,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setAggregatorSpecs(
new FloatMinAggregatorFactory("a0", "m1"),
new FloatMaxAggregatorFactory("a1", "m1")
)
- .setPostAggregatorSpecs(ImmutableList.of(EXPRESSION_POST_AGG("p0", "(\"a0\" + \"a1\")")))
+ .setPostAggregatorSpecs(ImmutableList.of(expresionPostAgg("p0", "(\"a0\" + \"a1\")")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -2457,16 +2457,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setAggregatorSpecs(
new FloatMinAggregatorFactory("a0", "m1"),
new FloatMaxAggregatorFactory("a1", "m1")
)
.setPostAggregatorSpecs(
ImmutableList.of(
- EXPRESSION_POST_AGG("p0", "(\"a0\" + \"a1\")")
+ expresionPostAgg("p0", "(\"a0\" + \"a1\")")
)
)
.setLimitSpec(
@@ -2513,68 +2513,68 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new FilteredAggregatorFactory(
new LongSumAggregatorFactory("a0", "cnt"),
- SELECTOR("dim1", "abc", null)
+ selector("dim1", "abc", null)
),
new FilteredAggregatorFactory(
new LongSumAggregatorFactory("a1", "cnt"),
- NOT(SELECTOR("dim1", "abc", null))
+ not(selector("dim1", "abc", null))
),
new FilteredAggregatorFactory(
new LongSumAggregatorFactory("a2", "cnt"),
- SELECTOR("dim1", "a", new SubstringDimExtractionFn(0, 1))
+ selector("dim1", "a", new SubstringDimExtractionFn(0, 1))
),
new FilteredAggregatorFactory(
new CountAggregatorFactory("a3"),
- AND(
- NOT(SELECTOR("dim2", null, null)),
- NOT(SELECTOR("dim1", "1", null))
+ and(
+ not(selector("dim2", null, null)),
+ not(selector("dim1", "1", null))
)
),
new FilteredAggregatorFactory(
new CountAggregatorFactory("a4"),
- NOT(SELECTOR("dim1", "1", null))
+ not(selector("dim1", "1", null))
),
new FilteredAggregatorFactory(
new CountAggregatorFactory("a5"),
- NOT(SELECTOR("dim1", "1", null))
+ not(selector("dim1", "1", null))
),
new FilteredAggregatorFactory(
new LongSumAggregatorFactory("a6", "cnt"),
- SELECTOR("dim2", "a", null)
+ selector("dim2", "a", null)
),
new FilteredAggregatorFactory(
new LongSumAggregatorFactory("a7", "cnt"),
- AND(
- SELECTOR("dim2", "a", null),
- NOT(SELECTOR("dim1", "1", null))
+ and(
+ selector("dim2", "a", null),
+ not(selector("dim1", "1", null))
)
),
new FilteredAggregatorFactory(
new LongSumAggregatorFactory("a8", "cnt"),
- NOT(SELECTOR("dim1", "1", null))
+ not(selector("dim1", "1", null))
),
new FilteredAggregatorFactory(
new LongMaxAggregatorFactory("a9", "cnt"),
- NOT(SELECTOR("dim1", "1", null))
+ not(selector("dim1", "1", null))
),
new FilteredAggregatorFactory(
new CardinalityAggregatorFactory(
"a10",
null,
- DIMS(new DefaultDimensionSpec("m1", "m1", ValueType.FLOAT)),
+ dimensionSpec(new DefaultDimensionSpec("m1", "m1", ValueType.FLOAT)),
false,
true
),
- NOT(SELECTOR("dim1", "1", null))
+ not(selector("dim1", "1", null))
),
new FilteredAggregatorFactory(
new LongSumAggregatorFactory("a11", "cnt"),
- AND(SELECTOR("dim2", "a", null), SELECTOR("dim1", "b", null))
+ and(selector("dim2", "a", null), selector("dim1", "b", null))
)
))
.context(TIMESERIES_CONTEXT_DEFAULT)
@@ -2602,17 +2602,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(
new FilteredAggregatorFactory(
new CountAggregatorFactory("a0"),
- NOT(SELECTOR("dim1", "1", null))
+ not(selector("dim1", "1", null))
),
new LongSumAggregatorFactory("a1", "cnt")
))
- .setPostAggregatorSpecs(ImmutableList.of(EXPRESSION_POST_AGG("p0", "(\"a0\" + \"a1\")")))
+ .setPostAggregatorSpecs(ImmutableList.of(expresionPostAgg("p0", "(\"a0\" + \"a1\")")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -2633,19 +2633,19 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(
- AGGS(
+ aggregators(
new FilteredAggregatorFactory(
new CountAggregatorFactory("a0"),
- NOT(SELECTOR("dim1", "1", null))
+ not(selector("dim1", "1", null))
),
new FilteredAggregatorFactory(
new CountAggregatorFactory("a1"),
- AND(
- NOT(SELECTOR("dim2", null, null)),
- NOT(SELECTOR("dim1", "1", null))
+ and(
+ not(selector("dim2", null, null)),
+ not(selector("dim1", "1", null))
)
)
)
@@ -2679,9 +2679,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new LongSumAggregatorFactory("a0", null, "(\"cnt\" * 3)", macroTable),
new LongSumAggregatorFactory("a1", "cnt"),
new DoubleSumAggregatorFactory("a2", "m1"),
@@ -2689,8 +2689,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new DoubleMaxAggregatorFactory("a4", null, "(strlen(\"dim2\") + log(\"m1\"))", macroTable)
))
.postAggregators(
- EXPRESSION_POST_AGG("p0", "log((\"a1\" + \"a2\"))"),
- EXPRESSION_POST_AGG("p1", "(\"a1\" % 4)")
+ expresionPostAgg("p0", "log((\"a1\" + \"a2\"))"),
+ expresionPostAgg("p1", "(\"a1\" % 4)")
)
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
@@ -2715,14 +2715,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN("d0:v", "(floor((\"m1\" / 2)) * 2)", ValueType.FLOAT)
+ expression_Virtual_Column("d0:v", "(floor((\"m1\" / 2)) * 2)", ValueType.FLOAT)
)
- .setDimFilter(EXPRESSION_FILTER("((floor((\"m1\" / 2)) * 2) > -1)"))
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.FLOAT)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimFilter(expressionFilter("((floor((\"m1\" / 2)) * 2) > -1)"))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.FLOAT)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -2761,16 +2761,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN("d0:v", "((CAST(\"m1\", 'LONG') / 2) * 2)", ValueType.LONG)
+ expression_Virtual_Column("d0:v", "((CAST(\"m1\", 'LONG') / 2) * 2)", ValueType.LONG)
)
.setDimFilter(
- EXPRESSION_FILTER("(((CAST(\"m1\", 'LONG') / 2) * 2) > -1)")
+ expressionFilter("(((CAST(\"m1\", 'LONG') / 2) * 2) > -1)")
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -2809,20 +2809,20 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"(floor((CAST(\"dim1\", 'DOUBLE') / 2)) * 2)",
ValueType.FLOAT
)
)
.setDimFilter(
- EXPRESSION_FILTER("((floor((CAST(\"dim1\", 'DOUBLE') / 2)) * 2) > -1)")
+ expressionFilter("((floor((CAST(\"dim1\", 'DOUBLE') / 2)) * 2) > -1)")
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.FLOAT)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.FLOAT)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -2860,12 +2860,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setDimFilter(new InDimFilter("dim1", ImmutableList.of("abc", "def", "ghi"), null))
.setAggregatorSpecs(
- AGGS(
+ aggregators(
new CountAggregatorFactory("a0")
)
)
@@ -2899,12 +2899,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setDimFilter(new InDimFilter("dim1", elements, null))
.setAggregatorSpecs(
- AGGS(
+ aggregators(
new CountAggregatorFactory("a0")
)
)
@@ -2926,18 +2926,18 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
- AND(
- SELECTOR("dim2", "a", null),
- OR(
- BOUND("dim1", "a", null, true, false, null, StringComparators.LEXICOGRAPHIC),
- NOT(SELECTOR("dim1", null, null))
+ and(
+ selector("dim2", "a", null),
+ or(
+ bound("dim1", "a", null, true, false, null, StringComparators.LEXICOGRAPHIC),
+ not(selector("dim1", null, null))
)
)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2965,10 +2965,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .filters(BOUND("m1", "2.5", "3.5", true, true, null, StringComparators.NUMERIC))
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .filters(bound("m1", "2.5", "3.5", true, true, null, StringComparators.NUMERIC))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -2986,10 +2986,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .filters(BOUND("dim1", "a", "b", false, true, null, StringComparators.LEXICOGRAPHIC))
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .filters(bound("dim1", "a", "b", false, true, null, StringComparators.LEXICOGRAPHIC))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3007,10 +3007,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .filters(SELECTOR("dim1", "abc", null))
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .filters(selector("dim1", "abc", null))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3028,10 +3028,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .filters(NUMERIC_SELECTOR("dim1", "2", null))
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .filters(numeric_Selector("dim1", "2", null))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3050,9 +3050,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000-01-01/2001-01-01")))
+ .intervals(querySegmentSpec(Intervals.of("2000-01-01/2001-01-01")))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3078,9 +3078,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000/2001"), Intervals.of("2010/2011")))
+ .intervals(querySegmentSpec(Intervals.of("2000/2001"), Intervals.of("2010/2011")))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3101,13 +3101,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(
- QSS(
+ querySegmentSpec(
Intervals.of("2000-01-01T00:00:00.111/2000-01-01T00:00:00.112"),
Intervals.of("2000-01-01T00:00:00.888/2000-01-02T00:00:00.222")
)
)
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3131,14 +3131,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(
- QSS(
+ querySegmentSpec(
Intervals.of("2000-01-01/2001-01-01"),
Intervals.of("2001-02-01/2001-02-02"),
Intervals.of("2001-03-01/2001-03-02T00:00:00.001")
)
)
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3176,9 +3176,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000-01-01/2000-01-01T00:00:00.001")))
+ .intervals(querySegmentSpec(Intervals.of("2000-01-01/2000-01-01T00:00:00.001")))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3198,13 +3198,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(
- QSS(
+ querySegmentSpec(
Intervals.of("2000-01-01/2000-01-01T00:00:00.001"),
Intervals.of("2000-01-02/2000-01-02T00:00:00.001")
)
)
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3230,21 +3230,21 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000/2001"), Intervals.of("2002-05-01/2003-05-01")))
+ .intervals(querySegmentSpec(Intervals.of("2000/2001"), Intervals.of("2002-05-01/2003-05-01")))
.granularity(Granularities.ALL)
.filters(
- AND(
- SELECTOR("dim2", "a", null),
- OR(
- TIME_BOUND("2000/2001"),
- AND(
- SELECTOR("dim1", "abc", null),
- TIME_BOUND("2002-05-01/2003-05-01")
+ and(
+ selector("dim2", "a", null),
+ or(
+ timeBound("2000/2001"),
+ and(
+ selector("dim1", "abc", null),
+ timeBound("2002-05-01/2003-05-01")
)
)
)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3271,21 +3271,21 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.filters(
- OR(
- NOT(SELECTOR("dim2", "a", null)),
- AND(
- NOT(TIME_BOUND("2000/2001")),
- NOT(AND(
- SELECTOR("dim1", "abc", null),
- TIME_BOUND("2002-05-01/2003-05-01")
+ or(
+ not(selector("dim2", "a", null)),
+ and(
+ not(timeBound("2000/2001")),
+ not(and(
+ selector("dim1", "abc", null),
+ timeBound("2002-05-01/2003-05-01")
))
)
)
)
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3307,15 +3307,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(
- QSS(
+ querySegmentSpec(
new Interval(DateTimes.MIN, DateTimes.of("2000")),
Intervals.of("2001/2003"),
new Interval(DateTimes.of("2004"), DateTimes.MAX)
)
)
- .filters(NOT(SELECTOR("dim1", "xxx", null)))
+ .filters(not(selector("dim1", "xxx", null)))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3335,10 +3335,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000-01-01/2001-01-01")))
- .filters(NOT(SELECTOR("dim2", "a", null)))
+ .intervals(querySegmentSpec(Intervals.of("2000-01-01/2001-01-01")))
+ .filters(not(selector("dim2", "a", null)))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3358,14 +3358,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.filters(
- OR(
- NOT(SELECTOR("dim2", "a", null)),
- BOUND(
+ or(
+ not(selector("dim2", "a", null)),
+ bound(
"__time",
- String.valueOf(T("2000-01-01")),
- String.valueOf(T("2000-12-31T23:59:59.999")),
+ String.valueOf(t("2000-01-01")),
+ String.valueOf(t("2000-12-31T23:59:59.999")),
false,
false,
null,
@@ -3374,7 +3374,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
)
)
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3394,10 +3394,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
- BOUND(
+ bound(
"cnt",
String.valueOf(DateTimes.of("1970-01-01").getMillis()),
String.valueOf(DateTimes.of("1970-01-02").getMillis()),
@@ -3407,7 +3407,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
StringComparators.NUMERIC
)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3427,10 +3427,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
- BOUND(
+ bound(
"cnt",
String.valueOf(DateTimes.of("1970-01-01").getMillis()),
String.valueOf(DateTimes.of("1970-01-02").getMillis()),
@@ -3440,7 +3440,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
StringComparators.NUMERIC
)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3460,10 +3460,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(
- BOUND(
+ bound(
"cnt",
String.valueOf(DateTimes.of("1970-01-01").getMillis()),
String.valueOf(DateTimes.of("1970-01-02").getMillis()),
@@ -3473,7 +3473,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
StringComparators.NUMERIC
)
)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -3491,9 +3491,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new LongSumAggregatorFactory(
"a0",
null,
@@ -3518,9 +3518,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new LongSumAggregatorFactory(
"a0",
null,
@@ -3554,13 +3554,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new GroupByQuery.Builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN("d0:v", "timestamp_floor(\"cnt\",'P1Y',null,'UTC')", ValueType.LONG)
+ expression_Virtual_Column("d0:v", "timestamp_floor(\"cnt\",'P1Y',null,'UTC')", ValueType.LONG)
)
.setDimFilter(
- BOUND(
+ bound(
"cnt",
String.valueOf(DateTimes.of("1970-01-01").getMillis()),
String.valueOf(DateTimes.of("1970-01-02").getMillis()),
@@ -3570,13 +3570,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
StringComparators.NUMERIC
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{T("1970-01-01"), 6L}
+ new Object[]{t("1970-01-01"), 6L}
)
);
}
@@ -3589,20 +3589,20 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setDimFilter(
- OR(
- SELECTOR(
+ or(
+ selector(
"dim1",
"e",
- CASCADE(
+ cascade(
new SubstringDimExtractionFn(1, null),
new SubstringDimExtractionFn(0, 1)
)
),
- SELECTOR("dim2", "a", null)
+ selector("dim2", "a", null)
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
@@ -3625,13 +3625,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setDimFilter(
- OR(
- EXPRESSION_FILTER("(strlen(\"dim1\") == 3)"),
- EXPRESSION_FILTER("(CAST(strlen(\"dim1\"), 'STRING') == 3)")
+ or(
+ expressionFilter("(strlen(\"dim1\") == 3)"),
+ expressionFilter("(CAST(strlen(\"dim1\"), 'STRING') == 3)")
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
@@ -3654,7 +3654,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim2", "d0"))
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
@@ -3685,7 +3685,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim2", "d0"))
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
@@ -3716,7 +3716,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim2", "d0"))
.metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))
@@ -3759,7 +3759,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim2", "d0"))
.metric(new InvertedTopNMetricSpec(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)))
@@ -3790,15 +3790,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(
- AGGS(
+ aggregators(
new LongSumAggregatorFactory("a0", "cnt"),
new CardinalityAggregatorFactory(
"a1",
null,
- DIMS(new DefaultDimensionSpec("dim2", null)),
+ dimensionSpec(new DefaultDimensionSpec("dim2", null)),
false,
true
),
@@ -3826,10 +3826,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(
- AGGS(
+ aggregators(
new FilteredAggregatorFactory(
new CardinalityAggregatorFactory(
"a0",
@@ -3838,7 +3838,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
false,
true
),
- BOUND("m1", "4", null, false, false, null, StringComparators.NUMERIC)
+ bound("m1", "4", null, false, false, null, StringComparators.NUMERIC)
),
new FilteredAggregatorFactory(
new CardinalityAggregatorFactory(
@@ -3848,11 +3848,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
false,
true
),
- BOUND("m1", "4", null, false, false, null, StringComparators.NUMERIC)
+ bound("m1", "4", null, false, false, null, StringComparators.NUMERIC)
),
new FilteredAggregatorFactory(
new HyperUniquesAggregatorFactory("a2", "unique_dim1", false, true),
- BOUND("m1", "4", null, false, false, null, StringComparators.NUMERIC)
+ bound("m1", "4", null, false, false, null, StringComparators.NUMERIC)
)
)
)
@@ -3880,19 +3880,19 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new FilteredAggregatorFactory(
new CountAggregatorFactory("a0"),
- NOT(SELECTOR("d0", null, null))
+ not(selector("d0", null, null))
)
))
.setContext(QUERY_CONTEXT_DEFAULT)
@@ -3916,14 +3916,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(
- AGGS(
+ aggregators(
new CardinalityAggregatorFactory(
"a0",
null,
- DIMS(new DefaultDimensionSpec("dim2", null)),
+ dimensionSpec(new DefaultDimensionSpec("dim2", null)),
false,
true
)
@@ -3953,25 +3953,25 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(
+ .setDimensions(dimensionSpec(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("dim1", "d1")
))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0", "_d0")))
- .setAggregatorSpecs(AGGS(
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0", "_d0")))
+ .setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("_a0", "a0"),
new FilteredAggregatorFactory(
new CountAggregatorFactory("_a1"),
- NOT(SELECTOR("d1", null, null))
+ not(selector("d1", null, null))
)
))
.setContext(QUERY_CONTEXT_DEFAULT)
@@ -4007,18 +4007,18 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.virtualColumns(
- EXPRESSION_VIRTUAL_COLUMN("a4:v", "concat(substring(\"dim2\", 0, 1),'x')", ValueType.STRING)
+ expression_Virtual_Column("a4:v", "concat(substring(\"dim2\", 0, 1),'x')", ValueType.STRING)
)
.aggregators(
- AGGS(
+ aggregators(
new LongSumAggregatorFactory("a0", "cnt"),
new CardinalityAggregatorFactory(
"a1",
null,
- DIMS(new DefaultDimensionSpec("dim2", "dim2")),
+ dimensionSpec(new DefaultDimensionSpec("dim2", "dim2")),
false,
true
),
@@ -4026,16 +4026,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new CardinalityAggregatorFactory(
"a2",
null,
- DIMS(new DefaultDimensionSpec("dim2", "dim2")),
+ dimensionSpec(new DefaultDimensionSpec("dim2", "dim2")),
false,
true
),
- NOT(SELECTOR("dim2", "", null))
+ not(selector("dim2", "", null))
),
new CardinalityAggregatorFactory(
"a3",
null,
- DIMS(
+ dimensionSpec(
new ExtractionDimensionSpec(
"dim2",
"dim2",
@@ -4049,7 +4049,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new CardinalityAggregatorFactory(
"a4",
null,
- DIMS(new DefaultDimensionSpec("a4:v", "a4:v", ValueType.STRING)),
+ dimensionSpec(new DefaultDimensionSpec("a4:v", "a4:v", ValueType.STRING)),
false,
true
),
@@ -4093,31 +4093,31 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(
+ .setDimensions(dimensionSpec(
new DefaultDimensionSpec("m2", "d0", ValueType.DOUBLE),
new DefaultDimensionSpec("dim1", "d1")
))
.setDimFilter(new SelectorDimFilter("m1", "5.0", null))
- .setAggregatorSpecs(AGGS(new LongMaxAggregatorFactory("a0", "__time")))
+ .setAggregatorSpecs(aggregators(new LongMaxAggregatorFactory("a0", "__time")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"_d0:v",
"timestamp_floor(\"a0\",'PT1H',null,'UTC')",
ValueType.LONG
)
)
- .setDimensions(DIMS(
+ .setDimensions(dimensionSpec(
new DefaultDimensionSpec("_d0:v", "_d0", ValueType.LONG),
new DefaultDimensionSpec("d1", "_d1", ValueType.STRING)
))
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new CountAggregatorFactory("_a0")
))
.setContext(QUERY_CONTEXT_DEFAULT)
@@ -4151,26 +4151,26 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(
+ .setDimensions(dimensionSpec(
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("dim2", "d1")
))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("d1", "_d0")))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("_a0", "a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d1", "_d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("_a0", "a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("a0", "_a0"),
new CountAggregatorFactory("a1")
))
@@ -4230,17 +4230,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("_a0", "a0"),
new CountAggregatorFactory("_a1")
))
@@ -4274,24 +4274,24 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"timestamp_floor(\"__time\",'P1D',null,'UTC')",
ValueType.LONG
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new LongMaxAggregatorFactory("_a0", "a0"),
new LongMinAggregatorFactory("_a1", "a0"),
new LongSumAggregatorFactory("_a2:sum", "a0"),
@@ -4309,7 +4309,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new FieldAccessPostAggregator(null, "_a2:count")
)
),
- EXPRESSION_POST_AGG("s0", "timestamp_extract(\"_a3\",'EPOCH','UTC')")
+ expresionPostAgg("s0", "timestamp_extract(\"_a3\",'EPOCH','UTC')")
)
)
.setLimit(1)
@@ -4333,22 +4333,22 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"timestamp_floor(\"__time\",'P1D',null,'UTC')",
ValueType.LONG
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
.setAggregatorSpecs(
- AGGS(
+ aggregators(
new CardinalityAggregatorFactory(
"a0:a",
null,
- DIMS(new DefaultDimensionSpec("cnt", "cnt", ValueType.LONG)),
+ dimensionSpec(new DefaultDimensionSpec("cnt", "cnt", ValueType.LONG)),
false,
true
)
@@ -4363,9 +4363,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("_a0:sum", "a0"),
new CountAggregatorFactory("_a0:count")
))
@@ -4392,8 +4392,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
public void testTopNFilterJoin() throws Exception
{
DimFilter filter = NullHandling.replaceWithDefault() ?
- IN("dim2", Arrays.asList(null, "a"), null)
- : SELECTOR("dim2", "a", null);
+ in("dim2", Arrays.asList(null, "a"), null)
+ : selector("dim2", "a", null);
// Filters on top N values of some dimension by using an inner join.
testQuery(
"SELECT t1.dim1, SUM(t1.cnt)\n"
@@ -4412,21 +4412,21 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim2", "d0"))
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.metric(new NumericTopNMetricSpec("a0"))
.threshold(2)
.context(QUERY_CONTEXT_DEFAULT)
.build(),
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(filter)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -4479,21 +4479,21 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim2", "d0"))
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.metric(new NumericTopNMetricSpec("a0"))
.threshold(2)
.context(QUERY_CONTEXT_DEFAULT)
.build(),
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimFilter(IN("dim2", ImmutableList.of("", "a"), null))
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimFilter(in("dim2", ImmutableList.of("", "a"), null))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -4541,10 +4541,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -4586,10 +4586,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimFilter(NOT(SELECTOR("dim1", "", null)))
- .setDimensions(DIMS(new ExtractionDimensionSpec(
+ .setDimFilter(not(selector("dim1", "", null)))
+ .setDimensions(dimensionSpec(new ExtractionDimensionSpec(
"dim1",
"d0",
new SubstringDimExtractionFn(0, 1)
@@ -4601,21 +4601,21 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Intervals.of("2000-01-01/2002-01-01")))
+ .setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01")))
.setGranularity(Granularities.ALL)
- .setDimFilter(IN(
+ .setDimFilter(in(
"dim2",
ImmutableList.of("1", "2", "a", "d"),
new SubstringDimExtractionFn(0, 1)
))
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new CountAggregatorFactory("a0")
))
.setContext(QUERY_CONTEXT_DEFAULT)
@@ -4682,18 +4682,18 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
- .setDimFilter(NOT(SELECTOR("dim2", "", null)))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
+ .setDimFilter(not(selector("dim2", "", null)))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("_a0", "a0"),
new CountAggregatorFactory("_a1")
))
@@ -4721,18 +4721,18 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
- .setDimFilter(NOT(SELECTOR("dim2", null, null)))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
+ .setDimFilter(not(selector("dim2", null, null)))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("_a0", "a0"),
new CountAggregatorFactory("_a1")
))
@@ -4764,19 +4764,19 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimit(1)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setDimFilter(BOUND("a0", "0", null, true, false, null, StringComparators.NUMERIC))
- .setInterval(QSS(Filtration.eternity()))
+ .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("_a0", "a0"),
new CountAggregatorFactory("_a1")
))
@@ -4808,29 +4808,29 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimFilter(NOT(SELECTOR("dim1", "", null)))
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setDimFilter(not(selector("dim1", "", null)))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new CountAggregatorFactory("a0"),
new CardinalityAggregatorFactory(
"a1",
null,
- DIMS(new DefaultDimensionSpec("d0", null)),
+ dimensionSpec(new DefaultDimensionSpec("d0", null)),
false,
true
)
))
.setPostAggregatorSpecs(
ImmutableList.of(
- EXPRESSION_POST_AGG("p0", "((1 - (\"a1\" / \"a0\")) * 100)")
+ expresionPostAgg("p0", "((1 - (\"a1\" / \"a0\")) * 100)")
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
@@ -4857,18 +4857,18 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("a0", "_d0")))
- .setAggregatorSpecs(AGGS(
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("a0", "_d0")))
+ .setAggregatorSpecs(aggregators(
new CountAggregatorFactory("_a0")
))
.setContext(QUERY_CONTEXT_DEFAULT)
@@ -4902,18 +4902,18 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("a0", "_d0")))
- .setAggregatorSpecs(AGGS(
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("a0", "_d0")))
+ .setAggregatorSpecs(aggregators(
new CountAggregatorFactory("_a0")
))
.setLimitSpec(
@@ -4956,25 +4956,25 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(
- AGGS(
+ aggregators(
new LongSumAggregatorFactory("a0", "cnt"),
new CardinalityAggregatorFactory(
"a1",
null,
- DIMS(new DefaultDimensionSpec("dim2", null)),
+ dimensionSpec(new DefaultDimensionSpec("dim2", null)),
false,
true
)
)
)
.postAggregators(
- EXPRESSION_POST_AGG("p0", "CAST(\"a1\", 'DOUBLE')"),
- EXPRESSION_POST_AGG("p1", "(\"a0\" / \"a1\")"),
- EXPRESSION_POST_AGG("p2", "((\"a0\" / \"a1\") + 3)"),
- EXPRESSION_POST_AGG("p3", "((CAST(\"a0\", 'DOUBLE') / CAST(\"a1\", 'DOUBLE')) + 3)")
+ expresionPostAgg("p0", "CAST(\"a1\", 'DOUBLE')"),
+ expresionPostAgg("p1", "(\"a0\" / \"a1\")"),
+ expresionPostAgg("p2", "((\"a0\" / \"a1\") + 3)"),
+ expresionPostAgg("p3", "((CAST(\"a0\", 'DOUBLE') / CAST(\"a1\", 'DOUBLE')) + 3)")
)
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
@@ -4993,15 +4993,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .filters(NOT(SELECTOR("dim1", "", null)))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .filters(not(selector("dim1", "", null)))
.granularity(Granularities.ALL)
.aggregators(
- AGGS(
+ aggregators(
new CardinalityAggregatorFactory(
"a0",
null,
- DIMS(
+ dimensionSpec(
new ExtractionDimensionSpec(
"dim1",
null,
@@ -5032,17 +5032,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .filters(NOT(SELECTOR("dim1", "", null)))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .filters(not(selector("dim1", "", null)))
.granularity(Granularities.ALL)
- .virtualColumns(EXPRESSION_VIRTUAL_COLUMN("a0:v", "trim(\"dim1\",' ')", ValueType.STRING))
- .filters(EXPRESSION_FILTER("(trim(\"dim1\",' ') != '')"))
+ .virtualColumns(expression_Virtual_Column("a0:v", "trim(\"dim1\",' ')", ValueType.STRING))
+ .filters(expressionFilter("(trim(\"dim1\",' ') != '')"))
.aggregators(
- AGGS(
+ aggregators(
new CardinalityAggregatorFactory(
"a0",
null,
- DIMS(new DefaultDimensionSpec("a0:v", "a0:v", ValueType.STRING)),
+ dimensionSpec(new DefaultDimensionSpec("a0:v", "a0:v", ValueType.STRING)),
false,
true
)
@@ -5069,15 +5069,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setVirtualColumns(EXPRESSION_VIRTUAL_COLUMN(
+ .setVirtualColumns(expression_Virtual_Column(
"d0:v",
"(((timestamp_extract(\"__time\",'MONTH','UTC') - 1) / 3) + 1)",
ValueType.LONG
))
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -5100,17 +5100,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(
- NOT(SELECTOR(
+ not(selector(
"dim1",
"x",
new RegexDimExtractionFn("^(.)", 1, true, null)
))
)
.setDimensions(
- DIMS(
+ dimensionSpec(
new ExtractionDimensionSpec(
"dim1",
"d0",
@@ -5145,16 +5145,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
- DIMS(
+ dimensionSpec(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("dim1", "d1")
)
)
.setAggregatorSpecs(
- AGGS(
+ aggregators(
new LongSumAggregatorFactory("a0", "cnt")
)
)
@@ -5192,16 +5192,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
- DIMS(
+ dimensionSpec(
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("dim2", "d1")
)
)
.setAggregatorSpecs(
- AGGS(
+ aggregators(
new LongSumAggregatorFactory("a0", "cnt")
)
)
@@ -5213,7 +5213,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
4
)
)
- .setHavingSpec(HAVING(NUMERIC_SELECTOR("a0", "1", null)))
+ .setHavingSpec(having(numeric_Selector("a0", "1", null)))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -5244,9 +5244,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000/P2M")))
+ .intervals(querySegmentSpec(Intervals.of("2000/P2M")))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -5267,9 +5267,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000-01-01T01:02/2002")))
+ .intervals(querySegmentSpec(Intervals.of("2000-01-01T01:02/2002")))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -5289,7 +5289,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(),
ImmutableList.of(
- new Object[]{T("2000-01-01T00Z", LOS_ANGELES), D("1999-12-31"), D("2000-01-01")}
+ new Object[]{t("2000-01-01T00Z", LOS_ANGELES), d("1999-12-31"), d("2000-01-01")}
)
);
}
@@ -5306,9 +5306,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000-01-02T00Z/2002-01-01T08Z")))
+ .intervals(querySegmentSpec(Intervals.of("2000-01-02T00Z/2002-01-01T08Z")))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_LOS_ANGELES)
.build()
),
@@ -5326,9 +5326,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000-01-02/2002")))
+ .intervals(querySegmentSpec(Intervals.of("2000-01-02/2002")))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -5352,9 +5352,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000-01-02T00Z/2002-01-01T08Z")))
+ .intervals(querySegmentSpec(Intervals.of("2000-01-02T00Z/2002-01-01T08Z")))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_LOS_ANGELES)
.build()
),
@@ -5374,12 +5374,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(
+ .intervals(querySegmentSpec(
new Interval(DateTimes.MIN, DateTimes.of("2001-01-01")),
new Interval(DateTimes.of("2001-02-01"), DateTimes.MAX)
))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -5399,9 +5399,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(new Interval(DateTimes.MIN, DateTimes.of("2000-02-01"))))
+ .intervals(querySegmentSpec(new Interval(DateTimes.MIN, DateTimes.of("2000-02-01"))))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -5421,9 +5421,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(new Interval(DateTimes.MIN, DateTimes.of("2000-03-01"))))
+ .intervals(querySegmentSpec(new Interval(DateTimes.MIN, DateTimes.of("2000-03-01"))))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -5443,13 +5443,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.filters(
- AND(
- EXPRESSION_FILTER("(timestamp_extract(\"__time\",'YEAR','UTC') == 2000)"),
- EXPRESSION_FILTER("(timestamp_extract(\"__time\",'MONTH','UTC') == 1)")
+ and(
+ expressionFilter("(timestamp_extract(\"__time\",'YEAR','UTC') == 2000)"),
+ expressionFilter("(timestamp_extract(\"__time\",'MONTH','UTC') == 1)")
)
)
.context(TIMESERIES_CONTEXT_DEFAULT)
@@ -5471,16 +5471,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.filters(
- AND(
- EXPRESSION_FILTER("(timestamp_extract(\"__time\",'YEAR','UTC') == 2000)"),
- OR(
- EXPRESSION_FILTER("(timestamp_extract(\"__time\",'DAY','UTC') == 2)"),
- EXPRESSION_FILTER("(timestamp_extract(\"__time\",'DAY','UTC') == 3)"),
- EXPRESSION_FILTER("(timestamp_extract(\"__time\",'DAY','UTC') == 5)")
+ and(
+ expressionFilter("(timestamp_extract(\"__time\",'YEAR','UTC') == 2000)"),
+ or(
+ expressionFilter("(timestamp_extract(\"__time\",'DAY','UTC') == 2)"),
+ expressionFilter("(timestamp_extract(\"__time\",'DAY','UTC') == 3)"),
+ expressionFilter("(timestamp_extract(\"__time\",'DAY','UTC') == 5)")
)
)
)
@@ -5502,9 +5502,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS())
+ .intervals(querySegmentSpec())
.granularity(Granularities.ALL)
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -5522,13 +5522,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN("d0:v", "floor(CAST(\"dim1\", 'DOUBLE'))", ValueType.FLOAT)
+ expression_Virtual_Column("d0:v", "floor(CAST(\"dim1\", 'DOUBLE'))", ValueType.FLOAT)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.FLOAT)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.FLOAT)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -5549,17 +5549,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"floor(CAST(\"dim1\", 'DOUBLE'))",
ValueType.FLOAT
)
)
.setDimensions(
- DIMS(
+ dimensionSpec(
new DefaultDimensionSpec(
"d0:v",
"d0",
@@ -5567,7 +5567,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
)
)
)
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -5603,23 +5603,23 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"timestamp_floor(\"__time\",'P1Y',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
- DIMS(
+ dimensionSpec(
new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG),
new DefaultDimensionSpec("dim2", "d1")
)
)
.setAggregatorSpecs(
- AGGS(
+ aggregators(
new CountAggregatorFactory("a0")
)
)
@@ -5650,19 +5650,19 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
),
NullHandling.replaceWithDefault() ?
ImmutableList.of(
- new Object[]{T("2000"), "", 2L},
- new Object[]{T("2000"), "a", 1L},
- new Object[]{T("2001"), "", 1L},
- new Object[]{T("2001"), "a", 1L},
- new Object[]{T("2001"), "abc", 1L}
+ new Object[]{t("2000"), "", 2L},
+ new Object[]{t("2000"), "a", 1L},
+ new Object[]{t("2001"), "", 1L},
+ new Object[]{t("2001"), "a", 1L},
+ new Object[]{t("2001"), "abc", 1L}
) :
ImmutableList.of(
- new Object[]{T("2000"), null, 1L},
- new Object[]{T("2000"), "", 1L},
- new Object[]{T("2000"), "a", 1L},
- new Object[]{T("2001"), null, 1L},
- new Object[]{T("2001"), "a", 1L},
- new Object[]{T("2001"), "abc", 1L}
+ new Object[]{t("2000"), null, 1L},
+ new Object[]{t("2000"), "", 1L},
+ new Object[]{t("2000"), "a", 1L},
+ new Object[]{t("2001"), null, 1L},
+ new Object[]{t("2001"), "a", 1L},
+ new Object[]{t("2001"), "abc", 1L}
)
);
}
@@ -5675,11 +5675,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setVirtualColumns(EXPRESSION_VIRTUAL_COLUMN("d0:v", "strlen(\"dim1\")", ValueType.LONG))
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setVirtualColumns(expression_Virtual_Column("d0:v", "strlen(\"dim1\")", ValueType.LONG))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -5712,17 +5712,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(
- NOT(SELECTOR(
+ not(selector(
"dim1",
"xxx",
extractionFn
))
)
.setDimensions(
- DIMS(
+ dimensionSpec(
new ExtractionDimensionSpec(
"dim1",
"d0",
@@ -5732,7 +5732,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
)
)
.setAggregatorSpecs(
- AGGS(
+ aggregators(
new CountAggregatorFactory("a0")
)
)
@@ -5763,9 +5763,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new CardinalityAggregatorFactory(
"a0",
null,
@@ -5796,15 +5796,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.MONTH)
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{3L, T("2000-01-01")},
- new Object[]{3L, T("2001-01-01")}
+ new Object[]{3L, t("2000-01-01")},
+ new Object[]{3L, t("2001-01-01")}
)
);
}
@@ -5825,15 +5825,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000-01-01/2001-02-01")))
+ .intervals(querySegmentSpec(Intervals.of("2000-01-01/2001-02-01")))
.granularity(Granularities.ALL)
- .aggregators(AGGS(
+ .aggregators(aggregators(
new FilteredAggregatorFactory(
new LongSumAggregatorFactory("a0", "cnt"),
- BOUND(
+ bound(
"__time",
- String.valueOf(T("2000-01-01")),
- String.valueOf(T("2000-02-01")),
+ String.valueOf(t("2000-01-01")),
+ String.valueOf(t("2000-02-01")),
false,
true,
null,
@@ -5842,10 +5842,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
),
new FilteredAggregatorFactory(
new LongSumAggregatorFactory("a1", "cnt"),
- BOUND(
+ bound(
"__time",
- String.valueOf(T("2001-01-01")),
- String.valueOf(T("2001-02-01")),
+ String.valueOf(t("2001-01-01")),
+ String.valueOf(t("2001-02-01")),
false,
true,
null,
@@ -5878,17 +5878,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES)))
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_LOS_ANGELES)
.build()
),
ImmutableList.of(
- new Object[]{1L, T("1999-12-01", LOS_ANGELES)},
- new Object[]{2L, T("2000-01-01", LOS_ANGELES)},
- new Object[]{1L, T("2000-12-01", LOS_ANGELES)},
- new Object[]{2L, T("2001-01-01", LOS_ANGELES)}
+ new Object[]{1L, t("1999-12-01", LOS_ANGELES)},
+ new Object[]{2L, t("2000-01-01", LOS_ANGELES)},
+ new Object[]{1L, t("2000-12-01", LOS_ANGELES)},
+ new Object[]{2L, t("2001-01-01", LOS_ANGELES)}
)
);
}
@@ -5909,17 +5909,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES)))
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{1L, T("1999-12-01", LOS_ANGELES)},
- new Object[]{2L, T("2000-01-01", LOS_ANGELES)},
- new Object[]{1L, T("2000-12-01", LOS_ANGELES)},
- new Object[]{2L, T("2001-01-01", LOS_ANGELES)}
+ new Object[]{1L, t("1999-12-01", LOS_ANGELES)},
+ new Object[]{2L, t("2000-01-01", LOS_ANGELES)},
+ new Object[]{1L, t("2000-12-01", LOS_ANGELES)},
+ new Object[]{2L, t("2001-01-01", LOS_ANGELES)}
)
);
}
@@ -5937,15 +5937,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.MONTH)
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{3L, T("2000-01-01")},
- new Object[]{3L, T("2001-01-01")}
+ new Object[]{3L, t("2000-01-01")},
+ new Object[]{3L, t("2001-01-01")}
)
);
}
@@ -5963,17 +5963,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"timestamp_floor(timestamp_shift(\"__time\",'P1D',-1),'P1M',null,'UTC')",
ValueType.LONG
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -5990,10 +5990,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build()
),
ImmutableList.of(
- new Object[]{1L, T("1999-12-01")},
- new Object[]{2L, T("2000-01-01")},
- new Object[]{1L, T("2000-12-01")},
- new Object[]{2L, T("2001-01-01")}
+ new Object[]{1L, t("1999-12-01")},
+ new Object[]{2L, t("2000-01-01")},
+ new Object[]{1L, t("2000-12-01")},
+ new Object[]{2L, t("2001-01-01")}
)
);
}
@@ -6011,17 +6011,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"timestamp_floor((\"__time\" + -86400000),'P1M',null,'UTC')",
ValueType.LONG
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -6038,10 +6038,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build()
),
ImmutableList.of(
- new Object[]{1L, T("1999-12-01")},
- new Object[]{2L, T("2000-01-01")},
- new Object[]{1L, T("2000-12-01")},
- new Object[]{2L, T("2001-01-01")}
+ new Object[]{1L, t("1999-12-01")},
+ new Object[]{2L, t("2000-01-01")},
+ new Object[]{1L, t("2000-12-01")},
+ new Object[]{2L, t("2001-01-01")}
)
);
}
@@ -6059,7 +6059,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(
new PeriodGranularity(
Period.months(1),
@@ -6067,15 +6067,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
DateTimeZone.UTC
)
)
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{1L, T("1999-12-01T01:02:03")},
- new Object[]{2L, T("2000-01-01T01:02:03")},
- new Object[]{1L, T("2000-12-01T01:02:03")},
- new Object[]{2L, T("2001-01-01T01:02:03")}
+ new Object[]{1L, t("1999-12-01T01:02:03")},
+ new Object[]{2L, t("2000-01-01T01:02:03")},
+ new Object[]{1L, t("2000-12-01T01:02:03")},
+ new Object[]{2L, t("2001-01-01T01:02:03")}
)
);
}
@@ -6093,17 +6093,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES)))
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{1L, T("1999-12-01T08")},
- new Object[]{2L, T("2000-01-01T08")},
- new Object[]{1L, T("2000-12-01T08")},
- new Object[]{2L, T("2001-01-01T08")}
+ new Object[]{1L, t("1999-12-01T08")},
+ new Object[]{2L, t("2000-01-01T08")},
+ new Object[]{1L, t("2000-12-01T08")},
+ new Object[]{2L, t("2001-01-01T08")}
)
);
}
@@ -6124,17 +6124,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES)))
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_LOS_ANGELES)
.build()
),
ImmutableList.of(
- new Object[]{1L, T("1999-12-01", LOS_ANGELES)},
- new Object[]{2L, T("2000-01-01", LOS_ANGELES)},
- new Object[]{1L, T("2000-12-01", LOS_ANGELES)},
- new Object[]{2L, T("2001-01-01", LOS_ANGELES)}
+ new Object[]{1L, t("1999-12-01", LOS_ANGELES)},
+ new Object[]{2L, t("2000-01-01", LOS_ANGELES)},
+ new Object[]{1L, t("2000-12-01", LOS_ANGELES)},
+ new Object[]{2L, t("2001-01-01", LOS_ANGELES)}
)
);
}
@@ -6157,37 +6157,37 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000/2000-01-02")))
+ .intervals(querySegmentSpec(Intervals.of("2000/2000-01-02")))
.granularity(new PeriodGranularity(Period.hours(1), null, DateTimeZone.UTC))
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS)
.build()
),
ImmutableList.<Object[]>builder()
- .add(new Object[]{1L, T("2000-01-01")})
- .add(new Object[]{defaultVal, T("2000-01-01T01")})
- .add(new Object[]{defaultVal, T("2000-01-01T02")})
- .add(new Object[]{defaultVal, T("2000-01-01T03")})
- .add(new Object[]{defaultVal, T("2000-01-01T04")})
- .add(new Object[]{defaultVal, T("2000-01-01T05")})
- .add(new Object[]{defaultVal, T("2000-01-01T06")})
- .add(new Object[]{defaultVal, T("2000-01-01T07")})
- .add(new Object[]{defaultVal, T("2000-01-01T08")})
- .add(new Object[]{defaultVal, T("2000-01-01T09")})
- .add(new Object[]{defaultVal, T("2000-01-01T10")})
- .add(new Object[]{defaultVal, T("2000-01-01T11")})
- .add(new Object[]{defaultVal, T("2000-01-01T12")})
- .add(new Object[]{defaultVal, T("2000-01-01T13")})
- .add(new Object[]{defaultVal, T("2000-01-01T14")})
- .add(new Object[]{defaultVal, T("2000-01-01T15")})
- .add(new Object[]{defaultVal, T("2000-01-01T16")})
- .add(new Object[]{defaultVal, T("2000-01-01T17")})
- .add(new Object[]{defaultVal, T("2000-01-01T18")})
- .add(new Object[]{defaultVal, T("2000-01-01T19")})
- .add(new Object[]{defaultVal, T("2000-01-01T20")})
- .add(new Object[]{defaultVal, T("2000-01-01T21")})
- .add(new Object[]{defaultVal, T("2000-01-01T22")})
- .add(new Object[]{defaultVal, T("2000-01-01T23")})
+ .add(new Object[]{1L, t("2000-01-01")})
+ .add(new Object[]{defaultVal, t("2000-01-01T01")})
+ .add(new Object[]{defaultVal, t("2000-01-01T02")})
+ .add(new Object[]{defaultVal, t("2000-01-01T03")})
+ .add(new Object[]{defaultVal, t("2000-01-01T04")})
+ .add(new Object[]{defaultVal, t("2000-01-01T05")})
+ .add(new Object[]{defaultVal, t("2000-01-01T06")})
+ .add(new Object[]{defaultVal, t("2000-01-01T07")})
+ .add(new Object[]{defaultVal, t("2000-01-01T08")})
+ .add(new Object[]{defaultVal, t("2000-01-01T09")})
+ .add(new Object[]{defaultVal, t("2000-01-01T10")})
+ .add(new Object[]{defaultVal, t("2000-01-01T11")})
+ .add(new Object[]{defaultVal, t("2000-01-01T12")})
+ .add(new Object[]{defaultVal, t("2000-01-01T13")})
+ .add(new Object[]{defaultVal, t("2000-01-01T14")})
+ .add(new Object[]{defaultVal, t("2000-01-01T15")})
+ .add(new Object[]{defaultVal, t("2000-01-01T16")})
+ .add(new Object[]{defaultVal, t("2000-01-01T17")})
+ .add(new Object[]{defaultVal, t("2000-01-01T18")})
+ .add(new Object[]{defaultVal, t("2000-01-01T19")})
+ .add(new Object[]{defaultVal, t("2000-01-01T20")})
+ .add(new Object[]{defaultVal, t("2000-01-01T21")})
+ .add(new Object[]{defaultVal, t("2000-01-01T22")})
+ .add(new Object[]{defaultVal, t("2000-01-01T23")})
.build()
);
}
@@ -6205,19 +6205,19 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC))
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{1L, D("2000-01-01")},
- new Object[]{1L, D("2000-01-02")},
- new Object[]{1L, D("2000-01-03")},
- new Object[]{1L, D("2001-01-01")},
- new Object[]{1L, D("2001-01-02")},
- new Object[]{1L, D("2001-01-03")}
+ new Object[]{1L, d("2000-01-01")},
+ new Object[]{1L, d("2000-01-02")},
+ new Object[]{1L, d("2000-01-03")},
+ new Object[]{1L, d("2001-01-01")},
+ new Object[]{1L, d("2001-01-02")},
+ new Object[]{1L, d("2001-01-03")}
)
);
}
@@ -6227,7 +6227,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
{
testQuery(
"SELECT SUM(cnt), dt FROM (\n"
- + " SELECT CAST(FLOOR(__time TO QUARTER) AS DATE) AS dt,\n"
+ + " SELECT CASt(FLOOR(__time TO QUARTER) AS DATE) AS dt,\n"
+ " cnt FROM druid.foo\n"
+ ") AS x\n"
+ "GROUP BY dt\n"
@@ -6235,15 +6235,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(new PeriodGranularity(Period.months(3), null, DateTimeZone.UTC))
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{3L, D("2000-01-01")},
- new Object[]{3L, D("2001-01-01")}
+ new Object[]{3L, d("2000-01-01")},
+ new Object[]{3L, d("2001-01-01")}
)
);
}
@@ -6261,16 +6261,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.MONTH)
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.descending(true)
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{T("2001-01-01"), 3L},
- new Object[]{T("2000-01-01"), 3L}
+ new Object[]{t("2001-01-01"), 3L},
+ new Object[]{t("2000-01-01"), 3L}
)
);
}
@@ -6288,17 +6288,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"timestamp_extract(\"__time\",'YEAR','UTC')",
ValueType.LONG
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -6334,17 +6334,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"timestamp_format(\"__time\",'yyyy MM','UTC')",
ValueType.STRING
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.STRING)))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.STRING)))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -6378,17 +6378,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"timestamp_extract(timestamp_floor(\"__time\",'P1Y',null,'UTC'),'YEAR','UTC')",
ValueType.LONG
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -6413,17 +6413,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d0:v",
"timestamp_extract(timestamp_floor(\"__time\",'P1Y',null,'America/Los_Angeles'),'YEAR','America/Los_Angeles')",
ValueType.LONG
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_LOS_ANGELES)
.build()
),
@@ -6452,15 +6452,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.MONTH)
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.limit(1)
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{T("2000-01-01"), 3L}
+ new Object[]{t("2000-01-01"), 3L}
)
);
}
@@ -6479,15 +6479,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.MONTH)
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.limit(1)
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{T("2000-01-01"), 3L}
+ new Object[]{t("2000-01-01"), 3L}
)
);
}
@@ -6507,15 +6507,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.MONTH)
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.limit(1)
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{T("2000-01-01"), 3L}
+ new Object[]{t("2000-01-01"), 3L}
)
);
}
@@ -6531,22 +6531,22 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN(
+ expression_Virtual_Column(
"d1:v",
"timestamp_floor(\"__time\",'P1M',null,'UTC')",
ValueType.LONG
)
)
.setDimensions(
- DIMS(
+ dimensionSpec(
new DefaultDimensionSpec("dim2", "d0"),
new DefaultDimensionSpec("d1:v", "d1", ValueType.LONG)
)
)
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -6565,19 +6565,19 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
),
NullHandling.replaceWithDefault() ?
ImmutableList.of(
- new Object[]{"", T("2000-01-01"), 2L},
- new Object[]{"", T("2001-01-01"), 1L},
- new Object[]{"a", T("2000-01-01"), 1L},
- new Object[]{"a", T("2001-01-01"), 1L},
- new Object[]{"abc", T("2001-01-01"), 1L}
+ new Object[]{"", t("2000-01-01"), 2L},
+ new Object[]{"", t("2001-01-01"), 1L},
+ new Object[]{"a", t("2000-01-01"), 1L},
+ new Object[]{"a", t("2001-01-01"), 1L},
+ new Object[]{"abc", t("2001-01-01"), 1L}
) :
ImmutableList.of(
- new Object[]{null, T("2000-01-01"), 1L},
- new Object[]{null, T("2001-01-01"), 1L},
- new Object[]{"", T("2000-01-01"), 1L},
- new Object[]{"a", T("2000-01-01"), 1L},
- new Object[]{"a", T("2001-01-01"), 1L},
- new Object[]{"abc", T("2001-01-01"), 1L}
+ new Object[]{null, t("2000-01-01"), 1L},
+ new Object[]{null, t("2001-01-01"), 1L},
+ new Object[]{"", t("2000-01-01"), 1L},
+ new Object[]{"a", t("2000-01-01"), 1L},
+ new Object[]{"a", t("2001-01-01"), 1L},
+ new Object[]{"abc", t("2001-01-01"), 1L}
)
);
}
@@ -6595,29 +6595,29 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimFilter(NOT(SELECTOR("dim1", "", null)))
- .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0")))
+ .setDimFilter(not(selector("dim1", "", null)))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim1", "d0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build(),
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimFilter(
- AND(
- NOT(SELECTOR("dim1", "xxx", null)),
- IN("dim2", ImmutableList.of("1", "10.1", "2", "abc", "def"), null)
+ and(
+ not(selector("dim1", "xxx", null)),
+ in("dim2", ImmutableList.of("1", "10.1", "2", "abc", "def"), null)
)
)
.setDimensions(
- DIMS(
+ dimensionSpec(
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("dim2", "d1")
)
)
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(new OrderByColumnSpec("d1", OrderByColumnSpec.Direction.ASCENDING)),
@@ -6693,25 +6693,25 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimFilter(SELECTOR("dim2", "abc", null))
- .setDimensions(DIMS(
+ .setDimFilter(selector("dim2", "abc", null))
+ .setDimensions(dimensionSpec(
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("dim2", "d1")
))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
- .setHavingSpec(HAVING(NUMERIC_SELECTOR("a0", "1", null)))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
+ .setHavingSpec(having(numeric_Selector("a0", "1", null)))
.setContext(QUERY_CONTEXT_DEFAULT)
.build(),
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .filters(OR(
- SELECTOR("dim1", "def", null),
- AND(
- SELECTOR("dim1", "def", null),
- SELECTOR("dim2", "abc", null)
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .filters(or(
+ selector("dim1", "def", null),
+ and(
+ selector("dim1", "def", null),
+ selector("dim2", "abc", null)
)
))
.columns("__time", "cnt", "dim1", "dim2")
@@ -6720,7 +6720,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build()
),
ImmutableList.of(
- new Object[]{T("2001-01-02"), 1L, "def", "abc"}
+ new Object[]{t("2001-01-02"), 1L, "def", "abc"}
)
);
}
@@ -6742,9 +6742,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
.setLimitSpec(
new DefaultLimitSpec(
ImmutableList.of(
@@ -6761,8 +6761,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build(),
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .filters(IN("dim2", ImmutableList.of("", "a", "abc"), null))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .filters(in("dim2", ImmutableList.of("", "a", "abc"), null))
.columns("dim1", "dim2")
.context(QUERY_CONTEXT_DEFAULT)
.build()
@@ -6798,22 +6798,22 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
- .setDimFilter(SELECTOR("dim1", "def", null))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
+ .setDimFilter(selector("dim1", "def", null))
.setContext(QUERY_CONTEXT_DEFAULT)
.build(),
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
- EXPRESSION_VIRTUAL_COLUMN("v0", "timestamp_extract(\"__time\",'MONTH','UTC')", ValueType.LONG)
+ expression_Virtual_Column("v0", "timestamp_extract(\"__time\",'MONTH','UTC')", ValueType.LONG)
)
.filters(
- AND(
- NOT(SELECTOR("dim1", "", null)),
- SELECTOR("dim2", "abc", null)
+ and(
+ not(selector("dim1", "", null)),
+ selector("dim2", "abc", null)
)
)
.columns("dim1", "v0")
@@ -6842,29 +6842,29 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
GroupByQuery
.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
- .setDimFilter(SELECTOR("dim1", "def", null))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
+ .setDimFilter(selector("dim1", "def", null))
.setContext(QUERY_CONTEXT_DEFAULT)
.build(),
GroupByQuery
.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setVirtualColumns(
- EXPRESSION_VIRTUAL_COLUMN("d0:v", "timestamp_extract(\"__time\",'MONTH','UTC')", ValueType.LONG)
+ expression_Virtual_Column("d0:v", "timestamp_extract(\"__time\",'MONTH','UTC')", ValueType.LONG)
)
.setDimFilter(
- AND(
- NOT(SELECTOR("dim1", "", null)),
- SELECTOR("dim2", "abc", null)
+ and(
+ not(selector("dim1", "", null)),
+ selector("dim2", "abc", null)
)
)
- .setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
- .setInterval(QSS(Filtration.eternity()))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(
- AGGS(
+ aggregators(
new CardinalityAggregatorFactory(
"a0",
null,
@@ -6907,25 +6907,25 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimFilter(NOT(SELECTOR("dim1", "", null)))
+ .setDimFilter(not(selector("dim1", "", null)))
.setDimensions(
- DIMS(new ExtractionDimensionSpec("dim1", "d0", new SubstringDimExtractionFn(0, 1)))
+ dimensionSpec(new ExtractionDimensionSpec("dim1", "d0", new SubstringDimExtractionFn(0, 1)))
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build(),
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimFilter(IN(
+ .setDimFilter(in(
"dim2",
ImmutableList.of("1", "2", "a", "d"),
new SubstringDimExtractionFn(0, 1)
))
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -6952,17 +6952,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE2)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimFilter(OR(
+ .setDimFilter(or(
new LikeDimFilter("dim1", "דר%", null, null),
new SelectorDimFilter("dim1", "друид", null)
))
- .setDimensions(DIMS(
+ .setDimensions(dimensionSpec(
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("dim2", "d1")
))
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
@@ -6981,15 +6981,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
- DIMS(
+ dimensionSpec(
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("dim2", "d1")
)
)
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(
@@ -7020,18 +7020,18 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
- DIMS(
+ dimensionSpec(
new DefaultDimensionSpec("dim1", "d0"),
new DefaultDimensionSpec("dim2", "d1")
)
)
.setAggregatorSpecs(
- AGGS(new CountAggregatorFactory("a0"), new DoubleSumAggregatorFactory("a1", "m2"))
+ aggregators(new CountAggregatorFactory("a0"), new DoubleSumAggregatorFactory("a1", "m2"))
)
- .setPostAggregatorSpecs(Collections.singletonList(EXPRESSION_POST_AGG(
+ .setPostAggregatorSpecs(Collections.singletonList(expresionPostAgg(
"s0",
"(\"a1\" / \"a0\")"
)))
@@ -7065,14 +7065,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
- DIMS(
+ dimensionSpec(
new DefaultDimensionSpec("dim1", "d0")
)
)
- .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0")))
+ .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setLimitSpec(
new DefaultLimitSpec(
Collections.singletonList(
@@ -7122,9 +7122,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(
+ .setDimensions(dimensionSpec(
new DefaultDimensionSpec("__time", "d0", ValueType.LONG),
new DefaultDimensionSpec("m2", "d1", ValueType.DOUBLE),
new DefaultDimensionSpec("dim1", "d2")
@@ -7132,13 +7132,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(
+ .setDimensions(dimensionSpec(
new DefaultDimensionSpec("d0", "_d0", ValueType.LONG),
new DefaultDimensionSpec("d2", "_d1", ValueType.STRING)
))
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new CountAggregatorFactory("a0")
))
.setLimitSpec(
@@ -7180,17 +7180,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
Collections.singletonList(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .filters(SELECTOR("dim2", "a", null))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .filters(selector("dim2", "a", null))
.granularity(Granularities.YEAR)
.aggregators(
- AGGS(
+ aggregators(
new DoubleSumAggregatorFactory("a0", "m1"),
new DoubleSumAggregatorFactory("a1", "m2")
)
)
.postAggregators(
- EXPRESSION_POST_AGG("p0", "(\"a0\" + \"a1\")")
+ expresionPostAgg("p0", "(\"a0\" + \"a1\")")
)
.descending(true)
.context(TIMESERIES_CONTEXT_DEFAULT)
@@ -7220,11 +7220,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
Collections.singletonList(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("m1", "d0", ValueType.FLOAT))
.filters("dim2", "a")
- .aggregators(AGGS(
+ .aggregators(aggregators(
new DoubleSumAggregatorFactory("a0:sum", "m2"),
new CountAggregatorFactory("a0:count"),
new DoubleSumAggregatorFactory("a1", "m1"),
@@ -7240,7 +7240,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new FieldAccessPostAggregator(null, "a0:count")
)
),
- EXPRESSION_POST_AGG("p0", "(\"a1\" + \"a2\")")
+ expresionPostAgg("p0", "(\"a1\" + \"a2\")")
)
)
.metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC))
@@ -7263,8 +7263,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .virtualColumns(EXPRESSION_VIRTUAL_COLUMN(
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .virtualColumns(expression_Virtual_Column(
"v0",
"concat(\"dim1\",'-',\"dim1\",'_',\"dim1\")",
ValueType.STRING
@@ -7289,8 +7289,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .virtualColumns(EXPRESSION_VIRTUAL_COLUMN(
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .virtualColumns(expression_Virtual_Column(
"v0",
"concat(\"dim1\",concat(\"dim2\",'x'),\"m2\",9999,\"dim1\")",
ValueType.STRING
@@ -7319,8 +7319,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .virtualColumns(EXPRESSION_VIRTUAL_COLUMN("v0", "concat(\"dim1\",\"dim1\")", ValueType.STRING))
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .virtualColumns(expression_Virtual_Column("v0", "concat(\"dim1\",\"dim1\")", ValueType.STRING))
.columns("v0")
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
@@ -7341,8 +7341,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Filtration.eternity()))
- .virtualColumns(EXPRESSION_VIRTUAL_COLUMN(
+ .intervals(querySegmentSpec(Filtration.eternity()))
+ .virtualColumns(expression_Virtual_Column(
"v0",
"concat(\"dim1\",CAST(\"m2\", 'STRING'))",
ValueType.STRING
@@ -7380,15 +7380,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.of("2000-01-01/2002-01-01")))
+ .intervals(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01")))
.granularity(Granularities.MONTH)
- .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
- new Object[]{3L, T("2000-01-01")},
- new Object[]{3L, T("2001-01-01")}
+ new Object[]{3L, t("2000-01-01")},
+ new Object[]{3L, t("2001-01-01")}
)
);
@@ -7406,20 +7406,20 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Intervals.utc(
+ .setInterval(querySegmentSpec(Intervals.utc(
DateTimes.of("2000-01-01").getMillis(),
JodaUtils.MAX_INSTANT
)))
.setGranularity(Granularities.ALL)
- .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
- .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
+ .setDimensions(dimensionSpec(new DefaultDimensionSpec("dim2", "d0")))
+ .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
)
)
- .setInterval(QSS(Filtration.eternity()))
+ .setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
- .setAggregatorSpecs(AGGS(
+ .setAggregatorSpecs(aggregators(
new LongSumAggregatorFactory("_a0", "a0"),
new CountAggregatorFactory("_a1")
))
@@ -7447,10 +7447,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
- .setInterval(QSS(Intervals.utc(DateTimes.of("2000-01-01").getMillis(), JodaUtils.MAX_INSTANT)))
+ .setInterval(querySegmentSpec(Intervals.utc(DateTimes.of("2000-01-01").getMillis(), JodaUtils.MAX_INSTANT)))
.setGranularity(Granularities.ALL)
- .setDimFilter(NOT(SELECTOR("dim1", "", null)))
- .setDimensions(DIMS(new ExtractionDimensionSpec(
+ .setDimFilter(not(selector("dim1", "", null)))
+ .setDimensions(dimensionSpec(new ExtractionDimensionSpec(
"dim1",
"d0",
new SubstringDimExtractionFn(0, 1)
@@ -7459,14 +7459,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build(),
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
- .intervals(QSS(Intervals.utc(DateTimes.of("2000-01-01").getMillis(), JodaUtils.MAX_INSTANT)))
+ .intervals(querySegmentSpec(Intervals.utc(DateTimes.of("2000-01-01").getMillis(), JodaUtils.MAX_INSTANT)))
.granularity(Granularities.ALL)
- .filters(IN(
+ .filters(in(
"dim2",
ImmutableList.of("1", "2", "a", "d"),
new SubstringDimExtractionFn(0, 1)
))
- .aggregators(AGGS(new CountAggregatorFactory("a0")))
+ .aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
@@ -7541,9 +7541,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim1")
- .filters(SELECTOR("f1", "0.1", null))
+ .filters(selector("f1", "0.1", null))
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
.limit(1)
.context(QUERY_CONTEXT_DEFAULT)
@@ -7563,9 +7563,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim1")
- .filters(SELECTOR("d1", "1.7", null))
+ .filters(selector("d1", "1.7", null))
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
.limit(1)
.context(QUERY_CONTEXT_DEFAULT)
@@ -7585,9 +7585,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
- .intervals(QSS(Filtration.eternity()))
+ .intervals(querySegmentSpec(Filtration.eternity()))
.columns("dim1")
- .filters(SELECTOR("l1", "7", null))
+ .filters(selector("l1", "7", null))
.resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)
.limit(1)
.context(QUERY_CONTEXT_DEFAULT)
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org