mirror of https://github.com/apache/druid.git
Merge branch 'quidem-record' into quidem-msq
This commit is contained in:
commit
b1ab252b31
|
@ -52,7 +52,7 @@ jobs:
|
|||
stale-issue-label: stale
|
||||
stale-pr-label: stale
|
||||
exempt-issue-labels: 'Evergreen,Security,Bug,Proposal,Design Review,Improvement,Performance,Refactoring,Apache,Area - Automation/Static Analysis,Area - Batch Indexing,Area - Cache,Area - Deep Storage,Area - Dependencies,Area - Dependency Injection,Area - Dev,Area - Documentation,Area - Extension,Area - Kafka/Kinesis Indexing,Area - Lookups,Area - Metadata,Area - Metrics/Event Emitting,Area - Null Handling,Area - Operations,Area - Query UI,Area - Querying,Area - Router,Area - Segment Balancing/Coordination,Area - Segment Format and Ser/De,Area - SQL,Area - Testing,Area - Web Console,Area - Zookeeper/Curator,Compatibility,Contributions Welcome,Development Blocker,Ease of Use,Error handling,HTTP,Incompatible,Stable API'
|
||||
exempt-pr-labels: 'Evergreen'
|
||||
exempt-pr-labels: 'Evergreen,Area - Dependencies'
|
||||
exempt-milestones: true
|
||||
exempt-assignees: true
|
||||
ascending: true
|
||||
|
|
|
@ -0,0 +1,193 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.benchmark;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.jackson.AggregatorsModule;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.query.QueryRunnerTestHelper;
|
||||
import org.apache.druid.query.aggregation.SerializablePairLongString;
|
||||
import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
|
||||
import org.apache.druid.query.groupby.ResultRow;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
import org.openjdk.jmh.annotations.Level;
|
||||
import org.openjdk.jmh.annotations.Measurement;
|
||||
import org.openjdk.jmh.annotations.Mode;
|
||||
import org.openjdk.jmh.annotations.OutputTimeUnit;
|
||||
import org.openjdk.jmh.annotations.Param;
|
||||
import org.openjdk.jmh.annotations.Scope;
|
||||
import org.openjdk.jmh.annotations.Setup;
|
||||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.annotations.Warmup;
|
||||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@State(Scope.Benchmark)
|
||||
@Fork(value = 1)
|
||||
@Warmup(iterations = 3)
|
||||
@Measurement(iterations = 5)
|
||||
public class GroupByDeserializationBenchmark
|
||||
{
|
||||
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
AggregatorsModule.registerComplexMetricsAndSerde();
|
||||
}
|
||||
|
||||
@Param({"100", "1000"})
|
||||
private int numDimensions;
|
||||
|
||||
@Param({"0", "0.25", "0.5", "0.75", "0.85", "0.95", "0.99", "1.0"})
|
||||
private double primitiveToComplexDimensionRatio;
|
||||
|
||||
@Param({"json", "serializablePairLongString"})
|
||||
private String complexDimensionType;
|
||||
|
||||
@Param({"true", "false"})
|
||||
private boolean backwardCompatibility;
|
||||
|
||||
private GroupByQuery sqlQuery;
|
||||
private String serializedRow;
|
||||
private GroupByQueryQueryToolChest groupByQueryQueryToolChest;
|
||||
private ObjectMapper decoratedMapper;
|
||||
|
||||
@Setup(Level.Trial)
|
||||
public void setup() throws JsonProcessingException
|
||||
{
|
||||
final ObjectMapper undecoratedMapper = TestHelper.makeJsonMapper();
|
||||
undecoratedMapper.registerModules(NestedDataModule.getJacksonModulesList());
|
||||
undecoratedMapper.registerModule(new AggregatorsModule());
|
||||
final Pair<GroupByQuery, String> sqlQueryAndResultRow = sqlQueryAndResultRow(
|
||||
numDimensions,
|
||||
primitiveToComplexDimensionRatio,
|
||||
complexDimensionType,
|
||||
undecoratedMapper
|
||||
);
|
||||
sqlQuery = sqlQueryAndResultRow.lhs;
|
||||
serializedRow = sqlQueryAndResultRow.rhs;
|
||||
|
||||
groupByQueryQueryToolChest = new GroupByQueryQueryToolChest(
|
||||
null,
|
||||
() -> new GroupByQueryConfig()
|
||||
{
|
||||
@Override
|
||||
public boolean isIntermediateResultAsMapCompat()
|
||||
{
|
||||
return backwardCompatibility;
|
||||
}
|
||||
},
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
||||
decoratedMapper = groupByQueryQueryToolChest.decorateObjectMapper(undecoratedMapper, sqlQuery);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void deserializeResultRows(Blackhole blackhole) throws JsonProcessingException
|
||||
{
|
||||
blackhole.consume(decoratedMapper.readValue(serializedRow, ResultRow.class));
|
||||
}
|
||||
|
||||
private static Pair<GroupByQuery, String> sqlQueryAndResultRow(
|
||||
final int numDimensions,
|
||||
final double primitiveToComplexDimensionRatio,
|
||||
final String complexDimensionType,
|
||||
final ObjectMapper mapper
|
||||
) throws JsonProcessingException
|
||||
{
|
||||
final int numPrimitiveDimensions = (int) Math.floor(primitiveToComplexDimensionRatio * numDimensions);
|
||||
final int numComplexDimensions = numDimensions - numPrimitiveDimensions;
|
||||
|
||||
final List<DimensionSpec> dimensions = new ArrayList<>();
|
||||
final List<Object> rowList = new ArrayList<>();
|
||||
|
||||
// Add timestamp
|
||||
rowList.add(DateTimes.of("2000").getMillis());
|
||||
|
||||
for (int i = 0; i < numPrimitiveDimensions; ++i) {
|
||||
dimensions.add(
|
||||
new DefaultDimensionSpec(
|
||||
StringUtils.format("primitive%d", i),
|
||||
StringUtils.format("primitive%d", i),
|
||||
ColumnType.STRING
|
||||
)
|
||||
);
|
||||
rowList.add("foo");
|
||||
}
|
||||
|
||||
for (int i = 0; i < numComplexDimensions; ++i) {
|
||||
dimensions.add(
|
||||
new DefaultDimensionSpec(
|
||||
StringUtils.format("complex%d", i),
|
||||
StringUtils.format("complex%d", i),
|
||||
ColumnType.ofComplex(complexDimensionType)
|
||||
)
|
||||
);
|
||||
|
||||
// Serialized version of this object is a valid value for both json and long-string pair dimensions
|
||||
rowList.add(new SerializablePairLongString(1L, "test"));
|
||||
}
|
||||
|
||||
// Add aggregator
|
||||
rowList.add(100);
|
||||
|
||||
// Add post aggregator
|
||||
rowList.add(10.0);
|
||||
|
||||
GroupByQuery query = GroupByQuery.builder()
|
||||
.setDataSource("foo")
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC)
|
||||
.setDimensions(dimensions)
|
||||
.setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT)
|
||||
.setPostAggregatorSpecs(Collections.singletonList(new ConstantPostAggregator(
|
||||
"post",
|
||||
10
|
||||
)))
|
||||
.setContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true))
|
||||
.setGranularity(Granularities.DAY)
|
||||
.build();
|
||||
|
||||
return Pair.of(query, mapper.writeValueAsString(rowList));
|
||||
}
|
||||
}
|
|
@ -55,6 +55,7 @@ import org.apache.druid.sql.calcite.planner.PlannerResult;
|
|||
import org.apache.druid.sql.calcite.run.SqlEngine;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.hook.DruidHookDispatcher;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
|
@ -202,7 +203,8 @@ public class InPlanningBenchmark
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
String prefix = ("explain plan for select long1 from foo where long1 in ");
|
||||
|
|
|
@ -262,12 +262,8 @@ public class ScanBenchmark
|
|||
rowsPerSegment
|
||||
);
|
||||
|
||||
final ScanQueryConfig config = new ScanQueryConfig().setLegacy(false);
|
||||
factory = new ScanQueryRunnerFactory(
|
||||
new ScanQueryQueryToolChest(
|
||||
config,
|
||||
DefaultGenericQueryMetricsFactory.instance()
|
||||
),
|
||||
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
|
||||
new ScanQueryEngine(),
|
||||
new ScanQueryConfig()
|
||||
);
|
||||
|
|
|
@ -89,6 +89,7 @@ import org.apache.druid.sql.calcite.util.CalciteTests;
|
|||
import org.apache.druid.sql.calcite.util.LookylooModule;
|
||||
import org.apache.druid.sql.calcite.util.QueryFrameworkUtils;
|
||||
import org.apache.druid.sql.calcite.util.testoperator.CalciteTestOperatorModule;
|
||||
import org.apache.druid.sql.hook.DruidHookDispatcher;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
|
@ -674,7 +675,8 @@ public class SqlBenchmark
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
new JoinableFactoryWrapper(QueryFrameworkUtils.createDefaultJoinableFactory(injector)),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
return Pair.of(plannerFactory, engine);
|
||||
|
|
|
@ -59,6 +59,7 @@ import org.apache.druid.sql.calcite.planner.PlannerResult;
|
|||
import org.apache.druid.sql.calcite.run.SqlEngine;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.hook.DruidHookDispatcher;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
|
@ -364,7 +365,8 @@ public class SqlExpressionBenchmark
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
try {
|
||||
|
|
|
@ -57,6 +57,7 @@ import org.apache.druid.sql.calcite.planner.PlannerResult;
|
|||
import org.apache.druid.sql.calcite.run.SqlEngine;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.hook.DruidHookDispatcher;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
|
@ -347,7 +348,8 @@ public class SqlGroupByBenchmark
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
try {
|
||||
|
|
|
@ -63,6 +63,7 @@ import org.apache.druid.sql.calcite.planner.PlannerResult;
|
|||
import org.apache.druid.sql.calcite.run.SqlEngine;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.hook.DruidHookDispatcher;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
|
@ -402,7 +403,8 @@ public class SqlNestedDataBenchmark
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
|
||||
try {
|
||||
|
|
|
@ -50,6 +50,7 @@ import org.apache.druid.sql.calcite.planner.PlannerResult;
|
|||
import org.apache.druid.sql.calcite.run.SqlEngine;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.hook.DruidHookDispatcher;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
|
@ -132,7 +133,8 @@ public class SqlVsNativeBenchmark
|
|||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper(),
|
||||
CatalogResolver.NULL_RESOLVER,
|
||||
new AuthConfig()
|
||||
new AuthConfig(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
groupByQuery = GroupByQuery
|
||||
.builder()
|
||||
|
|
|
@ -92,21 +92,12 @@ public class NewestSegmentFirstPolicyBenchmark
|
|||
final String dataSource = DATA_SOURCE_PREFIX + i;
|
||||
compactionConfigs.put(
|
||||
dataSource,
|
||||
new DataSourceCompactionConfig(
|
||||
dataSource,
|
||||
0,
|
||||
inputSegmentSizeBytes,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
DataSourceCompactionConfig
|
||||
.builder()
|
||||
.forDataSource(dataSource)
|
||||
.withTaskPriority(0)
|
||||
.withInputSegmentSizeBytes(inputSegmentSizeBytes)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@ OPTS+=" -Dsurefire.rerunFailingTestsCount=0"
|
|||
OPTS+=" -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugin.surefire.SurefirePlugin=INFO"
|
||||
[[ $@ =~ "-q" ]] && OPTS+=" -Dsurefire.trimStackTrace=true"
|
||||
|
||||
OPTS+=" -pl quidem-it -Dtest=QTest"
|
||||
OPTS+=" -pl quidem-ut -Dtest=QTest"
|
||||
OPTS+=" org.apache.maven.plugins:maven-surefire-plugin:test"
|
||||
|
||||
case "$1" in
|
||||
|
|
|
@ -474,7 +474,6 @@ The response shows an example report for a query.
|
|||
"agent_type",
|
||||
"timestamp"
|
||||
],
|
||||
"legacy": false,
|
||||
"context": {
|
||||
"finalize": false,
|
||||
"finalizeAggregations": false,
|
||||
|
|
|
@ -80,7 +80,7 @@ All of these community extensions can be downloaded using [pull-deps](../operati
|
|||
|aliyun-oss-extensions|Aliyun OSS deep storage |[link](../development/extensions-contrib/aliyun-oss-extensions.md)|
|
||||
|ambari-metrics-emitter|Ambari Metrics Emitter |[link](../development/extensions-contrib/ambari-metrics-emitter.md)|
|
||||
|druid-cassandra-storage|Apache Cassandra deep storage.|[link](../development/extensions-contrib/cassandra.md)|
|
||||
|druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage and firehose.|[link](../development/extensions-contrib/cloudfiles.md)|
|
||||
|druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage.|[link](../development/extensions-contrib/cloudfiles.md)|
|
||||
|druid-compressed-bigdecimal|Compressed Big Decimal Type | [link](../development/extensions-contrib/compressed-big-decimal.md)|
|
||||
|druid-ddsketch|Support for DDSketch approximate quantiles based on [DDSketch](https://github.com/datadog/sketches-java) | [link](../development/extensions-contrib/ddsketch-quantiles.md)|
|
||||
|druid-deltalake-extensions|Support for ingesting Delta Lake tables.|[link](../development/extensions-contrib/delta-lake.md)|
|
||||
|
|
|
@ -395,7 +395,6 @@ Metric monitoring is an essential part of Druid operations. The following monito
|
|||
|`org.apache.druid.java.util.metrics.CgroupCpuSetMonitor`|Reports CPU core/HT and memory node allocations as per the `cpuset` cgroup.|
|
||||
|`org.apache.druid.java.util.metrics.CgroupDiskMonitor`|Reports disk statistic as per the blkio cgroup.|
|
||||
|`org.apache.druid.java.util.metrics.CgroupMemoryMonitor`|Reports memory statistic as per the memory cgroup.|
|
||||
|`org.apache.druid.server.metrics.EventReceiverFirehoseMonitor`|Reports how many events have been queued in the EventReceiverFirehose.|
|
||||
|`org.apache.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical services. Available only on Historical services.|
|
||||
|`org.apache.druid.server.metrics.SegmentStatsMonitor` | **EXPERIMENTAL** Reports statistics about segments on Historical services. Available only on Historical services. Not to be used when lazy loading is configured.|
|
||||
|`org.apache.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted.|
|
||||
|
@ -607,7 +606,7 @@ the [HDFS input source](../ingestion/input-sources.md#hdfs-input-source).
|
|||
|
||||
|Property|Possible values|Description|Default|
|
||||
|--------|---------------|-----------|-------|
|
||||
|`druid.ingestion.hdfs.allowedProtocols`|List of protocols|Allowed protocols for the HDFS input source and HDFS firehose.|`["hdfs"]`|
|
||||
|`druid.ingestion.hdfs.allowedProtocols`|List of protocols|Allowed protocols for the HDFS input source.|`["hdfs"]`|
|
||||
|
||||
#### HTTP input source
|
||||
|
||||
|
@ -616,7 +615,7 @@ the [HTTP input source](../ingestion/input-sources.md#http-input-source).
|
|||
|
||||
|Property|Possible values|Description|Default|
|
||||
|--------|---------------|-----------|-------|
|
||||
|`druid.ingestion.http.allowedProtocols`|List of protocols|Allowed protocols for the HTTP input source and HTTP firehose.|`["http", "https"]`|
|
||||
|`druid.ingestion.http.allowedProtocols`|List of protocols|Allowed protocols for the HTTP input source.|`["http", "https"]`|
|
||||
|
||||
### External data access security configuration
|
||||
|
||||
|
@ -1501,7 +1500,6 @@ Additional Peon configs include:
|
|||
|`druid.peon.mode`|One of `local` or `remote`. Setting this property to `local` means you intend to run the Peon as a standalone process which is not recommended.|`remote`|
|
||||
|`druid.indexer.task.baseDir`|Base temporary working directory.|`System.getProperty("java.io.tmpdir")`|
|
||||
|`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|`${druid.indexer.task.baseDir}/persistent/task`|
|
||||
|`druid.indexer.task.batchProcessingMode`| Batch ingestion tasks have three operating modes to control construction and tracking for intermediary segments: `OPEN_SEGMENTS`, `CLOSED_SEGMENTS`, and `CLOSED_SEGMENT_SINKS`. `OPEN_SEGMENTS` uses the streaming ingestion code path and performs a `mmap` on intermediary segments to build a timeline to make these segments available to realtime queries. Batch ingestion doesn't require intermediary segments, so the default mode, `CLOSED_SEGMENTS`, eliminates `mmap` of intermediary segments. `CLOSED_SEGMENTS` mode still tracks the entire set of segments in heap. The `CLOSED_SEGMENTS_SINKS` mode is the most aggressive configuration and should have the smallest memory footprint. It eliminates in-memory tracking and `mmap` of intermediary segments produced during segment creation. `CLOSED_SEGMENTS_SINKS` mode isn't as well tested as other modes so is currently considered experimental. You can use `OPEN_SEGMENTS` mode if problems occur with the 2 newer modes. |`CLOSED_SEGMENTS`|
|
||||
|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|`org.apache.hadoop:hadoop-client-api:3.3.6`, `org.apache.hadoop:hadoop-client-runtime:3.3.6`|
|
||||
|`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|75000|
|
||||
|`druid.indexer.task.directoryLockTimeout`|Wait this long for zombie Peons to exit before giving up on their replacements.|PT10M|
|
||||
|
|
|
@ -40,59 +40,3 @@ To use this Apache Druid extension, [include](../../configuration/extensions.md#
|
|||
|`druid.cloudfiles.apiKey`||Rackspace Cloud API key.|Must be set.|
|
||||
|`druid.cloudfiles.provider`|rackspace-cloudfiles-us,rackspace-cloudfiles-uk|Name of the provider depending on the region.|Must be set.|
|
||||
|`druid.cloudfiles.useServiceNet`|true,false|Whether to use the internal service net.|true|
|
||||
|
||||
## Firehose
|
||||
|
||||
<a name="firehose"></a>
|
||||
|
||||
#### StaticCloudFilesFirehose
|
||||
|
||||
This firehose ingests events, similar to the StaticAzureBlobStoreFirehose, but from Rackspace's Cloud Files.
|
||||
|
||||
Data is newline delimited, with one JSON object per line and parsed as per the `InputRowParser` configuration.
|
||||
|
||||
The storage account is shared with the one used for Rackspace's Cloud Files deep storage functionality, but blobs can be in a different region and container.
|
||||
|
||||
As with the Azure blobstore, it is assumed to be gzipped if the extension ends in .gz
|
||||
|
||||
This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native-batch.md).
|
||||
Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object.
|
||||
|
||||
Sample spec:
|
||||
|
||||
```json
|
||||
"firehose" : {
|
||||
"type" : "static-cloudfiles",
|
||||
"blobs": [
|
||||
{
|
||||
"region": "DFW"
|
||||
"container": "container",
|
||||
"path": "/path/to/your/file.json"
|
||||
},
|
||||
{
|
||||
"region": "ORD"
|
||||
"container": "anothercontainer",
|
||||
"path": "/another/path.json"
|
||||
}
|
||||
]
|
||||
}
|
||||
```
|
||||
This firehose provides caching and prefetching features. In IndexTask, a firehose can be read twice if intervals or
|
||||
shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow.
|
||||
|
||||
|property|description|default|required?|
|
||||
|--------|-----------|-------|---------|
|
||||
|type|This should be `static-cloudfiles`.|N/A|yes|
|
||||
|blobs|JSON array of Cloud Files blobs.|N/A|yes|
|
||||
|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache.|1073741824|no|
|
||||
|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|no|
|
||||
|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|no|
|
||||
|fetchTimeout|Timeout for fetching a Cloud Files object.|60000|no|
|
||||
|maxFetchRetry|Maximum retry for fetching a Cloud Files object.|3|no|
|
||||
|
||||
Cloud Files Blobs:
|
||||
|
||||
|property|description|default|required?|
|
||||
|--------|-----------|-------|---------|
|
||||
|container|Name of the Cloud Files container|N/A|yes|
|
||||
|path|The path where data is located.|N/A|yes|
|
||||
|
|
|
@ -87,7 +87,7 @@ In most cases, the configuration options map directly to the [postgres JDBC conn
|
|||
| `druid.metadata.postgres.ssl.sslPasswordCallback` | The classname of the SSL password provider. | none | no |
|
||||
| `druid.metadata.postgres.dbTableSchema` | druid meta table schema | `public` | no |
|
||||
|
||||
### PostgreSQL Firehose
|
||||
### PostgreSQL InputSource
|
||||
|
||||
The PostgreSQL extension provides an implementation of an [SQL input source](../../ingestion/input-sources.md) which can be used to ingest data into Druid from a PostgreSQL database.
|
||||
|
||||
|
|
|
@ -53,8 +53,17 @@ Most of the coordination logic for (real-time) ingestion is in the Druid indexin
|
|||
|
||||
## Real-time Ingestion
|
||||
|
||||
Druid loads data through `FirehoseFactory.java` classes. Firehoses often wrap other firehoses, where, similar to the design of the
|
||||
query runners, each firehose adds a layer of logic, and the persist and hand-off logic is in `RealtimePlumber.java`.
|
||||
Druid streaming tasks are based on the 'seekable stream' classes such as `SeekableStreamSupervisor.java`,
|
||||
`SeekableStreamIndexTask.java`, and `SeekableStreamIndexTaskRunner.java`. The data processing happens through
|
||||
`StreamAppenderator.java`, and the persist and hand-off logic is in `StreamAppenderatorDriver.java`.
|
||||
|
||||
## Native Batch Ingestion
|
||||
|
||||
Druid native batch ingestion main task types are based on `AbstractBatchTask.java` and `AbstractBatchSubtask.java`.
|
||||
Parallel processing uses `ParallelIndexSupervisorTask.java`, which spawns subtasks to perform various operations such
|
||||
as data analysis and partitioning depending on the task specification. Segment generation happens in
|
||||
`SinglePhaseSubTask.java`, `PartialHashSegmentGenerateTask.java`, or `PartialRangeSegmentGenerateTask.java` through
|
||||
`BatchAppenderator`, and the persist and hand-off logic is in `BatchAppenderatorDriver.java`.
|
||||
|
||||
## Hadoop-based Batch Ingestion
|
||||
|
||||
|
|
|
@ -254,7 +254,6 @@ The Kinesis indexing service supports the following values for `inputFormat`:
|
|||
* `json`
|
||||
* `kafka`
|
||||
* `avro_stream`
|
||||
* `avro_ocf`
|
||||
* `protobuf`
|
||||
|
||||
You can use `parser` to read [`thrift`](../development/extensions-contrib/thrift.md) formats.
|
||||
|
|
|
@ -143,7 +143,6 @@ The Kinesis indexing service supports the following values for `inputFormat`:
|
|||
* `tvs`
|
||||
* `json`
|
||||
* `avro_stream`
|
||||
* `avro_ocf`
|
||||
* `protobuf`
|
||||
|
||||
You can use `parser` to read [`thrift`](../development/extensions-contrib/thrift.md) formats.
|
||||
|
|
|
@ -24,319 +24,5 @@ sidebar_label: "Firehose (deprecated)"
|
|||
-->
|
||||
|
||||
:::info
|
||||
Firehose ingestion is deprecated. See [Migrate from firehose to input source ingestion](../operations/migrate-from-firehose-ingestion.md) for instructions on migrating from firehose ingestion to using native batch ingestion input sources.
|
||||
Firehose ingestion has been removed in Druid 26.0. See [Migrate from firehose to input source ingestion](../operations/migrate-from-firehose-ingestion.md) for instructions on migrating from firehose ingestion to using native batch ingestion input sources.
|
||||
:::
|
||||
|
||||
There are several firehoses readily available in Druid, some are meant for examples, others can be used directly in a production environment.
|
||||
|
||||
## StaticS3Firehose
|
||||
|
||||
You need to include the [`druid-s3-extensions`](../development/extensions-core/s3.md) as an extension to use the StaticS3Firehose.
|
||||
|
||||
This firehose ingests events from a predefined list of S3 objects.
|
||||
This firehose is _splittable_ and can be used by the [Parallel task](./native-batch.md).
|
||||
Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object.
|
||||
|
||||
Sample spec:
|
||||
|
||||
```json
|
||||
"firehose" : {
|
||||
"type" : "static-s3",
|
||||
"uris": ["s3://foo/bar/file.gz", "s3://bar/foo/file2.gz"]
|
||||
}
|
||||
```
|
||||
|
||||
This firehose provides caching and prefetching features. In the Simple task, a firehose can be read twice if intervals or
|
||||
shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow.
|
||||
Note that prefetching or caching isn't that useful in the Parallel task.
|
||||
|
||||
|property|description|default|required?|
|
||||
|--------|-----------|-------|---------|
|
||||
|type|This should be `static-s3`.|None|yes|
|
||||
|uris|JSON array of URIs where s3 files to be ingested are located.|None|`uris` or `prefixes` must be set|
|
||||
|prefixes|JSON array of URI prefixes for the locations of s3 files to be ingested.|None|`uris` or `prefixes` must be set|
|
||||
|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|no|
|
||||
|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|no|
|
||||
|prefetchTriggerBytes|Threshold to trigger prefetching s3 objects.|maxFetchCapacityBytes / 2|no|
|
||||
|fetchTimeout|Timeout for fetching an s3 object.|60000|no|
|
||||
|maxFetchRetry|Maximum retry for fetching an s3 object.|3|no|
|
||||
|
||||
## StaticGoogleBlobStoreFirehose
|
||||
|
||||
You need to include the [`druid-google-extensions`](../development/extensions-core/google.md) as an extension to use the StaticGoogleBlobStoreFirehose.
|
||||
|
||||
This firehose ingests events, similar to the StaticS3Firehose, but from an Google Cloud Store.
|
||||
|
||||
As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz
|
||||
|
||||
This firehose is _splittable_ and can be used by the [Parallel task](./native-batch.md).
|
||||
Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object.
|
||||
|
||||
Sample spec:
|
||||
|
||||
```json
|
||||
"firehose" : {
|
||||
"type" : "static-google-blobstore",
|
||||
"blobs": [
|
||||
{
|
||||
"bucket": "foo",
|
||||
"path": "/path/to/your/file.json"
|
||||
},
|
||||
{
|
||||
"bucket": "bar",
|
||||
"path": "/another/path.json"
|
||||
}
|
||||
]
|
||||
}
|
||||
```
|
||||
|
||||
This firehose provides caching and prefetching features. In the Simple task, a firehose can be read twice if intervals or
|
||||
shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow.
|
||||
Note that prefetching or caching isn't that useful in the Parallel task.
|
||||
|
||||
|property|description|default|required?|
|
||||
|--------|-----------|-------|---------|
|
||||
|type|This should be `static-google-blobstore`.|None|yes|
|
||||
|blobs|JSON array of Google Blobs.|None|yes|
|
||||
|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|no|
|
||||
|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|no|
|
||||
|prefetchTriggerBytes|Threshold to trigger prefetching Google Blobs.|maxFetchCapacityBytes / 2|no|
|
||||
|fetchTimeout|Timeout for fetching a Google Blob.|60000|no|
|
||||
|maxFetchRetry|Maximum retry for fetching a Google Blob.|3|no|
|
||||
|
||||
Google Blobs:
|
||||
|
||||
|property|description|default|required?|
|
||||
|--------|-----------|-------|---------|
|
||||
|bucket|Name of the Google Cloud bucket|None|yes|
|
||||
|path|The path where data is located.|None|yes|
|
||||
|
||||
## HDFSFirehose
|
||||
|
||||
You need to include the [`druid-hdfs-storage`](../development/extensions-core/hdfs.md) as an extension to use the HDFSFirehose.
|
||||
|
||||
This firehose ingests events from a predefined list of files from the HDFS storage.
|
||||
This firehose is _splittable_ and can be used by the [Parallel task](./native-batch.md).
|
||||
Since each split represents an HDFS file, each worker task of `index_parallel` will read files.
|
||||
|
||||
Sample spec:
|
||||
|
||||
```json
|
||||
"firehose" : {
|
||||
"type" : "hdfs",
|
||||
"paths": "/foo/bar,/foo/baz"
|
||||
}
|
||||
```
|
||||
|
||||
This firehose provides caching and prefetching features. During native batch indexing, a firehose can be read twice if
|
||||
`intervals` are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scanning
|
||||
of files is slow.
|
||||
Note that prefetching or caching isn't that useful in the Parallel task.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|type|This should be `hdfs`.|none (required)|
|
||||
|paths|HDFS paths. Can be either a JSON array or comma-separated string of paths. Wildcards like `*` are supported in these paths.|none (required)|
|
||||
|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|
|
||||
|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|
|
||||
|prefetchTriggerBytes|Threshold to trigger prefetching files.|maxFetchCapacityBytes / 2|
|
||||
|fetchTimeout|Timeout for fetching each file.|60000|
|
||||
|maxFetchRetry|Maximum number of retries for fetching each file.|3|
|
||||
|
||||
You can also ingest from other storage using the HDFS firehose if the HDFS client supports that storage.
|
||||
However, if you want to ingest from cloud storage, consider using the service-specific input source for your data storage.
|
||||
If you want to use a non-hdfs protocol with the HDFS firehose, you need to include the protocol you want
|
||||
in `druid.ingestion.hdfs.allowedProtocols`. See [HDFS firehose security configuration](../configuration/index.md#hdfs-input-source) for more details.
|
||||
|
||||
## LocalFirehose
|
||||
|
||||
This Firehose can be used to read the data from files on local disk, and is mainly intended for proof-of-concept testing, and works with `string` typed parsers.
|
||||
This Firehose is _splittable_ and can be used by [native parallel index tasks](native-batch.md).
|
||||
Since each split represents a file in this Firehose, each worker task of `index_parallel` will read a file.
|
||||
A sample local Firehose spec is shown below:
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "local",
|
||||
"filter" : "*.csv",
|
||||
"baseDir": "/data/directory"
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|This should be "local".|yes|
|
||||
|filter|A wildcard filter for files. See [here](http://commons.apache.org/proper/commons-io/apidocs/org/apache/commons/io/filefilter/WildcardFileFilter) for more information.|yes|
|
||||
|baseDir|directory to search recursively for files to be ingested. |yes|
|
||||
|
||||
<a name="http-firehose"></a>
|
||||
|
||||
## HttpFirehose
|
||||
|
||||
This Firehose can be used to read the data from remote sites via HTTP, and works with `string` typed parsers.
|
||||
This Firehose is _splittable_ and can be used by [native parallel index tasks](native-batch.md).
|
||||
Since each split represents a file in this Firehose, each worker task of `index_parallel` will read a file.
|
||||
A sample HTTP Firehose spec is shown below:
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "http",
|
||||
"uris": ["http://example.com/uri1", "http://example2.com/uri2"]
|
||||
}
|
||||
```
|
||||
|
||||
You can only use protocols listed in the `druid.ingestion.http.allowedProtocols` property as HTTP firehose input sources.
|
||||
The `http` and `https` protocols are allowed by default. See [HTTP firehose security configuration](../configuration/index.md#http-input-source) for more details.
|
||||
|
||||
The below configurations can be optionally used if the URIs specified in the spec require a Basic Authentication Header.
|
||||
Omitting these fields from your spec will result in HTTP requests with no Basic Authentication Header.
|
||||
|
||||
|property|description|default|
|
||||
|--------|-----------|-------|
|
||||
|httpAuthenticationUsername|Username to use for authentication with specified URIs|None|
|
||||
|httpAuthenticationPassword|PasswordProvider to use with specified URIs|None|
|
||||
|
||||
Example with authentication fields using the DefaultPassword provider (this requires the password to be in the ingestion spec):
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "http",
|
||||
"uris": ["http://example.com/uri1", "http://example2.com/uri2"],
|
||||
"httpAuthenticationUsername": "username",
|
||||
"httpAuthenticationPassword": "password123"
|
||||
}
|
||||
```
|
||||
|
||||
You can also use the other existing Druid PasswordProviders. Here is an example using the EnvironmentVariablePasswordProvider:
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "http",
|
||||
"uris": ["http://example.com/uri1", "http://example2.com/uri2"],
|
||||
"httpAuthenticationUsername": "username",
|
||||
"httpAuthenticationPassword": {
|
||||
"type": "environment",
|
||||
"variable": "HTTP_FIREHOSE_PW"
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
The below configurations can optionally be used for tuning the Firehose performance.
|
||||
Note that prefetching or caching isn't that useful in the Parallel task.
|
||||
|
||||
|property|description|default|
|
||||
|--------|-----------|-------|
|
||||
|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|
|
||||
|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|
|
||||
|prefetchTriggerBytes|Threshold to trigger prefetching HTTP objects.|maxFetchCapacityBytes / 2|
|
||||
|fetchTimeout|Timeout for fetching an HTTP object.|60000|
|
||||
|maxFetchRetry|Maximum retries for fetching an HTTP object.|3|
|
||||
|
||||
<a name="segment-firehose"></a>
|
||||
|
||||
## IngestSegmentFirehose
|
||||
|
||||
This Firehose can be used to read the data from existing druid segments, potentially using a new schema and changing the name, dimensions, metrics, rollup, etc. of the segment.
|
||||
This Firehose is _splittable_ and can be used by [native parallel index tasks](native-batch.md).
|
||||
This firehose will accept any type of parser, but will only utilize the list of dimensions and the timestamp specification.
|
||||
A sample ingest Firehose spec is shown below:
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "ingestSegment",
|
||||
"dataSource": "wikipedia",
|
||||
"interval": "2013-01-01/2013-01-02"
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|This should be "ingestSegment".|yes|
|
||||
|dataSource|A String defining the data source to fetch rows from, very similar to a table in a relational database|yes|
|
||||
|interval|A String representing the ISO-8601 interval. This defines the time range to fetch the data over.|yes|
|
||||
|dimensions|The list of dimensions to select. If left empty, no dimensions are returned. If left null or not defined, all dimensions are returned. |no|
|
||||
|metrics|The list of metrics to select. If left empty, no metrics are returned. If left null or not defined, all metrics are selected.|no|
|
||||
|filter| See [Filters](../querying/filters.md)|no|
|
||||
|maxInputSegmentBytesPerTask|Deprecated. Use [Segments Split Hint Spec](./native-batch.md#segments-split-hint-spec) instead. When used with the native parallel index task, the maximum number of bytes of input segments to process in a single task. If a single segment is larger than this number, it will be processed by itself in a single task (input segments are never split across tasks). Defaults to 150MB.|no|
|
||||
|
||||
<a name="sql-firehose"></a>
|
||||
|
||||
## SqlFirehose
|
||||
|
||||
This Firehose can be used to ingest events residing in an RDBMS. The database connection information is provided as part of the ingestion spec.
|
||||
For each query, the results are fetched locally and indexed.
|
||||
If there are multiple queries from which data needs to be indexed, queries are prefetched in the background, up to `maxFetchCapacityBytes` bytes.
|
||||
This Firehose is _splittable_ and can be used by [native parallel index tasks](native-batch.md).
|
||||
This firehose will accept any type of parser, but will only utilize the list of dimensions and the timestamp specification. See the extension documentation for more detailed ingestion examples.
|
||||
|
||||
Requires one of the following extensions:
|
||||
* [MySQL Metadata Store](../development/extensions-core/mysql.md).
|
||||
* [PostgreSQL Metadata Store](../development/extensions-core/postgresql.md).
|
||||
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "sql",
|
||||
"database": {
|
||||
"type": "mysql",
|
||||
"connectorConfig": {
|
||||
"connectURI": "jdbc:mysql://host:port/schema",
|
||||
"user": "user",
|
||||
"password": "password"
|
||||
}
|
||||
},
|
||||
"sqls": ["SELECT * FROM table1", "SELECT * FROM table2"]
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|default|required?|
|
||||
|--------|-----------|-------|---------|
|
||||
|type|This should be "sql".||Yes|
|
||||
|database|Specifies the database connection details. The database type corresponds to the extension that supplies the `connectorConfig` support. The specified extension must be loaded into Druid:<br/><br/><ul><li>[mysql-metadata-storage](../development/extensions-core/mysql.md) for `mysql`</li><li> [postgresql-metadata-storage](../development/extensions-core/postgresql.md) extension for `postgresql`.</li></ul><br/><br/>You can selectively allow JDBC properties in `connectURI`. See [JDBC connections security config](../configuration/index.md#jdbc-connections-to-external-databases) for more details.||Yes|
|
||||
|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|No|
|
||||
|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|No|
|
||||
|prefetchTriggerBytes|Threshold to trigger prefetching SQL result objects.|maxFetchCapacityBytes / 2|No|
|
||||
|fetchTimeout|Timeout for fetching the result set.|60000|No|
|
||||
|foldCase|Toggle case folding of database column names. This may be enabled in cases where the database returns case insensitive column names in query results.|false|No|
|
||||
|sqls|List of SQL queries where each SQL query would retrieve the data to be indexed.||Yes|
|
||||
|
||||
### Database
|
||||
|
||||
|property|description|default|required?|
|
||||
|--------|-----------|-------|---------|
|
||||
|type|The type of database to query. Valid values are `mysql` and `postgresql`_||Yes|
|
||||
|connectorConfig|Specify the database connection properties via `connectURI`, `user` and `password`||Yes|
|
||||
|
||||
## InlineFirehose
|
||||
|
||||
This Firehose can be used to read the data inlined in its own spec.
|
||||
It can be used for demos or for quickly testing out parsing and schema, and works with `string` typed parsers.
|
||||
A sample inline Firehose spec is shown below:
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "inline",
|
||||
"data": "0,values,formatted\n1,as,CSV"
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|This should be "inline".|yes|
|
||||
|data|Inlined data to ingest.|yes|
|
||||
|
||||
## CombiningFirehose
|
||||
|
||||
This Firehose can be used to combine and merge data from a list of different Firehoses.
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "combining",
|
||||
"delegates": [ { firehose1 }, { firehose2 }, ... ]
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|This should be "combining"|yes|
|
||||
|delegates|List of Firehoses to combine data from|yes|
|
|
@ -428,15 +428,6 @@ These metrics are available only when `druid.zk.service.enabled = true`.
|
|||
|`zk/connected`|Indicator of connection status. `1` for connected, `0` for disconnected. Emitted once per monitor period.|None|1|
|
||||
|`zk/reconnect/time`|Amount of time, in milliseconds, that a server was disconnected from ZooKeeper before reconnecting. Emitted on reconnection. Not emitted if connection to ZooKeeper is permanently lost, because in this case, there is no reconnection.|None|Not present|
|
||||
|
||||
### EventReceiverFirehose
|
||||
|
||||
The following metric is only available if the `EventReceiverFirehoseMonitor` module is included.
|
||||
|
||||
|Metric|Description|Dimensions|Normal value|
|
||||
|------|-----------|----------|------------|
|
||||
|`ingest/events/buffered`|Number of events queued in the `EventReceiverFirehose` buffer.|`serviceName`, `dataSource`, `taskId`, `taskType`, `bufferCapacity`|Equal to the current number of events in the buffer queue.|
|
||||
|`ingest/bytes/received`|Number of bytes received by the `EventReceiverFirehose`.|`serviceName`, `dataSource`, `taskId`, `taskType`|Varies|
|
||||
|
||||
## Sys [Deprecated]
|
||||
|
||||
> SysMonitor is now deprecated and will be removed in future releases.
|
||||
|
|
|
@ -23,9 +23,7 @@ sidebar_label: "Migrate from firehose"
|
|||
~ under the License.
|
||||
-->
|
||||
|
||||
Apache deprecated support for Druid firehoses in version 0.17. Support for firehose ingestion will be removed in version 26.0.
|
||||
|
||||
If you're using a firehose for batch ingestion, we strongly recommend that you follow the instructions on this page to transition to using native batch ingestion input sources as soon as possible.
|
||||
Apache deprecated support for Druid firehoses in version 0.17. Support for firehose ingestion was removed in version 26.0.
|
||||
|
||||
Firehose ingestion doesn't work with newer Druid versions, so you must be using an ingestion spec with a defined input source before you upgrade.
|
||||
|
||||
|
|
|
@ -217,7 +217,6 @@ The following shows an example log emitter output:
|
|||
"user",
|
||||
"v0"
|
||||
],
|
||||
"legacy": false,
|
||||
"context":
|
||||
{
|
||||
"populateCache": false,
|
||||
|
|
|
@ -52,7 +52,7 @@ The following recommendations apply to the network where Druid runs:
|
|||
The following recommendation applies to Druid's authorization and authentication model:
|
||||
* Only grant `WRITE` permissions to any `DATASOURCE` to trusted users. Druid's trust model assumes those users have the same privileges as the operating system user that runs the web console process. Additionally, users with `WRITE` permissions can make changes to datasources and they have access to both task and supervisor update (POST) APIs which may affect ingestion.
|
||||
* Only grant `STATE READ`, `STATE WRITE`, `CONFIG WRITE`, and `DATASOURCE WRITE` permissions to highly-trusted users. These permissions allow users to access resources on behalf of the Druid server process regardless of the datasource.
|
||||
* If your Druid client application allows less-trusted users to control the input source or firehose of an ingestion task, validate the URLs from the users. It is possible to point unchecked URLs to other locations and resources within your network or local file system.
|
||||
* If your Druid client application allows less-trusted users to control the input source of an ingestion task, validate the URLs from the users. It is possible to point unchecked URLs to other locations and resources within your network or local file system.
|
||||
|
||||
## Enable TLS
|
||||
|
||||
|
|
|
@ -71,46 +71,10 @@ The following shows an example `dimensionsSpec` for native ingestion of the data
|
|||
|
||||
### SQL-based ingestion
|
||||
|
||||
#### `arrayIngestMode`
|
||||
|
||||
Arrays can be inserted with [SQL-based ingestion](../multi-stage-query/index.md) when you include the query context
|
||||
parameter `arrayIngestMode: array`.
|
||||
|
||||
When `arrayIngestMode` is `array`, SQL ARRAY types are stored using Druid array columns. This is recommended for new
|
||||
tables.
|
||||
|
||||
When `arrayIngestMode` is `mvd`, SQL `VARCHAR ARRAY` are implicitly wrapped in [`ARRAY_TO_MV`](sql-functions.md#array_to_mv).
|
||||
This causes them to be stored as [multi-value strings](multi-value-dimensions.md), using the same `STRING` column type
|
||||
as regular scalar strings. SQL `BIGINT ARRAY` and `DOUBLE ARRAY` cannot be loaded under `arrayIngestMode: mvd`. This
|
||||
is the default behavior when `arrayIngestMode` is not provided in your query context, although the default behavior
|
||||
may change to `array` in a future release.
|
||||
|
||||
When `arrayIngestMode` is `none`, Druid throws an exception when trying to store any type of arrays. This mode is most
|
||||
useful when set in the system default query context with `druid.query.default.context.arrayIngestMode = none`, in cases
|
||||
where the cluster administrator wants SQL query authors to explicitly provide one or the other in their query context.
|
||||
|
||||
The following table summarizes the differences in SQL ARRAY handling between `arrayIngestMode: array` and
|
||||
`arrayIngestMode: mvd`.
|
||||
|
||||
| SQL type | Stored type when `arrayIngestMode: array` | Stored type when `arrayIngestMode: mvd` (default) |
|
||||
|---|---|---|
|
||||
|`VARCHAR ARRAY`|`ARRAY<STRING>`|[multi-value `STRING`](multi-value-dimensions.md)|
|
||||
|`BIGINT ARRAY`|`ARRAY<LONG>`|not possible (validation error)|
|
||||
|`DOUBLE ARRAY`|`ARRAY<DOUBLE>`|not possible (validation error)|
|
||||
|
||||
In either mode, you can explicitly wrap string arrays in `ARRAY_TO_MV` to cause them to be stored as
|
||||
[multi-value strings](multi-value-dimensions.md).
|
||||
|
||||
When validating a SQL INSERT or REPLACE statement that contains arrays, Druid checks whether the statement would lead
|
||||
to mixing string arrays and multi-value strings in the same column. If this condition is detected, the statement fails
|
||||
validation unless the column is named under the `skipTypeVerification` context parameter. This parameter can be either
|
||||
a comma-separated list of column names, or a JSON array in string form. This validation is done to prevent accidentally
|
||||
mixing arrays and multi-value strings in the same column.
|
||||
Arrays can be inserted with [SQL-based ingestion](../multi-stage-query/index.md).
|
||||
|
||||
#### Examples
|
||||
|
||||
Set [`arrayIngestMode: array`](#arrayingestmode) in your query context to run the following examples.
|
||||
|
||||
```sql
|
||||
REPLACE INTO "array_example" OVERWRITE ALL
|
||||
WITH "ext" AS (
|
||||
|
@ -169,6 +133,35 @@ GROUP BY 1,2,3,4,5
|
|||
PARTITIONED BY DAY
|
||||
```
|
||||
|
||||
#### `arrayIngestMode`
|
||||
|
||||
For seamless backwards compatible behavior with Druid versions older than 31, there is an `arrayIngestMode` query context flag.
|
||||
|
||||
When `arrayIngestMode` is `array`, SQL ARRAY types are stored using Druid array columns. This is recommended for new
|
||||
tables and the default configuration for Druid 31 and newer.
|
||||
|
||||
When `arrayIngestMode` is `mvd` (legacy), SQL `VARCHAR ARRAY` are implicitly wrapped in [`ARRAY_TO_MV`](sql-functions.md#array_to_mv).
|
||||
This causes them to be stored as [multi-value strings](multi-value-dimensions.md), using the same `STRING` column type
|
||||
as regular scalar strings. SQL `BIGINT ARRAY` and `DOUBLE ARRAY` cannot be loaded under `arrayIngestMode: mvd`. This
|
||||
mode is not recommended and will be removed in a future release, but provided for backwards compatibility.
|
||||
|
||||
The following table summarizes the differences in SQL ARRAY handling between `arrayIngestMode: array` and
|
||||
`arrayIngestMode: mvd`.
|
||||
|
||||
| SQL type | Stored type when `arrayIngestMode: array` (default) | Stored type when `arrayIngestMode: mvd` |
|
||||
|---|---|---|
|
||||
|`VARCHAR ARRAY`|`ARRAY<STRING>`|[multi-value `STRING`](multi-value-dimensions.md)|
|
||||
|`BIGINT ARRAY`|`ARRAY<LONG>`|not possible (validation error)|
|
||||
|`DOUBLE ARRAY`|`ARRAY<DOUBLE>`|not possible (validation error)|
|
||||
|
||||
In either mode, you can explicitly wrap string arrays in `ARRAY_TO_MV` to cause them to be stored as
|
||||
[multi-value strings](multi-value-dimensions.md).
|
||||
|
||||
When validating a SQL INSERT or REPLACE statement that contains arrays, Druid checks whether the statement would lead
|
||||
to mixing string arrays and multi-value strings in the same column. If this condition is detected, the statement fails
|
||||
validation unless the column is named under the `skipTypeVerification` context parameter. This parameter can be either
|
||||
a comma-separated list of column names, or a JSON array in string form. This validation is done to prevent accidentally
|
||||
mixing arrays and multi-value strings in the same column.
|
||||
|
||||
## Querying arrays
|
||||
|
||||
|
@ -284,9 +277,9 @@ Avoid confusing string arrays with [multi-value dimensions](multi-value-dimensio
|
|||
|
||||
Use care during ingestion to ensure you get the type you want.
|
||||
|
||||
To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays and set the context parameter `"arrayIngestMode": "array"`. Arrays may contain strings or numbers.
|
||||
To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays. Arrays may contain strings or numbers.
|
||||
|
||||
To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion), which ensures you get multi-value dimensions in any `arrayIngestMode`. Multi-value dimensions can only contain strings.
|
||||
To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion), which ensures you get multi-value dimensions. Multi-value dimensions can only contain strings.
|
||||
|
||||
You can tell which type you have by checking the `INFORMATION_SCHEMA.COLUMNS` table, using a query like:
|
||||
|
||||
|
|
|
@ -50,6 +50,10 @@ Other lookup types are available as extensions, including:
|
|||
- Globally cached lookups from local files, remote URIs, or JDBC through [lookups-cached-global](./lookups-cached-global.md).
|
||||
- Globally cached lookups from a Kafka topic through [kafka-extraction-namespace](./kafka-extraction-namespace.md).
|
||||
|
||||
:::info
|
||||
[Multi-value dimensions](multi-value-dimensions.md) (MVDs) are not supported as keys in lookups. For example, to map the MVD `["A", "B", "C"]` to the value `x` in your lookup, flatten the MVD and map each element of the MVD to the value. Your lookup will have separate key-value pairs for each element of the MVD: `"A": "x"`, `"B": "x"`, and `"C": "x"`.
|
||||
:::
|
||||
|
||||
Query Syntax
|
||||
------------
|
||||
|
||||
|
|
|
@ -507,9 +507,9 @@ Avoid confusing string arrays with [multi-value dimensions](multi-value-dimensio
|
|||
|
||||
Use care during ingestion to ensure you get the type you want.
|
||||
|
||||
To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays and set the context parameter [`"arrayIngestMode": "array"`](arrays.md#arrayingestmode). Arrays may contain strings or numbers.
|
||||
To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays. Arrays may contain strings or numbers.
|
||||
|
||||
To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion), which ensures you get multi-value dimensions in any [`arrayIngestMode`](arrays.md#arrayingestmode). Multi-value dimensions can only contain strings.
|
||||
To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion). Multi-value dimensions can only contain strings.
|
||||
|
||||
You can tell which type you have by checking the `INFORMATION_SCHEMA.COLUMNS` table, using a query like:
|
||||
|
||||
|
|
|
@ -66,7 +66,6 @@ The following are the main parameters for Scan queries:
|
|||
|limit|How many rows to return. If not specified, all rows will be returned.|no|
|
||||
|offset|Skip this many rows when returning results. Skipped rows will still need to be generated internally and then discarded, meaning that raising offsets to high values can cause queries to use additional resources.<br /><br />Together, "limit" and "offset" can be used to implement pagination. However, note that if the underlying datasource is modified in between page fetches in ways that affect overall query results, then the different pages will not necessarily align with each other.|no|
|
||||
|order|The ordering of returned rows based on timestamp. "ascending", "descending", and "none" (default) are supported. Currently, "ascending" and "descending" are only supported for queries where the `__time` column is included in the `columns` field and the requirements outlined in the [time ordering](#time-ordering) section are met.|none|
|
||||
|legacy|Return results consistent with the legacy "scan-query" contrib extension. Defaults to the value set by `druid.query.scan.legacy`, which in turn defaults to false. See [Legacy mode](#legacy-mode) for details.|no|
|
||||
|context|An additional JSON Object which can be used to specify certain flags (see the `query context properties` section below).|no|
|
||||
|
||||
## Example results
|
||||
|
@ -159,14 +158,14 @@ The format of the result when resultFormat equals `compactedList`:
|
|||
|
||||
## Time ordering
|
||||
|
||||
The Scan query currently supports ordering based on timestamp for non-legacy queries. Note that using time ordering
|
||||
will yield results that do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore,
|
||||
time ordering is only supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForOrdering`
|
||||
rows **or** all segments scanned have fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` partitions. Also,
|
||||
time ordering is not supported for queries issued directly to historicals unless a list of segments is specified. The
|
||||
reasoning behind these limitations is that the implementation of time ordering uses two strategies that can consume too
|
||||
much heap memory if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on
|
||||
query result set limit and the number of segments being scanned.
|
||||
The Scan query currently supports ordering based on timestamp. Note that using time ordering will yield results that
|
||||
do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore, time ordering is only
|
||||
supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForOrdering` rows **or** all segments
|
||||
scanned have fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` partitions. Also, time ordering is not
|
||||
supported for queries issued directly to historicals unless a list of segments is specified. The reasoning behind
|
||||
these limitations is that the implementation of time ordering uses two strategies that can consume too much heap memory
|
||||
if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on query result set
|
||||
limit and the number of segments being scanned.
|
||||
|
||||
1. Priority Queue: Each segment on a Historical is opened sequentially. Every row is added to a bounded priority
|
||||
queue which is ordered by timestamp. For every row above the result set limit, the row with the earliest (if descending)
|
||||
|
@ -187,21 +186,6 @@ configurable and can be tuned based on hardware specs and number of dimensions b
|
|||
can also be overridden using the `maxRowsQueuedForOrdering` and `maxSegmentPartitionsOrderedInMemory` properties in
|
||||
the query context (see the Query Context Properties section).
|
||||
|
||||
## Legacy mode
|
||||
|
||||
The Scan query supports a legacy mode designed for protocol compatibility with the former scan-query contrib extension.
|
||||
In legacy mode you can expect the following behavior changes:
|
||||
|
||||
- The `__time` column is returned as `"timestamp"` rather than `"__time"`. This will take precedence over any other column
|
||||
you may have that is named `"timestamp"`.
|
||||
- The `__time` column is included in the list of columns even if you do not specifically ask for it.
|
||||
- Timestamps are returned as ISO8601 time strings rather than integers (milliseconds since 1970-01-01 00:00:00 UTC).
|
||||
|
||||
Legacy mode can be triggered either by passing `"legacy" : true` in your query JSON, or by setting
|
||||
`druid.query.scan.legacy = true` on your Druid processes. If you were previously using the scan-query contrib extension,
|
||||
the best way to migrate is to activate legacy mode during a rolling upgrade, then switch it off after the upgrade
|
||||
is complete.
|
||||
|
||||
## Configuration Properties
|
||||
|
||||
Configuration properties:
|
||||
|
@ -210,7 +194,6 @@ Configuration properties:
|
|||
|--------|-----------|------|-------|
|
||||
|druid.query.scan.maxRowsQueuedForOrdering|The maximum number of rows returned when time ordering is used|An integer in [1, 2147483647]|100000|
|
||||
|druid.query.scan.maxSegmentPartitionsOrderedInMemory|The maximum number of segments scanned per historical when time ordering is used|An integer in [1, 2147483647]|50|
|
||||
|druid.query.scan.legacy|Whether legacy mode should be turned on for Scan queries|true or false|false|
|
||||
|
||||
|
||||
## Query context properties
|
||||
|
@ -228,3 +211,7 @@ Sample query context JSON object:
|
|||
"maxSegmentPartitionsOrderedInMemory": 100
|
||||
}
|
||||
```
|
||||
|
||||
## Legacy mode
|
||||
|
||||
In older versions of Druid, the scan query supported a legacy mode designed for protocol compatibility with the former scan-query contrib extension from versions of Druid older than 0.11. This mode has been removed.
|
||||
|
|
|
@ -27,26 +27,69 @@ sidebar_label: "All functions"
|
|||
Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
|
||||
This document describes the SQL language.
|
||||
:::
|
||||
<!-- The **Learn More** at the end of each function section provides further documentation. -->
|
||||
This page provides a reference of Apache Druid® SQL functions in alphabetical order. For more details on a function, refer to the following:
|
||||
* [Aggregation functions](sql-aggregations.md)
|
||||
* [Array functions](sql-array-functions.md)
|
||||
* [JSON functions](sql-json-functions.md)
|
||||
* [Multi-value string functions](sql-multivalue-string-functions.md)
|
||||
* [Scalar functions](sql-scalar.md)
|
||||
* [Window functions](sql-window-functions.md)
|
||||
|
||||
|
||||
This page provides a reference of all Druid SQL functions in alphabetical order.
|
||||
Click the linked function type for documentation on a particular function.
|
||||
The examples on this page use the following example datasources:
|
||||
* `flight-carriers` using `FlightCarrierOnTime (1 month)`
|
||||
* `taxi-trips` using `NYC Taxi cabs (3 files)`
|
||||
|
||||
## ABS
|
||||
|
||||
`ABS(<NUMERIC>)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Calculates the absolute value of a numeric expression.
|
||||
|
||||
* **Syntax:** `ABS(<NUMERIC>)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example applies the ABS function to the `ArrDelay` column from the `flight-carriers` datasource.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"ArrDelay" AS "arrival_delay",
|
||||
ABS("ArrDelay") AS "absolute_arrival_delay"
|
||||
FROM "flight-carriers"
|
||||
WHERE "ArrDelay" < 0
|
||||
LIMIT 1
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `arrival_delay` | `absolute_arrival_delay` |
|
||||
| -- | -- |
|
||||
| `-27` | `27` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## ACOS
|
||||
|
||||
`ACOS(<NUMERIC>)`
|
||||
Calculates the arc cosine (arccosine) of a numeric expression.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `ACOS(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Calculates the arc cosine of a numeric expression.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example calculates the arc cosine of `0`.
|
||||
|
||||
```sql
|
||||
SELECT ACOS(0) AS "arc_cosine"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `arc_cosine` |
|
||||
| -- |
|
||||
| `1.5707963267948966` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## ANY_VALUE
|
||||
|
||||
|
@ -64,6 +107,7 @@ Returns any value of the specified expression.
|
|||
|
||||
Counts distinct values of a regular column or a prebuilt sketch column.
|
||||
|
||||
## APPROX_COUNT_DISTINCT_BUILTIN
|
||||
`APPROX_COUNT_DISTINCT_BUILTIN(expr)`
|
||||
|
||||
**Function type:** [Aggregation](sql-aggregations.md)
|
||||
|
@ -251,27 +295,72 @@ Joins all elements of `arr` by the delimiter specified by `str`.
|
|||
|
||||
## ASIN
|
||||
|
||||
`ASIN(<NUMERIC>)`
|
||||
Calculates the arc sine (arcsine) of a numeric expression.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `ASIN(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Calculates the arc sine of a numeric expression.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example calculates the arc sine of `1`.
|
||||
|
||||
```sql
|
||||
SELECT ASIN(1) AS "arc_sine"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `arc_sine` |
|
||||
| -- |
|
||||
| `1.5707963267948966` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## ATAN
|
||||
|
||||
`ATAN(<NUMERIC>)`
|
||||
Calculates the arc tangent (arctangent) of a numeric expression.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `ATAN(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Calculates the arc tangent of a numeric expression.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example calculates the arc tangent of `1`.
|
||||
|
||||
```sql
|
||||
SELECT ATAN(1) AS "arc_tangent"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `arc_tangent` |
|
||||
| -- |
|
||||
| `0.7853981633974483` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## ATAN2
|
||||
|
||||
`ATAN2(<NUMERIC>, <NUMERIC>)`
|
||||
Calculates the arc tangent (arctangent) of a specified x and y coordinate.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `ATAN2(x, y)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Calculates the arc tangent of the two arguments.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example calculates the arc tangent of the coordinate `(1, -1)`
|
||||
|
||||
```sql
|
||||
SELECT ATAN2(1,-1) AS "arc_tangent_2"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `arc_tangent_2` |
|
||||
| -- |
|
||||
| `2.356194490192345` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## AVG
|
||||
|
||||
|
@ -419,11 +508,29 @@ Rounds up a timestamp by a given time unit.
|
|||
|
||||
## CEIL (numeric)
|
||||
|
||||
`CEIL(<NUMERIC>)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Calculates the smallest integer value greater than or equal to the numeric expression.
|
||||
* **Syntax:** `CEIL(<NUMERIC>)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example applies the CEIL function to the `fare_amount` column from the `taxi-trips` datasource.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"fare_amount" AS "fare_amount",
|
||||
CEIL("fare_amount") AS "ceiling_fare_amount"
|
||||
FROM "taxi-trips"
|
||||
LIMIT 1
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `fare_amount` | `ceiling_fare_amount` |
|
||||
| -- | -- |
|
||||
| `21.25` | `22` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## CHAR_LENGTH
|
||||
|
||||
|
@ -467,20 +574,50 @@ Finds whether a string is in a given expression, case-sensitive.
|
|||
|
||||
## COS
|
||||
|
||||
`COS(<NUMERIC>)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Calculates the trigonometric cosine of an angle expressed in radians.
|
||||
|
||||
* **Syntax:** `COS(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example calculates the cosine of angle `PI/3` radians.
|
||||
|
||||
```sql
|
||||
SELECT COS(PI / 3) AS "cosine"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `cosine` |
|
||||
| -- |
|
||||
| `0.5000000000000001` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## COT
|
||||
|
||||
`COT(<NUMERIC>)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Calculates the trigonometric cotangent of an angle expressed in radians.
|
||||
|
||||
* **Syntax:** `COT(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example calculates the cotangent of angle `PI/3` radians.
|
||||
|
||||
```sql
|
||||
SELECT COT(PI / 3) AS "cotangent"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `cotangent` |
|
||||
| -- |
|
||||
| `0.577350269189626` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## COUNT
|
||||
|
||||
`COUNT([DISTINCT] expr)`
|
||||
|
@ -542,12 +679,27 @@ Decodes a Base64-encoded string into a UTF-8 encoded string.
|
|||
|
||||
## DEGREES
|
||||
|
||||
`DEGREES(<NUMERIC>)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Converts an angle from radians to degrees.
|
||||
|
||||
* **Syntax:** `DEGREES(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example converts an angle of `PI` radians to degrees
|
||||
|
||||
```sql
|
||||
SELECT DEGREES(PI) AS "degrees"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `degrees` |
|
||||
| -- |
|
||||
| `180` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## DENSE_RANK
|
||||
|
||||
`DENSE_RANK()`
|
||||
|
@ -697,12 +849,27 @@ Returns the value of a numeric or string expression corresponding to the earlies
|
|||
|
||||
## EXP
|
||||
|
||||
`EXP(<NUMERIC>)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Calculates _e_ raised to the power of the numeric expression.
|
||||
|
||||
* **Syntax:** `EXP(<NUMERIC>)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example calculates _e_ to the power of 1.
|
||||
|
||||
```sql
|
||||
SELECT EXP(1) AS "exponential"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `exponential` |
|
||||
| -- |
|
||||
| `2.7182818284590455` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## EXTRACT
|
||||
|
||||
`EXTRACT(<TIME_UNIT> FROM <TIMESTAMP>)`
|
||||
|
@ -729,11 +896,30 @@ Rounds down a timestamp by a given time unit.
|
|||
|
||||
## FLOOR (numeric)
|
||||
|
||||
`FLOOR(<NUMERIC>)`
|
||||
Calculates the largest integer less than or equal to the numeric expression.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `FLOOR(<NUMERIC>)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Calculates the largest integer value less than or equal to the numeric expression.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example applies the FLOOR function to the `fare_amount` column from the `taxi-trips` datasource.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"fare_amount" AS "fare_amount",
|
||||
FLOOR("fare_amount") AS "floor_fare_amount"
|
||||
FROM "taxi-trips"
|
||||
LIMIT 1
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `fare_amount` | `floor_fare_amount` |
|
||||
| -- | -- |
|
||||
| `21.25` | `21` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## GREATEST
|
||||
|
||||
|
@ -961,19 +1147,57 @@ Returns the length of the expression in UTF-16 encoding.
|
|||
|
||||
## LN
|
||||
|
||||
`LN(expr)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Calculates the natural logarithm of the numeric expression.
|
||||
|
||||
* **Syntax:** `LN(<NUMERIC>)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example applies the LN function to the `max_temperature` column from the `taxi-trips` datasource.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"max_temperature" AS "max_temperature",
|
||||
LN("max_temperature") AS "natural_log_max_temp"
|
||||
FROM "taxi-trips"
|
||||
LIMIT 1
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `max_temperature` | `natural_log_max_temp` |
|
||||
| -- | -- |
|
||||
| `76` | `4.330733340286331` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## LOG10
|
||||
|
||||
`LOG10(expr)`
|
||||
Calculates the base-10 logarithm of the numeric expression.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `LOG10(<NUMERIC>)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Calculates the base-10 of the numeric expression.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example applies the LOG10 function to the `max_temperature` column from the `taxi-trips` datasource.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"max_temperature" AS "max_temperature",
|
||||
LOG10("max_temperature") AS "log10_max_temp"
|
||||
FROM "taxi-trips"
|
||||
LIMIT 1
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `max_temperature` | `log10_max_temp` |
|
||||
| -- | -- |
|
||||
| `76` | `1.8808135922807914` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## LOOKUP
|
||||
|
||||
|
@ -1033,11 +1257,26 @@ Returns the minimum value of a set of values.
|
|||
|
||||
## MOD
|
||||
|
||||
`MOD(x, y)`
|
||||
Calculates x modulo y, or the remainder of x divided by y. Where x and y are numeric expressions.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `MOD(x, y)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Calculates x modulo y, or the remainder of x divided by y.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following calculates 78 MOD 10.
|
||||
|
||||
```sql
|
||||
SELECT MOD(78, 10) as "modulo"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `modulo` |
|
||||
| -- |
|
||||
| `8` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## MV_APPEND
|
||||
|
||||
|
@ -1217,20 +1456,50 @@ Returns the one-based index position of a substring within an expression, option
|
|||
|
||||
## POWER
|
||||
|
||||
`POWER(expr, power)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Calculates a numerical expression raised to the specified power.
|
||||
|
||||
* **Syntax:** `POWER(base, exponent)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example raises 5 to the power of 2.
|
||||
|
||||
```sql
|
||||
SELECT POWER(5, 2) AS "power"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `power` |
|
||||
| -- |
|
||||
| `25` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## RADIANS
|
||||
|
||||
`RADIANS(expr)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Converts an angle from degrees to radians.
|
||||
|
||||
* **Syntax:** `RADIANS(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example converts an angle of `180` degrees to radians
|
||||
|
||||
```sql
|
||||
SELECT RADIANS(180) AS "radians"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `radians` |
|
||||
| -- |
|
||||
| `3.141592653589793` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## RANK
|
||||
|
||||
`RANK()`
|
||||
|
@ -1298,12 +1567,32 @@ Returns the rightmost number of characters from an expression.
|
|||
|
||||
## ROUND
|
||||
|
||||
`ROUND(expr[, digits])`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Calculates the rounded value for a numerical expression.
|
||||
|
||||
* **Syntax:** `ROUND(expr[, digits])`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following applies the ROUND function to 0 decimal points on the `pickup_longitude` column from the `taxi-trips` datasource.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"pickup_longitude" AS "pickup_longitude",
|
||||
ROUND("pickup_longitude", 0) as "rounded_pickup_longitude"
|
||||
FROM "taxi-trips"
|
||||
WHERE "pickup_longitude" IS NOT NULL
|
||||
LIMIT 1
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `pickup_longitude` | `rounded_pickup_longitude` |
|
||||
| -- | -- |
|
||||
| `-73.9377670288086` | `-74` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## ROW_NUMBER
|
||||
|
||||
`ROW_NUMBER()`
|
||||
|
@ -1338,20 +1627,50 @@ Returns `x` divided by `y`, guarded on division by 0.
|
|||
|
||||
## SIN
|
||||
|
||||
`SIN(expr)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Calculates the trigonometric sine of an angle expressed in radians.
|
||||
|
||||
* **Syntax:** `SIN(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example calculates the sine of angle `PI/3` radians.
|
||||
|
||||
```sql
|
||||
SELECT SIN(PI / 3) AS "sine"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `sine` |
|
||||
| -- |
|
||||
| `0.8660254037844386` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## SQRT
|
||||
|
||||
`SQRT(expr)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Calculates the square root of a numeric expression.
|
||||
|
||||
* **Syntax:** `SQRT(<NUMERIC>)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example calculates the square root of 25.
|
||||
|
||||
```sql
|
||||
SELECT SQRT(25) AS "square_root"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `square_root` |
|
||||
| -- |
|
||||
| `5` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## STDDEV
|
||||
|
||||
`STDDEV(expr)`
|
||||
|
@ -1451,12 +1770,27 @@ Calculates the sum of a set of values.
|
|||
|
||||
## TAN
|
||||
|
||||
`TAN(expr)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Calculates the trigonometric tangent of an angle expressed in radians.
|
||||
|
||||
* **Syntax:** `TAN(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example calculates the tangent of angle `PI/3` radians.
|
||||
|
||||
```sql
|
||||
SELECT TAN(PI / 3) AS "tangent"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `tangent` |
|
||||
| -- |
|
||||
| `1.7320508075688767` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## TDIGEST_GENERATE_SKETCH
|
||||
|
||||
`TDIGEST_GENERATE_SKETCH(expr, [compression])`
|
||||
|
@ -1620,20 +1954,41 @@ Trims the leading or trailing characters of an expression.
|
|||
|
||||
## TRUNC
|
||||
|
||||
`TRUNC(expr[, digits])`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Alias for [`TRUNCATE`](#truncate).
|
||||
|
||||
* **Syntax:** `TRUNC(expr[, digits])`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## TRUNCATE
|
||||
|
||||
`TRUNCATE(expr[, digits])`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Truncates a numerical expression to a specific number of decimal digits.
|
||||
|
||||
* **Syntax:** `TRUNCATE(expr[, digits])`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following applies the TRUNCATE function to 1 decimal place on the `pickup_longitude` column from the `taxi-trips` datasource.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"pickup_longitude" as "pickup_longitude",
|
||||
TRUNCATE("pickup_longitude", 1) as "truncate_pickup_longitude"
|
||||
FROM "taxi-trips"
|
||||
WHERE "pickup_longitude" IS NOT NULL
|
||||
LIMIT 1
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `pickup_longitude` | `truncate_pickup_longitude` |
|
||||
| -- | -- |
|
||||
| `-73.9377670288086` | `-73.9` |
|
||||
</details>
|
||||
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## TRY_PARSE_JSON
|
||||
|
||||
|
@ -1683,3 +2038,4 @@ Calculates the sample variance of a set of values.
|
|||
|
||||
Alias for [`VAR_SAMP`](#var_samp).
|
||||
|
||||
|
||||
|
|
|
@ -342,7 +342,6 @@ The above EXPLAIN PLAN returns the following result:
|
|||
"regionName",
|
||||
"v0"
|
||||
],
|
||||
"legacy": false,
|
||||
"context": {
|
||||
"finalizeAggregations": false,
|
||||
"forceExpressionVirtualColumns": true,
|
||||
|
@ -562,7 +561,6 @@ The above EXPLAIN PLAN query returns the following result:
|
|||
"regionName",
|
||||
"v0"
|
||||
],
|
||||
"legacy": false,
|
||||
"context": {
|
||||
"finalizeAggregations": false,
|
||||
"groupByEnableMultiValueUnnesting": false,
|
||||
|
|
|
@ -246,11 +246,8 @@ Druid has guardrail logic to prevent you from executing window function queries
|
|||
|
||||
For example:
|
||||
- You cannot set expressions as bounds for window frames.
|
||||
- You cannot use two FOLLOWING expressions in the window frame. For example: `ROWS BETWEEN 2 ROWS FOLLOWING and 3 ROWS FOLLOWING`.
|
||||
- You can only use a RANGE frames when both endpoints are unbounded or current row.
|
||||
|
||||
If you write a query that violates one of these conditions, Druid throws an error: "The query contains a window frame which may return incorrect results. To disregard this warning, set `windowingStrictValidation` to false in the query context."
|
||||
|
||||
## Window function reference
|
||||
|
||||
|Function|Notes|
|
||||
|
|
|
@ -0,0 +1,386 @@
|
|||
---
|
||||
id: migr-ansi-sql-null
|
||||
title: "Migration guide: SQL compliant mode"
|
||||
sidebar_label: SQL compliant mode
|
||||
---
|
||||
|
||||
<!--
|
||||
~ Licensed to the Apache Software Foundation (ASF) under one
|
||||
~ or more contributor license agreements. See the NOTICE file
|
||||
~ distributed with this work for additional information
|
||||
~ regarding copyright ownership. The ASF licenses this file
|
||||
~ to you under the Apache License, Version 2.0 (the
|
||||
~ "License"); you may not use this file except in compliance
|
||||
~ with the License. You may obtain a copy of the License at
|
||||
~
|
||||
~ http://www.apache.org/licenses/LICENSE-2.0
|
||||
~
|
||||
~ Unless required by applicable law or agreed to in writing,
|
||||
~ software distributed under the License is distributed on an
|
||||
~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
~ KIND, either express or implied. See the License for the
|
||||
~ specific language governing permissions and limitations
|
||||
~ under the License.
|
||||
-->
|
||||
import Tabs from '@theme/Tabs';
|
||||
import TabItem from '@theme/TabItem';
|
||||
|
||||
In Apache Druid 28.0.0, the default [null handling](../querying/sql-data-types.md#null-values) mode changed to be compliant with the ANSI SQL standard.
|
||||
This guide provides strategies for Druid operators who rely on legacy Druid null handling behavior in their applications to transition to SQL compliant mode.
|
||||
Legacy mode is planned for removal from Druid.
|
||||
|
||||
## SQL compliant null handling
|
||||
|
||||
As of Druid 28.0.0, Druid writes segments in an ANSI SQL compatible null handling mode by default.
|
||||
This means that Druid stores null values distinctly from empty strings for string dimensions and distinctly from 0 for numeric dimensions.
|
||||
|
||||
This can impact your application behavior because the ANSI SQL standard defines any comparison to null to be unknown.
|
||||
According to this three-valued logic, `x <> 'some value'` only returns non-null values.
|
||||
|
||||
The default Druid configurations for 28.0.0 and later that enable ANSI SQL compatible null handling mode are the following:
|
||||
|
||||
* `druid.generic.useDefaultValueForNull=false`
|
||||
* `druid.expressions.useStrictBooleans=true`
|
||||
* `druid.generic.useThreeValueLogicForNativeFilters=true`
|
||||
|
||||
Follow the [Null handling tutorial](../tutorials/tutorial-sql-null.md) to learn how the default null handling works in Druid.
|
||||
|
||||
## Legacy null handling and two-valued logic
|
||||
|
||||
Prior to Druid 28.0.0, Druid defaulted to a legacy mode which stored default values instead of nulls.
|
||||
In legacy mode, Druid created segments with the following characteristics at ingestion time:
|
||||
|
||||
- String columns couldn't distinguish an empty string, `''`, from null.
|
||||
Therefore, Druid treated them both as interchangeable values.
|
||||
- Numeric columns couldn't represent null valued rows.
|
||||
Therefore, Druid stored `0` instead of `null`.
|
||||
|
||||
The Druid configurations for the deprecated legacy mode were the following:
|
||||
|
||||
* `druid.generic.useDefaultValueForNull=true`
|
||||
* `druid.expressions.useStrictBooleans=false`
|
||||
* `druid.generic.useThreeValueLogicForNativeFilters=true`
|
||||
|
||||
These configurations are deprecated and scheduled for removal.
|
||||
After the configurations are removed, Druid will ignore them if they exist in your configuration files and use the default SQL compliant mode.
|
||||
|
||||
## Migrate to SQL compliant mode
|
||||
|
||||
If your business logic relies on the behavior of legacy mode, you have the following options to operate Druid in an ANSI SQL compatible null handling mode:
|
||||
|
||||
- Modify incoming data to either [avoid nulls](#replace-null-values-at-ingestion-time) or [avoid empty strings](#coerce-empty-strings-to-null-at-ingestion-time) to achieve the same query behavior as legacy mode. This means modifying your ingestion SQL queries and ingestion specs to handle nulls or empty strings.
|
||||
For example, replacing a null for a string column with an empty string or a 0 for a numeric column.
|
||||
However, it means that your existing queries should operate as if Druid were in legacy mode.
|
||||
If you do not care about preserving null values, this is a good option for you.
|
||||
|
||||
- Preserve null values and [update all of your SQL queries to be ANSI SQL compliant](#rewrite-your-queries-to-be-sql-compliant).
|
||||
This means you can preserve the incoming data with nulls intact.
|
||||
However, you must rewrite any affected client-side queries to be ANSI SQL compliant.
|
||||
If you have a requirement to preserve null values, choose this option.
|
||||
|
||||
### Replace null values at ingestion time
|
||||
|
||||
If you don't need to preserve null values within Druid, you can use a transform at ingestion time to replace nulls with other values.
|
||||
|
||||
Consider the following input data:
|
||||
|
||||
```json
|
||||
{"time":"2024-01-01T00:00:00.000Z","string_example":"my_string","number_example":99}
|
||||
{"time":"2024-01-02T00:00:00.000Z","string_example":"","number_example":0}
|
||||
{"time":"2024-01-03T00:00:00.000Z","string_example":null,"number_example":null}
|
||||
```
|
||||
|
||||
The following example illustrates how to use COALESCE and NVL at ingestion time to avoid null values in Druid:
|
||||
|
||||
<Tabs>
|
||||
|
||||
<TabItem value="0" label="SQL-based batch">
|
||||
|
||||
```sql
|
||||
REPLACE INTO "no_nulls_example" OVERWRITE ALL
|
||||
WITH "ext" AS (
|
||||
SELECT *
|
||||
FROM TABLE(
|
||||
EXTERN(
|
||||
'{"type":"inline","data":"{\"time\":\"2024-01-01T00:00:00.000Z\",\"string_example\":\"my_string\",\"number_example\":99}\n{\"time\":\"2024-01-02T00:00:00.000Z\",\"string_example\":\"\",\"number_example\":0}\n{\"time\":\"2024-01-03T00:00:00.000Z\",\"string_example\":null,\"number_example\":null}"}',
|
||||
'{"type":"json"}'
|
||||
)
|
||||
) EXTEND ("time" VARCHAR, "string_example" VARCHAR, "number_example" BIGINT)
|
||||
)
|
||||
SELECT
|
||||
TIME_PARSE("time") AS "__time",
|
||||
-- Replace any null string values with an empty string
|
||||
COALESCE("string_example",'') AS string_example,
|
||||
-- Replace any null numeric values with 0
|
||||
NVL("number_example",0) AS number_example
|
||||
FROM "ext"
|
||||
PARTITIONED BY MONTH
|
||||
```
|
||||
</TabItem>
|
||||
|
||||
<TabItem value="1" label="JSON-based batch">
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "index_parallel",
|
||||
"spec": {
|
||||
"ioConfig": {
|
||||
"type": "index_parallel",
|
||||
"inputSource": {
|
||||
"type": "inline",
|
||||
"data": "{\"time\":\"2024-01-01T00:00:00.000Z\",\"string_example\":\"my_string\",\"number_example\":99}\n{\"time\":\"2024-01-02T00:00:00.000Z\",\"string_example\":\"\",\"number_example\":0}\n{\"time\":\"2024-01-03T00:00:00.000Z\",\"string_example\":null,\"number_example\":null}"
|
||||
},
|
||||
"inputFormat": {
|
||||
"type": "json"
|
||||
}
|
||||
},
|
||||
"tuningConfig": {
|
||||
"type": "index_parallel",
|
||||
"partitionsSpec": {
|
||||
"type": "dynamic"
|
||||
}
|
||||
},
|
||||
"dataSchema": {
|
||||
"dataSource": "inline_data_native",
|
||||
"timestampSpec": {
|
||||
"column": "time",
|
||||
"format": "iso"
|
||||
},
|
||||
"dimensionsSpec": {
|
||||
"dimensions": [
|
||||
"string_example",
|
||||
{
|
||||
"type": "long",
|
||||
"name": "number_example"
|
||||
}
|
||||
]
|
||||
},
|
||||
"granularitySpec": {
|
||||
"queryGranularity": "none",
|
||||
"rollup": false,
|
||||
"segmentGranularity": "MONTH"
|
||||
},
|
||||
"transformSpec": {
|
||||
"transforms": [
|
||||
{
|
||||
"type": "expression",
|
||||
"name": "string_example",
|
||||
"expression": "COALESCE(\"string_example\",'')"
|
||||
},
|
||||
{
|
||||
"type": "expression",
|
||||
"name": "number_example",
|
||||
"expression": "NVL(\"number_example\",0)"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
</TabItem>
|
||||
</Tabs>
|
||||
|
||||
Druid ingests the data with no null values as follows:
|
||||
|
||||
| `__time` | `string_examle` | `number_example`|
|
||||
| -- | -- | -- |
|
||||
| `2024-01-01T00:00:00.000Z`| `my_string`| 99 |
|
||||
| `2024-01-02T00:00:00.000Z`| `empty`| 0 |
|
||||
| `2024-01-03T00:00:00.000Z`| `empty`| 0 |
|
||||
|
||||
### Coerce empty strings to null at ingestion time
|
||||
|
||||
In legacy mode, Druid recognized empty strings as nulls for equality comparison.
|
||||
If your queries rely on empty strings to represent nulls, you can coerce empty strings to null at ingestion time using NULLIF.
|
||||
|
||||
For example, consider the following sample input data:
|
||||
|
||||
```json
|
||||
{"time":"2024-01-01T00:00:00.000Z","string_example":"my_string"}
|
||||
{"time":"2024-01-02T00:00:00.000Z","string_example":""}
|
||||
{"time":"2024-01-03T00:00:00.000Z","string_example":null}
|
||||
```
|
||||
|
||||
In legacy mode, Druid wrote an empty string for the third record.
|
||||
Therefore the following query returned 2:
|
||||
|
||||
```sql
|
||||
SELECT count(*)
|
||||
FROM "null_string"
|
||||
WHERE "string_example" IS NULL
|
||||
```
|
||||
|
||||
In SQL compliant mode, Druid differentiates between empty strings and nulls, so the same query would return 1.
|
||||
The following example shows how to coerce empty strings into null to accommodate IS NULL comparisons:
|
||||
|
||||
<Tabs>
|
||||
|
||||
<TabItem value="0" label="SQL-based batcn">
|
||||
|
||||
```sql
|
||||
REPLACE INTO "null_string" OVERWRITE ALL
|
||||
WITH "ext" AS (
|
||||
SELECT *
|
||||
FROM TABLE(
|
||||
EXTERN(
|
||||
'{"type":"inline","data":"{\"time\":\"2024-01-01T00:00:00.000Z\",\"string_example\":\"my_string\"}\n{\"time\":\"2024-01-02T00:00:00.000Z\",\"string_example\":\"\"}\n{\"time\":\"2024-01-03T00:00:00.000Z\",\"string_example\":null}"}',
|
||||
'{"type":"json"}'
|
||||
)
|
||||
) EXTEND ("time" VARCHAR, "string_example" VARCHAR)
|
||||
)
|
||||
SELECT
|
||||
TIME_PARSE("time") AS "__time",
|
||||
NULLIF("string_example",'') AS "string_example"
|
||||
FROM "ext"
|
||||
PARTITIONED BY MONTH
|
||||
```
|
||||
|
||||
</TabItem>
|
||||
|
||||
<TabItem value="1" label="JSON-based batch">
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "index_parallel",
|
||||
"spec": {
|
||||
"ioConfig": {
|
||||
"type": "index_parallel",
|
||||
"inputSource": {
|
||||
"type": "inline",
|
||||
"data": "{\"time\":\"2024-01-01T00:00:00.000Z\",\"string_example\":\"my_string\"}\n{\"time\":\"2024-01-02T00:00:00.000Z\",\"string_example\":\"\"}\n{\"time\":\"2024-01-03T00:00:00.000Z\",\"string_example\":null}"
|
||||
},
|
||||
"inputFormat": {
|
||||
"type": "json"
|
||||
}
|
||||
},
|
||||
"tuningConfig": {
|
||||
"type": "index_parallel",
|
||||
"partitionsSpec": {
|
||||
"type": "dynamic"
|
||||
}
|
||||
},
|
||||
"dataSchema": {
|
||||
"dataSource": "null_string",
|
||||
"timestampSpec": {
|
||||
"column": "time",
|
||||
"format": "iso"
|
||||
},
|
||||
"transformSpec": {
|
||||
"transforms": [
|
||||
{
|
||||
"type": "expression",
|
||||
"expression": "case_searched((\"string_example\" == ''),null,\"string_example\")",
|
||||
"name": "string_example"
|
||||
}
|
||||
]
|
||||
},
|
||||
"dimensionsSpec": {
|
||||
"dimensions": [
|
||||
"string_example"
|
||||
]
|
||||
},
|
||||
"granularitySpec": {
|
||||
"queryGranularity": "none",
|
||||
"rollup": false,
|
||||
"segmentGranularity": "month"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
</TabItem>
|
||||
</Tabs>
|
||||
|
||||
Druid ingests the data with no empty strings as follows:
|
||||
|
||||
| `__time` | `string_examle` |
|
||||
| -- | -- | -- |
|
||||
| `2024-01-01T00:00:00.000Z`| `my_string`|
|
||||
| `2024-01-02T00:00:00.000Z`| `null`|
|
||||
| `2024-01-03T00:00:00.000Z`| `null`|
|
||||
|
||||
Therefore `SELECT count(*) FROM "null_string" WHERE "string_example" IS NULL` returns 2.
|
||||
|
||||
### Rewrite your queries to be SQL compliant
|
||||
|
||||
If you want to maintain null values in your data within Druid, you can use the following ANSI SQL compliant querying strategies to achieve the same results as legacy null handling:
|
||||
|
||||
- Modify inequality queries to include null values.
|
||||
For example, `x <> 'some value'` becomes `(x <> 'some value' OR x IS NULL)`.
|
||||
- Use COALESCE or NVL to replace nulls with a value.
|
||||
For example, `x + 1` becomes `NVL(numeric_value, 0)+1`
|
||||
|
||||
Consider the following Druid datasource `null_example`:
|
||||
|
||||
| `__time` | `string_examle` | `number_example`|
|
||||
| -- | -- | -- |
|
||||
| `2024-01-01T00:00:00.000Z`| `my_string`| 99 |
|
||||
| `2024-01-02T00:00:00.000Z`| `empty`| 0 |
|
||||
| `2024-01-03T00:00:00.000Z`| `null`| null |
|
||||
|
||||
Druid excludes null strings from equality comparisons. For example:
|
||||
|
||||
```sql
|
||||
SELECT COUNT(*) AS count_example
|
||||
FROM "null_example"
|
||||
WHERE "string_example"<> 'my_string'
|
||||
```
|
||||
|
||||
Druid returns 1 because null is considered unknown: neither equal nor unequal to the value.
|
||||
|
||||
To count null values in the result, you can use an OR operator:
|
||||
|
||||
```sql
|
||||
SELECT COUNT(*) AS count_example
|
||||
FROM "null_example"
|
||||
WHERE ("string_example"<> 'my_string') OR "string_example" IS NULL
|
||||
```
|
||||
|
||||
Druid returns 2.
|
||||
To achieve the same result, you can use IS DISTINCT FROM for null-safe comparison:
|
||||
|
||||
```sql
|
||||
SELECT COUNT(*) as count_example
|
||||
FROM "null_example"
|
||||
WHERE "string_example" IS DISTINCT FROM 'my_string'
|
||||
```
|
||||
|
||||
Similarly, arithmetic operators on null return null. For example:
|
||||
|
||||
```sql
|
||||
SELECT "number_example" + 1 AS additon_example
|
||||
FROM "null_example"
|
||||
```
|
||||
|
||||
Druid returns the following because null + any value is null for the ANSI SQL standard:
|
||||
|
||||
| `addition_example`|
|
||||
| -- |
|
||||
| 100 |
|
||||
| 1 |
|
||||
| null |
|
||||
|
||||
Use NVL to avoid nulls with arithmetic. For example:
|
||||
|
||||
```sql
|
||||
SELECT NVL("number_example",0) + 1 AS additon_example
|
||||
FROM "null_example"
|
||||
```
|
||||
|
||||
Druid returns the following:
|
||||
|
||||
| `addition_example` |
|
||||
| -- |
|
||||
| 100 |
|
||||
| 1 |
|
||||
| null |
|
||||
|
||||
## Learn more
|
||||
|
||||
See the following topics for more information:
|
||||
- [Null handling tutorial](../tutorials/tutorial-sql-null.md) to learn how the default null handling works in Druid.
|
||||
- [Null values](../querying/sql-data-types.md#null-values) for a description of Druid's behavior with null values.
|
||||
- [Handling null values](../design/segments.md#handling-null-values) for details about how Druid stores null values.
|
|
@ -30,12 +30,18 @@ The guides in this section outline breaking changes introduced in Druid 25.0.0 a
|
|||
|
||||
## Migrate from multi-value dimensions to arrays
|
||||
|
||||
Druid now supports SQL-compliant array types. Whenever possible, you should use the array type over multi-value dimensions. See [Migration guide: MVDs to arrays](migr-mvd-array.md).
|
||||
Druid now supports SQL-compliant array types. Whenever possible, you should use the array type over multi-value dimensions. See [Migration guide: MVDs to arrays](./migr-mvd-array.md).
|
||||
|
||||
## Migrate to front-coded dictionary encoding
|
||||
|
||||
Druid encodes string columns into dictionaries for better compression. Front-coded dictionary encoding reduces storage and improves performance by optimizing for strings that share similar beginning substrings. See [Migration guide: front-coded dictionaries](migr-front-coded-dict.md) for more information.
|
||||
Druid encodes string columns into dictionaries for better compression. Front-coded dictionary encoding reduces storage and improves performance by optimizing for strings that share similar beginning substrings. See [Migration guide: front-coded dictionaries](./migr-front-coded-dict.md) for more information.
|
||||
|
||||
## Migrate from `maxSubqueryRows` to `maxSubqueryBytes`
|
||||
|
||||
Druid allows you to set a byte-based limit on subquery size to prevent Brokers from running out of memory when handling large subqueries. The byte-based subquery limit overrides Druid's row-based subquery limit. We recommend that you move towards using byte-based limits starting in Druid 30.0.0. See [Migration guide: subquery limit](migr-subquery-limit.md) for more information.
|
||||
Druid allows you to set a byte-based limit on subquery size to prevent Brokers from running out of memory when handling large subqueries. The byte-based subquery limit overrides Druid's row-based subquery limit. We recommend that you move towards using byte-based limits starting in Druid 30.0.0. See [Migration guide: subquery limit](./migr-subquery-limit.md) for more information.
|
||||
|
||||
## Migrate to SQL compliant null handling mode
|
||||
|
||||
By default, the Druid [null handling](../querying/sql-data-types.md#null-values) mode is now compliant with ANSI SQL.
|
||||
This guide provides strategies for Druid operators and users who rely on the legacy Druid null handling behavior in their applications to transition to ANSI SQL compliant mode. See [Migration guide: SQL compliant mode](./migr-ansi-sql-null.md
|
||||
) for more information.
|
|
@ -24,18 +24,21 @@ sidebar_label: Aggregate data with rollup
|
|||
-->
|
||||
|
||||
|
||||
Apache Druid can summarize raw data at ingestion time using a process we refer to as "rollup". Rollup is a first-level aggregation operation over a selected set of columns that reduces the size of stored data.
|
||||
Apache Druid® can summarize raw data at ingestion time using a process known as "rollup." [Rollup](../multi-stage-query/concepts.md#rollup) is a first-level aggregation operation over a selected set of columns that reduces the size of stored data.
|
||||
|
||||
This tutorial will demonstrate the effects of rollup on an example dataset.
|
||||
This tutorial demonstrates how to apply rollup during ingestion and highlights its effects during query execution. The examples in the tutorial use the [multi-stage query (MSQ)](../multi-stage-query/index.md) task engine to execute SQL statements.
|
||||
|
||||
For this tutorial, we'll assume you've already downloaded Druid as described in
|
||||
the [single-machine quickstart](index.md) and have it running on your local machine.
|
||||
## Prerequisites
|
||||
|
||||
It will also be helpful to have finished [Load a file](../tutorials/tutorial-batch.md) and [Query data](../tutorials/tutorial-query.md) tutorials.
|
||||
Before proceeding, download Druid as described in [Quickstart (local)](index.md) and have it running on your local machine. You don't need to load any data into the Druid cluster.
|
||||
|
||||
## Example data
|
||||
You should be familiar with data querying in Druid. If you haven't already, go through the [Query data](../tutorials/tutorial-query.md) tutorial first.
|
||||
|
||||
For this tutorial, we'll use a small sample of network flow event data, representing packet and byte counts for traffic from a source to a destination IP address that occurred within a particular second.
|
||||
|
||||
## Load the example data
|
||||
|
||||
For this tutorial, you use a small sample of network flow event data representing IP traffic.
|
||||
The data contains packet and byte counts from a source IP address to a destination IP address.
|
||||
|
||||
```json
|
||||
{"timestamp":"2018-01-01T01:01:35Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":20,"bytes":9024}
|
||||
|
@ -49,102 +52,63 @@ For this tutorial, we'll use a small sample of network flow event data, represen
|
|||
{"timestamp":"2018-01-02T21:35:45Z","srcIP":"7.7.7.7", "dstIP":"8.8.8.8","packets":12,"bytes":2818}
|
||||
```
|
||||
|
||||
A file containing this sample input data is located at `quickstart/tutorial/rollup-data.json`.
|
||||
Load the sample dataset using the [`INSERT INTO`](../multi-stage-query/reference.md/#insert) statement and the [`EXTERN`](../multi-stage-query/reference.md/#extern-function) function to ingest the data inline. In the [Druid web console](../operations/web-console.md), go to the **Query** view and run the following query:
|
||||
|
||||
We'll ingest this data using the following ingestion task spec, located at `quickstart/tutorial/rollup-index.json`.
|
||||
|
||||
```json
|
||||
{
|
||||
"type" : "index_parallel",
|
||||
"spec" : {
|
||||
"dataSchema" : {
|
||||
"dataSource" : "rollup-tutorial",
|
||||
"dimensionsSpec" : {
|
||||
"dimensions" : [
|
||||
"srcIP",
|
||||
"dstIP"
|
||||
]
|
||||
},
|
||||
"timestampSpec": {
|
||||
"column": "timestamp",
|
||||
"format": "iso"
|
||||
},
|
||||
"metricsSpec" : [
|
||||
{ "type" : "count", "name" : "count" },
|
||||
{ "type" : "longSum", "name" : "packets", "fieldName" : "packets" },
|
||||
{ "type" : "longSum", "name" : "bytes", "fieldName" : "bytes" }
|
||||
],
|
||||
"granularitySpec" : {
|
||||
"type" : "uniform",
|
||||
"segmentGranularity" : "week",
|
||||
"queryGranularity" : "minute",
|
||||
"intervals" : ["2018-01-01/2018-01-03"],
|
||||
"rollup" : true
|
||||
}
|
||||
},
|
||||
"ioConfig" : {
|
||||
"type" : "index_parallel",
|
||||
"inputSource" : {
|
||||
"type" : "local",
|
||||
"baseDir" : "quickstart/tutorial",
|
||||
"filter" : "rollup-data.json"
|
||||
},
|
||||
"inputFormat" : {
|
||||
"type" : "json"
|
||||
},
|
||||
"appendToExisting" : false
|
||||
},
|
||||
"tuningConfig" : {
|
||||
"type" : "index_parallel",
|
||||
"partitionsSpec": {
|
||||
"type": "dynamic"
|
||||
},
|
||||
"maxRowsInMemory" : 25000
|
||||
}
|
||||
}
|
||||
}
|
||||
```sql
|
||||
INSERT INTO "rollup_tutorial"
|
||||
WITH "inline_data" AS (
|
||||
SELECT *
|
||||
FROM TABLE(EXTERN('{
|
||||
"type":"inline",
|
||||
"data":"{\"timestamp\":\"2018-01-01T01:01:35Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":20,\"bytes\":9024}\n{\"timestamp\":\"2018-01-01T01:02:14Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":38,\"bytes\":6289}\n{\"timestamp\":\"2018-01-01T01:01:59Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":11,\"bytes\":5780}\n{\"timestamp\":\"2018-01-01T01:01:51Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":255,\"bytes\":21133}\n{\"timestamp\":\"2018-01-01T01:02:29Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":377,\"bytes\":359971}\n{\"timestamp\":\"2018-01-01T01:03:29Z\",\"srcIP\":\"1.1.1.1\",\"dstIP\":\"2.2.2.2\",\"packets\":49,\"bytes\":10204}\n{\"timestamp\":\"2018-01-02T21:33:14Z\",\"srcIP\":\"7.7.7.7\",\"dstIP\":\"8.8.8.8\",\"packets\":38,\"bytes\":6289}\n{\"timestamp\":\"2018-01-02T21:33:45Z\",\"srcIP\":\"7.7.7.7\",\"dstIP\":\"8.8.8.8\",\"packets\":123,\"bytes\":93999}\n{\"timestamp\":\"2018-01-02T21:35:45Z\",\"srcIP\":\"7.7.7.7\",\"dstIP\":\"8.8.8.8\",\"packets\":12,\"bytes\":2818}"}',
|
||||
'{"type":"json"}'))
|
||||
EXTEND ("timestamp" VARCHAR, "srcIP" VARCHAR, "dstIP" VARCHAR, "packets" BIGINT, "bytes" BIGINT)
|
||||
)
|
||||
SELECT
|
||||
FLOOR(TIME_PARSE("timestamp") TO MINUTE) AS __time,
|
||||
"srcIP",
|
||||
"dstIP",
|
||||
SUM("bytes") AS "bytes",
|
||||
SUM("packets") AS "packets",
|
||||
COUNT(*) AS "count"
|
||||
FROM "inline_data"
|
||||
GROUP BY 1, 2, 3
|
||||
PARTITIONED BY DAY
|
||||
```
|
||||
|
||||
Rollup has been enabled by setting `"rollup" : true` in the `granularitySpec`.
|
||||
Note the following aspects of the ingestion statement:
|
||||
* You transform the timestamp field using the `FLOOR` function to round timestamps down to the minute.
|
||||
* You group by the dimensions `timestamp`, `srcIP`, and `dstIP`.
|
||||
* You create the `bytes` and `packets` metrics, which are summed from their respective input fields.
|
||||
* You also create the `count` metric that records the number of rows that get rolled-up per each row in the datasource.
|
||||
|
||||
Note that we have `srcIP` and `dstIP` defined as dimensions, a longSum metric is defined for the `packets` and `bytes` columns, and the `queryGranularity` has been defined as `minute`.
|
||||
With rollup, Druid combines rows with identical timestamp and dimension values after the timestamp truncation. Druid computes and stores the metric values using the specified aggregation function over each set of rolled-up rows.
|
||||
|
||||
We will see how these definitions are used after we load this data.
|
||||
|
||||
## Load the example data
|
||||
|
||||
From the apache-druid-{{DRUIDVERSION}} package root, run the following command:
|
||||
|
||||
```bash
|
||||
bin/post-index-task --file quickstart/tutorial/rollup-index.json --url http://localhost:8081
|
||||
```
|
||||
|
||||
After the script completes, we will query the data.
|
||||
After the ingestion completes, you can query the data.
|
||||
|
||||
## Query the example data
|
||||
|
||||
Let's run `bin/dsql` and issue a `select * from "rollup-tutorial";` query to see what data was ingested.
|
||||
In the web console, open a new tab in the **Query** view. Run the following query to view the ingested data:
|
||||
|
||||
```bash
|
||||
$ bin/dsql
|
||||
Welcome to dsql, the command-line client for Druid SQL.
|
||||
Type "\h" for help.
|
||||
dsql> select * from "rollup-tutorial";
|
||||
┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐
|
||||
│ __time │ bytes │ count │ dstIP │ packets │ srcIP │
|
||||
├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤
|
||||
│ 2018-01-01T01:01:00.000Z │ 35937 │ 3 │ 2.2.2.2 │ 286 │ 1.1.1.1 │
|
||||
│ 2018-01-01T01:02:00.000Z │ 366260 │ 2 │ 2.2.2.2 │ 415 │ 1.1.1.1 │
|
||||
│ 2018-01-01T01:03:00.000Z │ 10204 │ 1 │ 2.2.2.2 │ 49 │ 1.1.1.1 │
|
||||
│ 2018-01-02T21:33:00.000Z │ 100288 │ 2 │ 8.8.8.8 │ 161 │ 7.7.7.7 │
|
||||
│ 2018-01-02T21:35:00.000Z │ 2818 │ 1 │ 8.8.8.8 │ 12 │ 7.7.7.7 │
|
||||
└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘
|
||||
Retrieved 5 rows in 1.18s.
|
||||
|
||||
dsql>
|
||||
```sql
|
||||
SELECT * FROM "rollup_tutorial"
|
||||
```
|
||||
|
||||
Let's look at the three events in the original input data that occurred during `2018-01-01T01:01`:
|
||||
Returns the following:
|
||||
|
||||
| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` |
|
||||
| -- | -- | -- | -- | -- | -- |
|
||||
| `2018-01-01T01:01:00.000Z` | `1.1.1.1` | `2.2.2.2` | `35,937` | `3` | `286` |
|
||||
| `2018-01-01T01:02:00.000Z` | `1.1.1.1` | `2.2.2.2` | `366,260` | `2` | `415` |
|
||||
| `2018-01-01T01:03:00.000Z` | `1.1.1.1` | `2.2.2.2` | `10,204` | `1` | `49` |
|
||||
| `2018-01-02T21:33:00.000Z` | `7.7.7.7` | `8.8.8.8` | `100,288` | `2` | `161` |
|
||||
| `2018-01-02T21:35:00.000Z` | `7.7.7.7` | `8.8.8.8` | `2,818` | `1` | `12` |
|
||||
|
||||
Notice there are only five rows as opposed to the nine rows in the example data. In the next section, you explore the components of the rolled-up rows.
|
||||
|
||||
## View rollup in action
|
||||
|
||||
Consider the three events in the original input data that occur over the course of minute `2018-01-01T01:01`:
|
||||
|
||||
```json
|
||||
{"timestamp":"2018-01-01T01:01:35Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":20,"bytes":9024}
|
||||
|
@ -152,47 +116,47 @@ Let's look at the three events in the original input data that occurred during `
|
|||
{"timestamp":"2018-01-01T01:01:59Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":11,"bytes":5780}
|
||||
```
|
||||
|
||||
These three rows have been "rolled up" into the following row:
|
||||
Druid combines the three rows into one during rollup:
|
||||
|
||||
```bash
|
||||
┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐
|
||||
│ __time │ bytes │ count │ dstIP │ packets │ srcIP │
|
||||
├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤
|
||||
│ 2018-01-01T01:01:00.000Z │ 35937 │ 3 │ 2.2.2.2 │ 286 │ 1.1.1.1 │
|
||||
└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘
|
||||
```
|
||||
| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` |
|
||||
| -- | -- | -- | -- | -- | -- |
|
||||
| `2018-01-01T01:01:00.000Z` | `1.1.1.1` | `2.2.2.2` | `35,937` | `3` | `286` |
|
||||
|
||||
The input rows have been grouped by the timestamp and dimension columns `{timestamp, srcIP, dstIP}` with sum aggregations on the metric columns `packets` and `bytes`.
|
||||
Before the grouping occurs, the `FLOOR(TIME_PARSE("timestamp") TO MINUTE)` expression buckets (floors) the timestamp column of the original input by minute.
|
||||
|
||||
Before the grouping occurs, the timestamps of the original input data are bucketed/floored by minute, due to the `"queryGranularity":"minute"` setting in the ingestion spec.
|
||||
The input rows are grouped because they have the same values for their dimension columns `{timestamp, srcIP, dstIP}`. The metric columns calculate the sum aggregation of the grouped rows for `packets` and `bytes`. The `count` metric shows how many rows from the original input data contributed to the final rolled-up row.
|
||||
|
||||
Likewise, these two events that occurred during `2018-01-01T01:02` have been rolled up:
|
||||
Now, consider the two events in the original input data that occur over the course of minute `2018-01-01T01:02`:
|
||||
|
||||
```json
|
||||
{"timestamp":"2018-01-01T01:02:14Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":38,"bytes":6289}
|
||||
{"timestamp":"2018-01-01T01:02:29Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":377,"bytes":359971}
|
||||
```
|
||||
|
||||
```bash
|
||||
┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐
|
||||
│ __time │ bytes │ count │ dstIP │ packets │ srcIP │
|
||||
├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤
|
||||
│ 2018-01-01T01:02:00.000Z │ 366260 │ 2 │ 2.2.2.2 │ 415 │ 1.1.1.1 │
|
||||
└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘
|
||||
```
|
||||
The rows are grouped into the following during rollup:
|
||||
|
||||
For the last event recording traffic between 1.1.1.1 and 2.2.2.2, no rollup took place, because this was the only event that occurred during `2018-01-01T01:03`:
|
||||
| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` |
|
||||
| -- | -- | -- | -- | -- | -- |
|
||||
| `2018-01-01T01:02:00.000Z` | `1.1.1.1` | `2.2.2.2` | `366,260` | `2` | `415` |
|
||||
|
||||
In the original input data, only one event occurs over the course of minute `2018-01-01T01:03`:
|
||||
|
||||
```json
|
||||
{"timestamp":"2018-01-01T01:03:29Z","srcIP":"1.1.1.1", "dstIP":"2.2.2.2","packets":49,"bytes":10204}
|
||||
```
|
||||
|
||||
```bash
|
||||
┌──────────────────────────┬────────┬───────┬─────────┬─────────┬─────────┐
|
||||
│ __time │ bytes │ count │ dstIP │ packets │ srcIP │
|
||||
├──────────────────────────┼────────┼───────┼─────────┼─────────┼─────────┤
|
||||
│ 2018-01-01T01:03:00.000Z │ 10204 │ 1 │ 2.2.2.2 │ 49 │ 1.1.1.1 │
|
||||
└──────────────────────────┴────────┴───────┴─────────┴─────────┴─────────┘
|
||||
```
|
||||
Therefore, no rollup takes place:
|
||||
|
||||
Note that the `count` metric shows how many rows in the original input data contributed to the final "rolled up" row.
|
||||
| `__time` | `srcIP` | `dstIP` | `bytes` | `count` | `packets` |
|
||||
| -- | -- | -- | -- | -- | -- |
|
||||
| `2018-01-01T01:03:00.000Z` | `1.1.1.1` | `2.2.2.2` | `10,204` | `1` | `49` |
|
||||
|
||||
|
||||
## Learn more
|
||||
|
||||
See the following topics for more information:
|
||||
|
||||
* [Data rollup](../ingestion/rollup.md) for suggestions and best practices when performing rollup.
|
||||
* [SQL-based ingestion concepts](../multi-stage-query/concepts.md#rollup) for information on rollup using SQL-based ingestion.
|
||||
* [SQL-based ingestion query examples](../multi-stage-query/examples.md#insert-with-rollup) for another example of data rollup.
|
||||
* [Druid schema model](../ingestion/schema-model.md) to learn about the primary timestamp, dimensions, and metrics.
|
||||
|
|
|
@ -118,7 +118,7 @@ Druid returns the following:
|
|||
|`another_value`|1|1|
|
||||
|`some_value`|1|1|
|
||||
|
||||
Also note that GROUP BY expressions yields distinct entries for `null` and the empty string.
|
||||
Also note that GROUP BY expressions yield distinct entries for `null` and the empty string.
|
||||
|
||||
### Filter for empty strings in addition to null
|
||||
|
||||
|
|
|
@ -304,7 +304,6 @@ The following native Scan query returns the rows of the datasource and unnests t
|
|||
"m2",
|
||||
"unnest-dim3"
|
||||
],
|
||||
"legacy": false,
|
||||
"granularity": {
|
||||
"type": "all"
|
||||
},
|
||||
|
@ -480,7 +479,6 @@ This query joins the `nested_data` table with itself and outputs the unnested da
|
|||
"m2",
|
||||
"v0"
|
||||
],
|
||||
"legacy": false,
|
||||
"context": {
|
||||
"sqlOuterLimit": 1001,
|
||||
"useNativeQueryExplain": true
|
||||
|
@ -523,7 +521,6 @@ This query joins the `nested_data` table with itself and outputs the unnested da
|
|||
"m2",
|
||||
"unnest-dim3"
|
||||
],
|
||||
"legacy": false,
|
||||
"context": {
|
||||
"sqlOuterLimit": 1001,
|
||||
"useNativeQueryExplain": true
|
||||
|
@ -572,7 +569,6 @@ The following query returns the columns `dim45` and `m1`. The `dim45` column is
|
|||
"dim45",
|
||||
"m1"
|
||||
],
|
||||
"legacy": false,
|
||||
"granularity": {
|
||||
"type": "all"
|
||||
},
|
||||
|
@ -632,7 +628,6 @@ The following Scan query unnests the column `dim3` into `d3` and a virtual colum
|
|||
"dim5",
|
||||
"d45"
|
||||
],
|
||||
"legacy": false,
|
||||
"context": {
|
||||
"queryId": "2618b9ce-6c0d-414e-b88d-16fb59b9c481",
|
||||
"sqlOuterLimit": 1001,
|
||||
|
|
|
@ -35,7 +35,7 @@
|
|||
|
||||
<properties>
|
||||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
|
||||
<jclouds.version>2.5.0</jclouds.version>
|
||||
<jclouds.version>2.6.0</jclouds.version>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,6 +28,7 @@ public class DruidK8sConstants
|
|||
public static final String TASK_TYPE = "task.type";
|
||||
public static final String TASK_GROUP_ID = "task.group.id";
|
||||
public static final String TASK_DATASOURCE = "task.datasource";
|
||||
public static final String TASK_JOB_TEMPLATE = "task.jobTemplate";
|
||||
public static final int PORT = 8100;
|
||||
public static final int TLS_PORT = 8091;
|
||||
public static final int DEFAULT_CPU_MILLICORES = 1000;
|
||||
|
@ -42,6 +43,7 @@ public class DruidK8sConstants
|
|||
public static final String DRUID_HOSTNAME_ENV = "HOSTNAME";
|
||||
public static final String LABEL_KEY = "druid.k8s.peons";
|
||||
public static final String DRUID_LABEL_PREFIX = "druid.";
|
||||
public static final String BASE_TEMPLATE_NAME = "base";
|
||||
public static final long MAX_ENV_VARIABLE_KBS = 130048; // 127 KB
|
||||
static final Predicate<Throwable> IS_TRANSIENT = e -> e instanceof KubernetesResourceNotFoundException;
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
|
|||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import io.fabric8.kubernetes.api.model.PodTemplate;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.k8s.overlord.taskadapter.PodTemplateWithName;
|
||||
|
||||
import javax.validation.constraints.NotNull;
|
||||
import java.util.Map;
|
||||
|
@ -42,7 +43,7 @@ public interface PodTemplateSelectStrategy
|
|||
* allows for customized resource allocation and management tailored to the task's specific requirements.
|
||||
*
|
||||
* @param task The task for which the Pod template is determined.
|
||||
* @return The pod template that should be used to run the task.
|
||||
* @return The PodTemplateWithName POJO that contains the name of the template selected and the template itself.
|
||||
*/
|
||||
@NotNull PodTemplate getPodTemplateForTask(Task task, Map<String, PodTemplate> templates);
|
||||
@NotNull PodTemplateWithName getPodTemplateForTask(Task task, Map<String, PodTemplate> templates);
|
||||
}
|
||||
|
|
|
@ -24,6 +24,8 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
import io.fabric8.kubernetes.api.model.PodTemplate;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.k8s.overlord.common.DruidK8sConstants;
|
||||
import org.apache.druid.k8s.overlord.taskadapter.PodTemplateWithName;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -53,15 +55,18 @@ public class SelectorBasedPodTemplateSelectStrategy implements PodTemplateSelect
|
|||
* @return the template if a selector matches, otherwise fallback to base template
|
||||
*/
|
||||
@Override
|
||||
public PodTemplate getPodTemplateForTask(Task task, Map<String, PodTemplate> templates)
|
||||
public PodTemplateWithName getPodTemplateForTask(Task task, Map<String, PodTemplate> templates)
|
||||
{
|
||||
String templateKey = selectors.stream()
|
||||
.filter(selector -> selector.evaluate(task))
|
||||
.findFirst()
|
||||
.map(Selector::getSelectionKey)
|
||||
.orElse("base");
|
||||
.orElse(DruidK8sConstants.BASE_TEMPLATE_NAME);
|
||||
|
||||
return templates.getOrDefault(templateKey, templates.get("base"));
|
||||
if (!templates.containsKey(templateKey)) {
|
||||
templateKey = DruidK8sConstants.BASE_TEMPLATE_NAME;
|
||||
}
|
||||
return new PodTemplateWithName(templateKey, templates.get(templateKey));
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
|
|
@ -22,6 +22,8 @@ package org.apache.druid.k8s.overlord.execution;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import io.fabric8.kubernetes.api.model.PodTemplate;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.k8s.overlord.common.DruidK8sConstants;
|
||||
import org.apache.druid.k8s.overlord.taskadapter.PodTemplateWithName;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -40,9 +42,10 @@ public class TaskTypePodTemplateSelectStrategy implements PodTemplateSelectStrat
|
|||
}
|
||||
|
||||
@Override
|
||||
public PodTemplate getPodTemplateForTask(Task task, Map<String, PodTemplate> templates)
|
||||
public PodTemplateWithName getPodTemplateForTask(Task task, Map<String, PodTemplate> templates)
|
||||
{
|
||||
return templates.getOrDefault(task.getType(), templates.get("base"));
|
||||
String templateKey = templates.containsKey(task.getType()) ? task.getType() : DruidK8sConstants.BASE_TEMPLATE_NAME;
|
||||
return new PodTemplateWithName(templateKey, templates.get(templateKey));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -138,19 +138,21 @@ public class PodTemplateTaskAdapter implements TaskAdapter
|
|||
podTemplateSelectStrategy = dynamicConfig.getPodTemplateSelectStrategy();
|
||||
}
|
||||
|
||||
PodTemplate podTemplate = podTemplateSelectStrategy.getPodTemplateForTask(task, templates);
|
||||
PodTemplateWithName podTemplateWithName = podTemplateSelectStrategy.getPodTemplateForTask(task, templates);
|
||||
|
||||
return new JobBuilder()
|
||||
.withNewMetadata()
|
||||
.withName(new K8sTaskId(task).getK8sJobName())
|
||||
.addToLabels(getJobLabels(taskRunnerConfig, task))
|
||||
.addToAnnotations(getJobAnnotations(taskRunnerConfig, task))
|
||||
.addToAnnotations(DruidK8sConstants.TASK_JOB_TEMPLATE, podTemplateWithName.getName())
|
||||
.endMetadata()
|
||||
.withNewSpec()
|
||||
.withTemplate(podTemplate.getTemplate())
|
||||
.withTemplate(podTemplateWithName.getPodTemplate().getTemplate())
|
||||
.editTemplate()
|
||||
.editOrNewMetadata()
|
||||
.addToAnnotations(getPodTemplateAnnotations(task))
|
||||
.addToAnnotations(DruidK8sConstants.TASK_JOB_TEMPLATE, podTemplateWithName.getName())
|
||||
.addToLabels(getPodLabels(taskRunnerConfig, task))
|
||||
.endMetadata()
|
||||
.editSpec()
|
||||
|
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.k8s.overlord.taskadapter;
|
||||
|
||||
import io.fabric8.kubernetes.api.model.PodTemplate;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import java.util.Objects;
|
||||
|
||||
public class PodTemplateWithName
|
||||
{
|
||||
private final String name;
|
||||
private final PodTemplate podTemplate;
|
||||
|
||||
public PodTemplateWithName(String name, PodTemplate podTemplate)
|
||||
{
|
||||
this.name = name;
|
||||
this.podTemplate = podTemplate;
|
||||
}
|
||||
|
||||
@Nonnull
|
||||
public String getName()
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
@Nonnull
|
||||
public PodTemplate getPodTemplate()
|
||||
{
|
||||
return podTemplate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
PodTemplateWithName that = (PodTemplateWithName) o;
|
||||
return Objects.equals(name, that.name) &&
|
||||
Objects.equals(podTemplate, that.podTemplate);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(name, podTemplate);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "PodTemplateWithName{" +
|
||||
"name='" + name + '\'' +
|
||||
", podTemplate=" + podTemplate +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -79,7 +79,6 @@ public class K8sTestUtils
|
|||
null
|
||||
),
|
||||
new IndexTask.IndexIOConfig(
|
||||
null,
|
||||
new LocalInputSource(new File("lol"), "rofl"),
|
||||
new NoopInputFormat(),
|
||||
true,
|
||||
|
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.k8s.overlord.common;
|
||||
|
||||
import io.fabric8.kubernetes.api.model.PodTemplate;
|
||||
import io.fabric8.kubernetes.api.model.PodTemplateBuilder;
|
||||
import org.apache.druid.k8s.overlord.taskadapter.PodTemplateWithName;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
|
||||
public class PodTemplateWithNameTest
|
||||
{
|
||||
@Test
|
||||
void testEqualityToMakeCoverageHappy()
|
||||
{
|
||||
PodTemplateWithName podTemplateWithName = new PodTemplateWithName(
|
||||
"name",
|
||||
new PodTemplateBuilder().build()
|
||||
);
|
||||
PodTemplateWithName podTemplateWithName2 = podTemplateWithName;
|
||||
|
||||
assertEquals(podTemplateWithName, podTemplateWithName2);
|
||||
assertNotEquals(podTemplateWithName, null);
|
||||
assertNotEquals(podTemplateWithName, "string");
|
||||
assertEquals(podTemplateWithName.hashCode(), podTemplateWithName2.hashCode());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGettersToMakeCoverageHappy()
|
||||
{
|
||||
String name = "name";
|
||||
PodTemplate podTemplate = new PodTemplateBuilder().build();
|
||||
PodTemplateWithName podTemplateWithName = new PodTemplateWithName(
|
||||
name,
|
||||
podTemplate
|
||||
);
|
||||
|
||||
assertEquals(name, podTemplateWithName.getName());
|
||||
assertEquals(podTemplate, podTemplateWithName.getPodTemplate());
|
||||
}
|
||||
}
|
|
@ -27,6 +27,7 @@ import io.fabric8.kubernetes.api.model.ObjectMeta;
|
|||
import io.fabric8.kubernetes.api.model.PodTemplate;
|
||||
import org.apache.druid.indexing.common.task.NoopTask;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.k8s.overlord.taskadapter.PodTemplateWithName;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
@ -97,7 +98,10 @@ public class SelectorBasedPodTemplateSelectStrategyTest
|
|||
List<Selector> emptySelectors = Collections.emptyList();
|
||||
SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(emptySelectors);
|
||||
Task task = NoopTask.create();
|
||||
Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).getMetadata().getName());
|
||||
PodTemplateWithName podTemplateWithName = strategy.getPodTemplateForTask(task, templates);
|
||||
Assert.assertEquals("base", podTemplateWithName.getName());
|
||||
Assert.assertEquals("base", podTemplateWithName.getPodTemplate().getMetadata().getName());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -107,7 +111,9 @@ public class SelectorBasedPodTemplateSelectStrategyTest
|
|||
List<Selector> selectors = Collections.singletonList(noMatchSelector);
|
||||
SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors);
|
||||
Task task = NoopTask.create();
|
||||
Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).getMetadata().getName());
|
||||
PodTemplateWithName podTemplateWithName = strategy.getPodTemplateForTask(task, templates);
|
||||
Assert.assertEquals("base", podTemplateWithName.getName());
|
||||
Assert.assertEquals("base", podTemplateWithName.getPodTemplate().getMetadata().getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -124,7 +130,9 @@ public class SelectorBasedPodTemplateSelectStrategyTest
|
|||
);
|
||||
SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors);
|
||||
Task task = NoopTask.create();
|
||||
Assert.assertEquals("match", strategy.getPodTemplateForTask(task, templates).getMetadata().getName());
|
||||
PodTemplateWithName podTemplateWithName = strategy.getPodTemplateForTask(task, templates);
|
||||
Assert.assertEquals("match", podTemplateWithName.getName());
|
||||
Assert.assertEquals("match", podTemplateWithName.getPodTemplate().getMetadata().getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.k8s.overlord.taskadapter;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import io.fabric8.kubernetes.api.model.Pod;
|
||||
|
@ -27,7 +26,6 @@ import io.fabric8.kubernetes.api.model.PodSpec;
|
|||
import io.fabric8.kubernetes.api.model.batch.v1.Job;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.druid.guice.FirehoseModule;
|
||||
import org.apache.druid.indexing.common.TestUtils;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.indexing.common.config.TaskConfigBuilder;
|
||||
|
@ -83,9 +81,6 @@ public class DruidPeonClientIntegrationTest
|
|||
{
|
||||
TestUtils utils = new TestUtils();
|
||||
jsonMapper = utils.getTestObjectMapper();
|
||||
for (Module jacksonModule : new FirehoseModule().getJacksonModules()) {
|
||||
jsonMapper.registerModule(jacksonModule);
|
||||
}
|
||||
jsonMapper.registerSubtypes(
|
||||
new NamedType(ParallelIndexTuningConfig.class, "index_parallel"),
|
||||
new NamedType(IndexTask.IndexTuningConfig.class, "index")
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.k8s.overlord.taskadapter;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.google.common.base.Joiner;
|
||||
|
@ -44,7 +43,6 @@ import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient;
|
|||
import org.apache.commons.lang.RandomStringUtils;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.guice.FirehoseModule;
|
||||
import org.apache.druid.indexing.common.TestUtils;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.indexing.common.config.TaskConfigBuilder;
|
||||
|
@ -101,9 +99,6 @@ class K8sTaskAdapterTest
|
|||
{
|
||||
TestUtils utils = new TestUtils();
|
||||
jsonMapper = utils.getTestObjectMapper();
|
||||
for (Module jacksonModule : new FirehoseModule().getJacksonModules()) {
|
||||
jsonMapper.registerModule(jacksonModule);
|
||||
}
|
||||
jsonMapper.registerSubtypes(
|
||||
new NamedType(ParallelIndexTuningConfig.class, "index_parallel"),
|
||||
new NamedType(IndexTask.IndexTuningConfig.class, "index")
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.k8s.overlord.taskadapter;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
@ -28,7 +27,6 @@ import io.fabric8.kubernetes.api.model.PodSpec;
|
|||
import io.fabric8.kubernetes.api.model.batch.v1.Job;
|
||||
import io.fabric8.kubernetes.client.KubernetesClient;
|
||||
import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient;
|
||||
import org.apache.druid.guice.FirehoseModule;
|
||||
import org.apache.druid.indexing.common.TestUtils;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.indexing.common.config.TaskConfigBuilder;
|
||||
|
@ -67,9 +65,6 @@ class MultiContainerTaskAdapterTest
|
|||
{
|
||||
TestUtils utils = new TestUtils();
|
||||
jsonMapper = utils.getTestObjectMapper();
|
||||
for (Module jacksonModule : new FirehoseModule().getJacksonModules()) {
|
||||
jsonMapper.registerModule(jacksonModule);
|
||||
}
|
||||
jsonMapper.registerSubtypes(
|
||||
new NamedType(ParallelIndexTuningConfig.class, "index_parallel"),
|
||||
new NamedType(IndexTask.IndexTuningConfig.class, "index")
|
||||
|
|
|
@ -163,7 +163,7 @@ public class PodTemplateTaskAdapterTest
|
|||
|
||||
Task task = new NoopTask("id", "id", "datasource", 0, 0, null);
|
||||
Job actual = adapter.fromTask(task);
|
||||
Job expected = K8sTestUtils.fileToResource("expectedNoopJob.yaml", Job.class);
|
||||
Job expected = K8sTestUtils.fileToResource("expectedNoopJobBase.yaml", Job.class);
|
||||
|
||||
assertJobSpecsEqual(actual, expected);
|
||||
}
|
||||
|
@ -197,7 +197,7 @@ public class PodTemplateTaskAdapterTest
|
|||
|
||||
Task task = new NoopTask("id", "id", "datasource", 0, 0, null);
|
||||
Job actual = adapter.fromTask(task);
|
||||
Job expected = K8sTestUtils.fileToResource("expectedNoopJobTlsEnabled.yaml", Job.class);
|
||||
Job expected = K8sTestUtils.fileToResource("expectedNoopJobTlsEnabledBase.yaml", Job.class);
|
||||
|
||||
assertJobSpecsEqual(actual, expected);
|
||||
}
|
||||
|
|
|
@ -19,14 +19,12 @@
|
|||
|
||||
package org.apache.druid.k8s.overlord.taskadapter;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import io.fabric8.kubernetes.api.model.Pod;
|
||||
import io.fabric8.kubernetes.api.model.batch.v1.Job;
|
||||
import io.fabric8.kubernetes.client.KubernetesClient;
|
||||
import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient;
|
||||
import org.apache.druid.guice.FirehoseModule;
|
||||
import org.apache.druid.indexing.common.TestUtils;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.indexing.common.config.TaskConfigBuilder;
|
||||
|
@ -66,9 +64,6 @@ class SingleContainerTaskAdapterTest
|
|||
{
|
||||
TestUtils utils = new TestUtils();
|
||||
jsonMapper = utils.getTestObjectMapper();
|
||||
for (Module jacksonModule : new FirehoseModule().getJacksonModules()) {
|
||||
jsonMapper.registerModule(jacksonModule);
|
||||
}
|
||||
jsonMapper.registerSubtypes(
|
||||
new NamedType(ParallelIndexTuningConfig.class, "index_parallel"),
|
||||
new NamedType(IndexTask.IndexTuningConfig.class, "index")
|
||||
|
|
|
@ -8,11 +8,13 @@ metadata:
|
|||
druid.task.type: "noop"
|
||||
druid.task.group.id: "id"
|
||||
druid.task.datasource: "datasource"
|
||||
|
||||
annotations:
|
||||
task.id: "id"
|
||||
task.type: "noop"
|
||||
task.group.id: "id"
|
||||
task.datasource: "datasource"
|
||||
task.jobTemplate: noop
|
||||
spec:
|
||||
activeDeadlineSeconds: 14400
|
||||
backoffLimit: 0
|
||||
|
@ -32,6 +34,7 @@ spec:
|
|||
task.type: "noop"
|
||||
task.group.id: "id"
|
||||
task.datasource: "datasource"
|
||||
task.jobTemplate: noop
|
||||
spec:
|
||||
containers:
|
||||
- command:
|
||||
|
|
|
@ -0,0 +1,55 @@
|
|||
apiVersion: batch/v1
|
||||
kind: Job
|
||||
metadata:
|
||||
name: "id-3e70afe5cd823dfc7dd308eea616426b"
|
||||
labels:
|
||||
druid.k8s.peons: "true"
|
||||
druid.task.id: "id"
|
||||
druid.task.type: "noop"
|
||||
druid.task.group.id: "id"
|
||||
druid.task.datasource: "datasource"
|
||||
|
||||
annotations:
|
||||
task.id: "id"
|
||||
task.type: "noop"
|
||||
task.group.id: "id"
|
||||
task.datasource: "datasource"
|
||||
task.jobTemplate: base
|
||||
spec:
|
||||
activeDeadlineSeconds: 14400
|
||||
backoffLimit: 0
|
||||
ttlSecondsAfterFinished: 172800
|
||||
template:
|
||||
metadata:
|
||||
labels:
|
||||
druid.k8s.peons: "true"
|
||||
druid.task.id: "id"
|
||||
druid.task.type: "noop"
|
||||
druid.task.group.id: "id"
|
||||
druid.task.datasource: "datasource"
|
||||
annotations:
|
||||
task: "H4sIAAAAAAAAAD2MvQ4CIRCE32VqijsTG1qLi7W+wArEbHICrmC8EN7dJf40k/lmJtNQthxgEVPKMGCvXsXgKqnm4x89FTqlKm6MBzw+YCA1nvmm8W4/TQYuxRJeBbZ17cJ3ZhvoSbzShVcu2zLOf9cS7pUl+ANlclrCzr2/AQUK0FqZAAAA"
|
||||
tls.enabled: "false"
|
||||
task.id: "id"
|
||||
task.type: "noop"
|
||||
task.group.id: "id"
|
||||
task.datasource: "datasource"
|
||||
task.jobTemplate: base
|
||||
spec:
|
||||
containers:
|
||||
- command:
|
||||
- sleep
|
||||
- "3600"
|
||||
env:
|
||||
- name: "TASK_DIR"
|
||||
value: "/tmp"
|
||||
- name: "TASK_ID"
|
||||
value: "id"
|
||||
- name: "LOAD_BROADCAST_SEGMENTS"
|
||||
value: "false"
|
||||
- name: "TASK_JSON"
|
||||
valueFrom:
|
||||
fieldRef:
|
||||
fieldPath: "metadata.annotations['task']"
|
||||
image: one
|
||||
name: primary
|
|
@ -13,6 +13,7 @@ metadata:
|
|||
task.type: "noop"
|
||||
task.group.id: "api-issued_kill_wikipedia3_omjobnbc_1000-01-01T00:00:00.000Z_2023-05-14T00:00:00.000Z_2023-05-15T17:03:01.220Z"
|
||||
task.datasource: "data_source"
|
||||
task.jobTemplate: noop
|
||||
spec:
|
||||
activeDeadlineSeconds: 14400
|
||||
backoffLimit: 0
|
||||
|
@ -32,6 +33,7 @@ spec:
|
|||
task.type: "noop"
|
||||
task.group.id: "api-issued_kill_wikipedia3_omjobnbc_1000-01-01T00:00:00.000Z_2023-05-14T00:00:00.000Z_2023-05-15T17:03:01.220Z"
|
||||
task.datasource: "data_source"
|
||||
task.jobTemplate: noop
|
||||
spec:
|
||||
containers:
|
||||
- command:
|
||||
|
|
|
@ -13,6 +13,7 @@ metadata:
|
|||
task.type: "noop"
|
||||
task.group.id: "id"
|
||||
task.datasource: "datasource"
|
||||
task.jobTemplate: noop
|
||||
spec:
|
||||
activeDeadlineSeconds: 14400
|
||||
backoffLimit: 0
|
||||
|
@ -31,6 +32,7 @@ spec:
|
|||
task.type: "noop"
|
||||
task.group.id: "id"
|
||||
task.datasource: "datasource"
|
||||
task.jobTemplate: noop
|
||||
spec:
|
||||
containers:
|
||||
- command:
|
||||
|
|
|
@ -13,6 +13,7 @@ metadata:
|
|||
task.type: "noop"
|
||||
task.group.id: "id"
|
||||
task.datasource: "datasource"
|
||||
task.jobTemplate: noop
|
||||
spec:
|
||||
activeDeadlineSeconds: 14400
|
||||
backoffLimit: 0
|
||||
|
@ -32,6 +33,7 @@ spec:
|
|||
task.type: "noop"
|
||||
task.group.id: "id"
|
||||
task.datasource: "datasource"
|
||||
task.jobTemplate: noop
|
||||
spec:
|
||||
containers:
|
||||
- command:
|
||||
|
|
|
@ -0,0 +1,54 @@
|
|||
apiVersion: batch/v1
|
||||
kind: Job
|
||||
metadata:
|
||||
name: "id-3e70afe5cd823dfc7dd308eea616426b"
|
||||
labels:
|
||||
druid.k8s.peons: "true"
|
||||
druid.task.id: "id"
|
||||
druid.task.type: "noop"
|
||||
druid.task.group.id: "id"
|
||||
druid.task.datasource: "datasource"
|
||||
annotations:
|
||||
task.id: "id"
|
||||
task.type: "noop"
|
||||
task.group.id: "id"
|
||||
task.datasource: "datasource"
|
||||
task.jobTemplate: base
|
||||
spec:
|
||||
activeDeadlineSeconds: 14400
|
||||
backoffLimit: 0
|
||||
ttlSecondsAfterFinished: 172800
|
||||
template:
|
||||
metadata:
|
||||
labels:
|
||||
druid.k8s.peons: "true"
|
||||
druid.task.id: "id"
|
||||
druid.task.type: "noop"
|
||||
druid.task.group.id: "id"
|
||||
druid.task.datasource: "datasource"
|
||||
annotations:
|
||||
task: "H4sIAAAAAAAAAD2MvQ4CIRCE32VqijsTG1qLi7W+wArEbHICrmC8EN7dJf40k/lmJtNQthxgEVPKMGCvXsXgKqnm4x89FTqlKm6MBzw+YCA1nvmm8W4/TQYuxRJeBbZ17cJ3ZhvoSbzShVcu2zLOf9cS7pUl+ANlclrCzr2/AQUK0FqZAAAA"
|
||||
tls.enabled: "true"
|
||||
task.id: "id"
|
||||
task.type: "noop"
|
||||
task.group.id: "id"
|
||||
task.datasource: "datasource"
|
||||
task.jobTemplate: base
|
||||
spec:
|
||||
containers:
|
||||
- command:
|
||||
- sleep
|
||||
- "3600"
|
||||
env:
|
||||
- name: "TASK_DIR"
|
||||
value: "/tmp"
|
||||
- name: "TASK_ID"
|
||||
value: "id"
|
||||
- name: "LOAD_BROADCAST_SEGMENTS"
|
||||
value: "false"
|
||||
- name: "TASK_JSON"
|
||||
valueFrom:
|
||||
fieldRef:
|
||||
fieldPath: "metadata.annotations['task']"
|
||||
image: one
|
||||
name: primary
|
|
@ -47,7 +47,7 @@ import org.apache.druid.metadata.SqlSegmentsMetadataManager;
|
|||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
|
||||
import org.apache.druid.segment.realtime.ChatHandlerProvider;
|
||||
import org.apache.druid.segment.transform.TransformSpec;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
|
|
@ -40,8 +40,8 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
|||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
|
||||
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
|
||||
import org.apache.druid.segment.realtime.ChatHandlerProvider;
|
||||
import org.apache.druid.segment.realtime.NoopChatHandlerProvider;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.easymock.EasyMock;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
|
|
|
@ -52,7 +52,7 @@ import org.apache.druid.segment.TestHelper;
|
|||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
|
||||
import org.apache.druid.segment.metadata.SegmentSchemaManager;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
|
||||
import org.apache.druid.segment.realtime.ChatHandlerProvider;
|
||||
import org.apache.druid.segment.transform.TransformSpec;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
|
|
@ -39,7 +39,7 @@
|
|||
<!-- These guava and grpc versions are used only in the opentelemetry-extension.
|
||||
Look at build section for more details about shading. -->
|
||||
<shade.guava.version>32.0.1-jre</shade.guava.version>
|
||||
<shade.grpc.version>1.57.2</shade.grpc.version>
|
||||
<shade.grpc.version>1.65.1</shade.grpc.version>
|
||||
</properties>
|
||||
<dependencyManagement>
|
||||
<dependencies>
|
||||
|
|
|
@ -201,9 +201,6 @@
|
|||
"zk/connected" : { "dimensions" : [], "type" : "gauge", "help": "Indicator of connection status to zookeeper."},
|
||||
"zk/reconnect/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Amount of time, in seconds, that a server was disconnected from ZooKeeper before reconnecting." },
|
||||
|
||||
"ingest/events/buffered" : { "dimensions" : ["serviceName", "dataSource", "bufferCapacity"], "type" : "gauge", "help": "Number of events queued in the EventReceiverFirehose's buffer"},
|
||||
"ingest/bytes/received" : { "dimensions" : ["serviceName", "dataSource"], "type" : "gauge", "help": "Number of bytes received by the EventReceiverFirehose."},
|
||||
|
||||
"sys/swap/free" : { "dimensions" : [], "type" : "gauge", "help": "Free swap"},
|
||||
"sys/swap/max" : { "dimensions" : [], "type" : "gauge", "help": "Max swap"},
|
||||
"sys/swap/pageIn" : { "dimensions" : [], "type" : "gauge", "help": "Paged in swap"},
|
||||
|
|
|
@ -92,6 +92,11 @@
|
|||
<artifactId>error_prone_annotations</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>joda-time</groupId>
|
||||
<artifactId>joda-time</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-sql</artifactId>
|
||||
|
@ -137,5 +142,10 @@
|
|||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.easymock</groupId>
|
||||
<artifactId>easymock</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
|
|
@ -19,18 +19,12 @@
|
|||
|
||||
package org.apache.druid.spectator.histogram;
|
||||
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ComplexColumn;
|
||||
import org.apache.druid.segment.data.ReadableOffset;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
public class SpectatorHistogramIndexBasedComplexColumn implements ComplexColumn
|
||||
{
|
||||
private final SpectatorHistogramIndexed index;
|
||||
private final String typeName;
|
||||
private static final Number ZERO = 0;
|
||||
|
||||
public SpectatorHistogramIndexBasedComplexColumn(String typeName, SpectatorHistogramIndexed index)
|
||||
{
|
||||
|
@ -59,72 +53,11 @@ public class SpectatorHistogramIndexBasedComplexColumn implements ComplexColumn
|
|||
@Override
|
||||
public int getLength()
|
||||
{
|
||||
return index.size();
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnValueSelector<SpectatorHistogram> makeColumnValueSelector(ReadableOffset offset)
|
||||
{
|
||||
// Use ColumnValueSelector directly so that we support being queried as a Number using
|
||||
// longSum or doubleSum aggregators, the NullableNumericBufferAggregator will call isNull.
|
||||
// This allows us to behave as a Number or SpectatorHistogram object.
|
||||
// When queried as a Number, we're returning the count of entries in the histogram.
|
||||
// As such, we can safely return 0 where the histogram is null.
|
||||
return new ColumnValueSelector<SpectatorHistogram>()
|
||||
{
|
||||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
return getObject() == null;
|
||||
}
|
||||
|
||||
private Number getOrZero()
|
||||
{
|
||||
SpectatorHistogram histogram = getObject();
|
||||
return histogram != null ? histogram : ZERO;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
return getOrZero().longValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
return getOrZero().floatValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble()
|
||||
{
|
||||
return getOrZero().doubleValue();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public SpectatorHistogram getObject()
|
||||
{
|
||||
return (SpectatorHistogram) getRowValue(offset.getOffset());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return getClazz();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("column", SpectatorHistogramIndexBasedComplexColumn.this);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,8 +20,14 @@
|
|||
package org.apache.druid.spectator.histogram;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.netflix.spectator.api.histogram.PercentileBuckets;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
import org.apache.druid.data.input.impl.NoopInputRowParser;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.query.Druids;
|
||||
|
@ -32,6 +38,9 @@ import org.apache.druid.query.Result;
|
|||
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.AggregatorUtil;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
|
||||
import org.apache.druid.query.groupby.ResultRow;
|
||||
|
@ -42,13 +51,17 @@ import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
|
|||
import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
|
||||
import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||
import org.apache.druid.query.timeseries.TimeseriesResultValue;
|
||||
import org.apache.druid.segment.IncrementalIndexSegment;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.QueryableIndexSegment;
|
||||
import org.apache.druid.segment.Segment;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
|
@ -59,6 +72,7 @@ import org.junit.runners.Parameterized;
|
|||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -716,6 +730,59 @@ public class SpectatorHistogramAggregatorTest extends InitializedNullHandlingTes
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBuildingAndCountingHistogramsIncrementalIndex() throws Exception
|
||||
{
|
||||
List<String> dimensions = Collections.singletonList("d");
|
||||
int n = 10;
|
||||
DateTime startOfDay = DateTimes.of("2000-01-01");
|
||||
List<InputRow> inputRows = new ArrayList<>(n);
|
||||
for (int i = 1; i <= n; i++) {
|
||||
String val = String.valueOf(i * 1.0d);
|
||||
|
||||
inputRows.add(new MapBasedInputRow(
|
||||
startOfDay.plusMinutes(i),
|
||||
dimensions,
|
||||
ImmutableMap.of("x", i, "d", val)
|
||||
));
|
||||
}
|
||||
|
||||
IncrementalIndex index = AggregationTestHelper.createIncrementalIndex(
|
||||
inputRows.iterator(),
|
||||
new NoopInputRowParser(null),
|
||||
new AggregatorFactory[]{
|
||||
new CountAggregatorFactory("count"),
|
||||
new SpectatorHistogramAggregatorFactory("histogram", "x")
|
||||
},
|
||||
0,
|
||||
Granularities.NONE,
|
||||
100,
|
||||
false
|
||||
);
|
||||
|
||||
ImmutableList<Segment> segments = ImmutableList.of(
|
||||
new IncrementalIndexSegment(index, SegmentId.dummy("test")),
|
||||
helper.persistIncrementalIndex(index, null)
|
||||
);
|
||||
|
||||
GroupByQuery query = new GroupByQuery.Builder()
|
||||
.setDataSource("test")
|
||||
.setGranularity(Granularities.HOUR)
|
||||
.setInterval("1970/2050")
|
||||
.setAggregatorSpecs(
|
||||
new DoubleSumAggregatorFactory("doubleSum", "histogram")
|
||||
).build();
|
||||
|
||||
Sequence<ResultRow> seq = helper.runQueryOnSegmentsObjs(segments, query);
|
||||
|
||||
List<ResultRow> results = seq.toList();
|
||||
Assert.assertEquals(1, results.size());
|
||||
// Check timestamp
|
||||
Assert.assertEquals(startOfDay.getMillis(), results.get(0).get(0));
|
||||
// Check doubleSum
|
||||
Assert.assertEquals(n * segments.size(), (Double) results.get(0).get(1), 0.001);
|
||||
}
|
||||
|
||||
private static void assertResultsMatch(List<ResultRow> results, int rowNum, String expectedProduct)
|
||||
{
|
||||
ResultRow row = results.get(rowNum);
|
||||
|
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.spectator.histogram;
|
||||
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class SpectatorHistogramIndexBasedComplexColumnTest
|
||||
{
|
||||
@Test
|
||||
public void testComplexColumn()
|
||||
{
|
||||
final SpectatorHistogramIndexed mockIndexed = EasyMock.createMock(SpectatorHistogramIndexed.class);
|
||||
EasyMock.replay(mockIndexed);
|
||||
|
||||
final String typeName = "type";
|
||||
final SpectatorHistogramIndexBasedComplexColumn column =
|
||||
new SpectatorHistogramIndexBasedComplexColumn("type", mockIndexed);
|
||||
Assert.assertEquals(typeName, column.getTypeName());
|
||||
Assert.assertEquals(-1, column.getLength());
|
||||
|
||||
EasyMock.verify(mockIndexed);
|
||||
}
|
||||
}
|
|
@ -204,10 +204,6 @@
|
|||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes>
|
||||
<!-- Ignore firehose code -->
|
||||
<exclude>org/apache/druid/firehose/azure/**/*</exclude> <!-- integration-tests -->
|
||||
</excludes>
|
||||
<rules>
|
||||
<rule>
|
||||
<element>BUNDLE</element>
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import org.apache.datasketches.hll.HllSketch;
|
||||
import org.apache.datasketches.hll.TgtHllType;
|
||||
import org.apache.datasketches.hll.Union;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.java.util.common.StringEncoding;
|
||||
import org.apache.druid.query.aggregation.Aggregator;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
|
@ -34,7 +35,9 @@ import org.apache.druid.query.aggregation.VectorAggregator;
|
|||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -107,6 +110,8 @@ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory
|
|||
@Override
|
||||
public Aggregator factorize(final ColumnSelectorFactory columnSelectorFactory)
|
||||
{
|
||||
validateInputs(columnSelectorFactory.getColumnCapabilities(getFieldName()));
|
||||
|
||||
final ColumnValueSelector<HllSketchHolder> selector = columnSelectorFactory.makeColumnValueSelector(getFieldName());
|
||||
return new HllSketchMergeAggregator(selector, getLgK(), TgtHllType.valueOf(getTgtHllType()));
|
||||
}
|
||||
|
@ -115,6 +120,8 @@ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory
|
|||
@Override
|
||||
public BufferAggregator factorizeBuffered(final ColumnSelectorFactory columnSelectorFactory)
|
||||
{
|
||||
validateInputs(columnSelectorFactory.getColumnCapabilities(getFieldName()));
|
||||
|
||||
final ColumnValueSelector<HllSketchHolder> selector = columnSelectorFactory.makeColumnValueSelector(getFieldName());
|
||||
return new HllSketchMergeBufferAggregator(
|
||||
selector,
|
||||
|
@ -133,6 +140,7 @@ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory
|
|||
@Override
|
||||
public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory)
|
||||
{
|
||||
validateInputs(selectorFactory.getColumnCapabilities(getFieldName()));
|
||||
return new HllSketchMergeVectorAggregator(
|
||||
selectorFactory,
|
||||
getFieldName(),
|
||||
|
@ -142,6 +150,34 @@ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory
|
|||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates whether the aggregator supports the input column type.
|
||||
* Supported column types are complex types of HLLSketch, HLLSketchBuild, HLLSketchMerge, as well as UNKNOWN_COMPLEX.
|
||||
* @param capabilities
|
||||
*/
|
||||
private void validateInputs(@Nullable ColumnCapabilities capabilities)
|
||||
{
|
||||
if (capabilities != null) {
|
||||
final ColumnType type = capabilities.toColumnType();
|
||||
boolean isSupportedComplexType = ValueType.COMPLEX.equals(type.getType()) &&
|
||||
(
|
||||
HllSketchModule.TYPE_NAME.equals(type.getComplexTypeName()) ||
|
||||
HllSketchModule.BUILD_TYPE_NAME.equals(type.getComplexTypeName()) ||
|
||||
HllSketchModule.MERGE_TYPE_NAME.equals(type.getComplexTypeName()) ||
|
||||
type.getComplexTypeName() == null
|
||||
);
|
||||
if (!isSupportedComplexType) {
|
||||
throw DruidException.forPersona(DruidException.Persona.USER)
|
||||
.ofCategory(DruidException.Category.UNSUPPORTED)
|
||||
.build(
|
||||
"Using aggregator [%s] is not supported for complex columns with type [%s].",
|
||||
getIntermediateType().getComplexTypeName(),
|
||||
type
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxIntermediateSize()
|
||||
{
|
||||
|
|
|
@ -21,28 +21,68 @@ package org.apache.druid.query.aggregation.datasketches.hll.sql;
|
|||
|
||||
import org.apache.calcite.sql.SqlAggFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.type.CastedLiteralOperandTypeCheckers;
|
||||
import org.apache.calcite.sql.type.InferTypes;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.druid.java.util.common.StringEncoding;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchMergeAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
|
||||
import org.apache.druid.sql.calcite.aggregation.Aggregation;
|
||||
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
||||
import org.apache.druid.sql.calcite.table.RowSignatures;
|
||||
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* Approximate count distinct aggregator using HLL sketches.
|
||||
* Supported column types: String, Numeric, HLLSketchMerge, HLLSketchBuild.
|
||||
*/
|
||||
public class HllSketchApproxCountDistinctSqlAggregator extends HllSketchBaseSqlAggregator implements SqlAggregator
|
||||
{
|
||||
public static final String NAME = "APPROX_COUNT_DISTINCT_DS_HLL";
|
||||
|
||||
private static final SqlSingleOperandTypeChecker AGGREGATED_COLUMN_TYPE_CHECKER = OperandTypes.or(
|
||||
OperandTypes.STRING,
|
||||
OperandTypes.NUMERIC,
|
||||
RowSignatures.complexTypeChecker(HllSketchMergeAggregatorFactory.TYPE),
|
||||
RowSignatures.complexTypeChecker(HllSketchBuildAggregatorFactory.TYPE)
|
||||
);
|
||||
|
||||
private static final SqlAggFunction FUNCTION_INSTANCE =
|
||||
OperatorConversions.aggregatorBuilder(NAME)
|
||||
.operandNames("column", "lgK", "tgtHllType")
|
||||
.operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING)
|
||||
.operandTypeInference(InferTypes.VARCHAR_1024)
|
||||
.requiredOperandCount(1)
|
||||
.literalOperands(1, 2)
|
||||
.operandTypeChecker(
|
||||
OperandTypes.or(
|
||||
// APPROX_COUNT_DISTINCT_DS_HLL(column)
|
||||
AGGREGATED_COLUMN_TYPE_CHECKER,
|
||||
// APPROX_COUNT_DISTINCT_DS_HLL(column, lgk)
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(
|
||||
StringUtils.format("'%s(column, lgk)'", NAME),
|
||||
AGGREGATED_COLUMN_TYPE_CHECKER,
|
||||
CastedLiteralOperandTypeCheckers.POSITIVE_INTEGER_LITERAL
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.EXACT_NUMERIC)
|
||||
),
|
||||
// APPROX_COUNT_DISTINCT_DS_HLL(column, lgk, tgtHllType)
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(
|
||||
StringUtils.format("'%s(column, lgk, tgtHllType)'", NAME),
|
||||
AGGREGATED_COLUMN_TYPE_CHECKER,
|
||||
CastedLiteralOperandTypeCheckers.POSITIVE_INTEGER_LITERAL,
|
||||
OperandTypes.STRING
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.EXACT_NUMERIC, SqlTypeFamily.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
.returnTypeNonNull(SqlTypeName.BIGINT)
|
||||
.functionCategory(SqlFunctionCategory.NUMERIC)
|
||||
.build();
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.druid.query.aggregation.datasketches.SketchQueryContext;
|
|||
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchMergeAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchModule;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -40,6 +41,7 @@ import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
|
|||
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||
import org.apache.druid.sql.calcite.expression.Expressions;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.rel.InputAccessor;
|
||||
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
|
||||
|
@ -115,7 +117,7 @@ public abstract class HllSketchBaseSqlAggregator implements SqlAggregator
|
|||
if (columnArg.isDirectColumnAccess()
|
||||
&& inputAccessor.getInputRowSignature()
|
||||
.getColumnType(columnArg.getDirectColumn())
|
||||
.map(type -> type.is(ValueType.COMPLEX))
|
||||
.map(this::isValidComplexInputType)
|
||||
.orElse(false)) {
|
||||
aggregatorFactory = new HllSketchMergeAggregatorFactory(
|
||||
aggregatorName,
|
||||
|
@ -154,6 +156,15 @@ public abstract class HllSketchBaseSqlAggregator implements SqlAggregator
|
|||
}
|
||||
|
||||
if (inputType.is(ValueType.COMPLEX)) {
|
||||
if (!isValidComplexInputType(inputType)) {
|
||||
plannerContext.setPlanningError(
|
||||
"Using APPROX_COUNT_DISTINCT() or enabling approximation with COUNT(DISTINCT) is not supported for"
|
||||
+ " column type [%s]. You can disable approximation by setting [%s: false] in the query context.",
|
||||
columnArg.getDruidType(),
|
||||
PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT
|
||||
);
|
||||
return null;
|
||||
}
|
||||
aggregatorFactory = new HllSketchMergeAggregatorFactory(
|
||||
aggregatorName,
|
||||
dimensionSpec.getOutputName(),
|
||||
|
@ -192,4 +203,11 @@ public abstract class HllSketchBaseSqlAggregator implements SqlAggregator
|
|||
boolean finalizeAggregations,
|
||||
AggregatorFactory aggregatorFactory
|
||||
);
|
||||
|
||||
private boolean isValidComplexInputType(ColumnType columnType)
|
||||
{
|
||||
return HllSketchMergeAggregatorFactory.TYPE.equals(columnType) ||
|
||||
HllSketchModule.TYPE_NAME.equals(columnType.getComplexTypeName()) ||
|
||||
HllSketchModule.BUILD_TYPE_NAME.equals(columnType.getComplexTypeName());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.datasketches.common.Util;
|
|||
import org.apache.datasketches.theta.SetOperation;
|
||||
import org.apache.datasketches.theta.Union;
|
||||
import org.apache.datasketches.thetacommon.ThetaUtil;
|
||||
import org.apache.druid.error.InvalidInput;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.aggregation.AggregateCombiner;
|
||||
import org.apache.druid.query.aggregation.Aggregator;
|
||||
|
@ -41,6 +41,7 @@ import org.apache.druid.segment.ColumnInspector;
|
|||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -80,10 +81,7 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName);
|
||||
if (capabilities != null && capabilities.isArray()) {
|
||||
throw InvalidInput.exception("ARRAY types are not supported for theta sketch");
|
||||
}
|
||||
validateInputs(metricFactory.getColumnCapabilities(fieldName));
|
||||
BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
return new SketchAggregator(selector, size);
|
||||
}
|
||||
|
@ -91,10 +89,7 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName);
|
||||
if (capabilities != null && capabilities.isArray()) {
|
||||
throw InvalidInput.exception("ARRAY types are not supported for theta sketch");
|
||||
}
|
||||
validateInputs(metricFactory.getColumnCapabilities(fieldName));
|
||||
BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
final SketchAggregator aggregator = new SketchAggregator(selector, size);
|
||||
return new AggregatorAndSize(aggregator, aggregator.getInitialSizeBytes());
|
||||
|
@ -104,10 +99,7 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName);
|
||||
if (capabilities != null && capabilities.isArray()) {
|
||||
throw InvalidInput.exception("ARRAY types are not supported for theta sketch");
|
||||
}
|
||||
validateInputs(metricFactory.getColumnCapabilities(fieldName));
|
||||
BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
return new SketchBufferAggregator(selector, size, getMaxIntermediateSizeWithNulls());
|
||||
}
|
||||
|
@ -115,9 +107,41 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory)
|
||||
{
|
||||
validateInputs(selectorFactory.getColumnCapabilities(fieldName));
|
||||
return new SketchVectorAggregator(selectorFactory, fieldName, size, getMaxIntermediateSizeWithNulls());
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates whether the aggregator supports the input column type.
|
||||
* Unsupported column types are:
|
||||
* <ul>
|
||||
* <li>Arrays</li>
|
||||
* <li>Complex types of thetaSketch, thetaSketchMerge, thetaSketchBuild.</li>
|
||||
* </ul>
|
||||
* @param capabilities
|
||||
*/
|
||||
private void validateInputs(@Nullable ColumnCapabilities capabilities)
|
||||
{
|
||||
if (capabilities != null) {
|
||||
boolean isUnsupportedComplexType = capabilities.is(ValueType.COMPLEX) && !(
|
||||
SketchModule.THETA_SKETCH_TYPE.equals(capabilities.toColumnType()) ||
|
||||
SketchModule.MERGE_TYPE.equals(capabilities.toColumnType()) ||
|
||||
SketchModule.BUILD_TYPE.equals(capabilities.toColumnType())
|
||||
);
|
||||
|
||||
if (capabilities.isArray() || isUnsupportedComplexType) {
|
||||
throw DruidException.forPersona(DruidException.Persona.USER)
|
||||
.ofCategory(DruidException.Category.UNSUPPORTED)
|
||||
.build(
|
||||
"Unsupported input [%s] of type [%s] for aggregator [%s].",
|
||||
getFieldName(),
|
||||
capabilities.asTypeString(),
|
||||
getIntermediateType()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(ColumnInspector columnInspector)
|
||||
{
|
||||
|
|
|
@ -21,27 +21,55 @@ package org.apache.druid.query.aggregation.datasketches.theta.sql;
|
|||
|
||||
import org.apache.calcite.sql.SqlAggFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.type.CastedLiteralOperandTypeCheckers;
|
||||
import org.apache.calcite.sql.type.InferTypes;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.theta.SketchModule;
|
||||
import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
|
||||
import org.apache.druid.sql.calcite.aggregation.Aggregation;
|
||||
import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
|
||||
import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
||||
import org.apache.druid.sql.calcite.table.RowSignatures;
|
||||
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* Approximate count distinct aggregator using theta sketches.
|
||||
* Supported column types: String, Numeric, Theta Sketch.
|
||||
*/
|
||||
public class ThetaSketchApproxCountDistinctSqlAggregator extends ThetaSketchBaseSqlAggregator implements SqlAggregator
|
||||
{
|
||||
public static final String NAME = "APPROX_COUNT_DISTINCT_DS_THETA";
|
||||
|
||||
private static final SqlSingleOperandTypeChecker AGGREGATED_COLUMN_TYPE_CHECKER = OperandTypes.or(
|
||||
OperandTypes.STRING,
|
||||
OperandTypes.NUMERIC,
|
||||
RowSignatures.complexTypeChecker(SketchModule.THETA_SKETCH_TYPE)
|
||||
);
|
||||
|
||||
private static final SqlAggFunction FUNCTION_INSTANCE =
|
||||
OperatorConversions.aggregatorBuilder(NAME)
|
||||
.operandNames("column", "size")
|
||||
.operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC)
|
||||
.operandTypeInference(InferTypes.VARCHAR_1024)
|
||||
.requiredOperandCount(1)
|
||||
.literalOperands(1)
|
||||
.operandTypeChecker(
|
||||
OperandTypes.or(
|
||||
// APPROX_COUNT_DISTINCT_DS_THETA(expr)
|
||||
AGGREGATED_COLUMN_TYPE_CHECKER,
|
||||
// APPROX_COUNT_DISTINCT_DS_THETA(expr, size)
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(
|
||||
StringUtils.format("'%s(expr, size)'", NAME),
|
||||
AGGREGATED_COLUMN_TYPE_CHECKER,
|
||||
CastedLiteralOperandTypeCheckers.POSITIVE_INTEGER_LITERAL
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.EXACT_NUMERIC)
|
||||
)
|
||||
)
|
||||
)
|
||||
.returnTypeNonNull(SqlTypeName.BIGINT)
|
||||
.functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION)
|
||||
.build();
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
|
|||
import org.apache.druid.query.aggregation.datasketches.SketchQueryContext;
|
||||
import org.apache.druid.query.aggregation.datasketches.theta.SketchAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.theta.SketchModule;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -38,6 +39,7 @@ import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
|
|||
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||
import org.apache.druid.sql.calcite.expression.Expressions;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.rel.InputAccessor;
|
||||
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
|
||||
|
@ -95,7 +97,11 @@ public abstract class ThetaSketchBaseSqlAggregator implements SqlAggregator
|
|||
if (columnArg.isDirectColumnAccess()
|
||||
&& inputAccessor.getInputRowSignature()
|
||||
.getColumnType(columnArg.getDirectColumn())
|
||||
.map(type -> type.is(ValueType.COMPLEX))
|
||||
.map(type -> (
|
||||
SketchModule.THETA_SKETCH_TYPE.equals(type) ||
|
||||
SketchModule.MERGE_TYPE.equals(type) ||
|
||||
SketchModule.BUILD_TYPE.equals(type)
|
||||
))
|
||||
.orElse(false)) {
|
||||
aggregatorFactory = new SketchMergeAggregatorFactory(
|
||||
aggregatorName,
|
||||
|
@ -116,6 +122,16 @@ public abstract class ThetaSketchBaseSqlAggregator implements SqlAggregator
|
|||
);
|
||||
}
|
||||
|
||||
if (inputType.is(ValueType.COMPLEX)) {
|
||||
plannerContext.setPlanningError(
|
||||
"Using APPROX_COUNT_DISTINCT() or enabling approximation with COUNT(DISTINCT) is not supported for"
|
||||
+ " column type [%s]. You can disable approximation by setting [%s: false] in the query context.",
|
||||
columnArg.getDruidType(),
|
||||
PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT
|
||||
);
|
||||
return null;
|
||||
}
|
||||
|
||||
final DimensionSpec dimensionSpec;
|
||||
|
||||
if (columnArg.isDirectColumnAccess()) {
|
||||
|
|
|
@ -22,10 +22,17 @@ package org.apache.druid.query.aggregation.datasketches.hll;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||
import org.apache.datasketches.hll.TgtHllType;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.java.util.common.StringEncoding;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.TestColumnSelectorFactory;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.vector.TestVectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -45,6 +52,9 @@ public class HllSketchMergeAggregatorFactoryTest
|
|||
private HllSketchMergeAggregatorFactory targetRound;
|
||||
private HllSketchMergeAggregatorFactory targetNoRound;
|
||||
|
||||
private ColumnSelectorFactory metricFactory;
|
||||
private VectorColumnSelectorFactory vectorFactory;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
|
@ -66,6 +76,10 @@ public class HllSketchMergeAggregatorFactoryTest
|
|||
SHOULD_FINALIZE,
|
||||
!ROUND
|
||||
);
|
||||
|
||||
final ColumnCapabilitiesImpl columnCapabilities = ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA);
|
||||
metricFactory = new TestColumnSelectorFactory().addCapabilities(FIELD_NAME, columnCapabilities);
|
||||
vectorFactory = new TestVectorColumnSelectorFactory().addCapabilities(FIELD_NAME, columnCapabilities);
|
||||
}
|
||||
|
||||
@Test(expected = AggregatorFactoryNotMergeableException.class)
|
||||
|
@ -291,4 +305,39 @@ public class HllSketchMergeAggregatorFactoryTest
|
|||
Assert.assertEquals(factory, factory.withName(targetRound.getName()));
|
||||
Assert.assertEquals("newTest", factory.withName("newTest").getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFactorizeOnUnsupportedComplexColumn()
|
||||
{
|
||||
final ColumnSelectorFactory metricFactory = new TestColumnSelectorFactory()
|
||||
.addCapabilities(
|
||||
FIELD_NAME,
|
||||
ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA)
|
||||
);
|
||||
Throwable exception = Assert.assertThrows(DruidException.class, () -> targetRound.factorize(metricFactory));
|
||||
Assert.assertEquals(
|
||||
"Using aggregator [HLLSketchMerge] is not supported for complex columns with type [COMPLEX<json>].",
|
||||
exception.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFactorizeBufferedOnUnsupportedComplexColumn()
|
||||
{
|
||||
Throwable exception = Assert.assertThrows(DruidException.class, () -> targetRound.factorizeBuffered(metricFactory));
|
||||
Assert.assertEquals(
|
||||
"Using aggregator [HLLSketchMerge] is not supported for complex columns with type [COMPLEX<json>].",
|
||||
exception.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFactorizeVectorOnUnsupportedComplexColumn()
|
||||
{
|
||||
Throwable exception = Assert.assertThrows(DruidException.class, () -> targetRound.factorizeVector(vectorFactory));
|
||||
Assert.assertEquals(
|
||||
"Using aggregator [HLLSketchMerge] is not supported for complex columns with type [COMPLEX<json>].",
|
||||
exception.getMessage()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,9 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.inject.Injector;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.guice.DruidInjectorBuilder;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.StringEncoding;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
|
@ -86,6 +88,7 @@ import org.apache.druid.sql.calcite.util.TestDataBuilder;
|
|||
import org.apache.druid.sql.guice.SqlModule;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.Assert;
|
||||
|
@ -100,6 +103,10 @@ import java.util.stream.Collectors;
|
|||
@SqlTestFrameworkConfig.ComponentSupplier(HllSketchComponentSupplier.class)
|
||||
public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
||||
{
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
}
|
||||
|
||||
private static final boolean ROUND = true;
|
||||
|
||||
// For testHllSketchPostAggsGroupBy, testHllSketchPostAggsTimeseries
|
||||
|
@ -300,6 +307,15 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
.size(0)
|
||||
.build(),
|
||||
index
|
||||
).add(
|
||||
DataSegment.builder()
|
||||
.dataSource(CalciteTests.WIKIPEDIA_FIRST_LAST)
|
||||
.interval(Intervals.of("2015-09-12/2015-09-13"))
|
||||
.version("1")
|
||||
.shardSpec(new NumberedShardSpec(0, 0))
|
||||
.size(0)
|
||||
.build(),
|
||||
TestDataBuilder.makeWikipediaIndexWithAggregation(tempDirProducer.newTempFolder())
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -508,6 +524,33 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testApproxCountDistinctOnUnsupportedComplexColumn()
|
||||
{
|
||||
assertQueryIsUnplannable(
|
||||
"SELECT COUNT(distinct double_first_added) FROM druid.wikipedia_first_last",
|
||||
"Query could not be planned. A possible reason is [Using APPROX_COUNT_DISTINCT() or enabling "
|
||||
+ "approximation with COUNT(DISTINCT) is not supported for column type [COMPLEX<serializablePairLongDouble>]."
|
||||
+ " You can disable approximation by setting [useApproximateCountDistinct: false] in the query context."
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testApproxCountDistinctFunctionOnUnsupportedComplexColumn()
|
||||
{
|
||||
DruidException druidException = Assert.assertThrows(
|
||||
DruidException.class,
|
||||
() -> testQuery(
|
||||
"SELECT APPROX_COUNT_DISTINCT_DS_HLL(double_first_added) FROM druid.wikipedia_first_last",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of()
|
||||
)
|
||||
);
|
||||
Assert.assertTrue(druidException.getMessage().contains(
|
||||
"Cannot apply 'APPROX_COUNT_DISTINCT_DS_HLL' to arguments of type 'APPROX_COUNT_DISTINCT_DS_HLL(<COMPLEX<SERIALIZABLEPAIRLONGDOUBLE>>)'"
|
||||
));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHllSketchFilteredAggregatorsGroupBy()
|
||||
{
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.query.aggregation.datasketches.theta;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.aggregation.AggregatorAndSize;
|
||||
|
@ -32,10 +33,15 @@ import org.apache.druid.query.timeseries.TimeseriesQuery;
|
|||
import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.TestColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.vector.TestVectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class SketchAggregatorFactoryTest
|
||||
|
@ -46,6 +52,17 @@ public class SketchAggregatorFactoryTest
|
|||
private static final SketchMergeAggregatorFactory AGGREGATOR_32768 =
|
||||
new SketchMergeAggregatorFactory("x", "x", 32768, null, false, null);
|
||||
|
||||
private ColumnSelectorFactory metricFactory;
|
||||
private VectorColumnSelectorFactory vectorFactory;
|
||||
|
||||
@Before
|
||||
public void setup()
|
||||
{
|
||||
final ColumnCapabilitiesImpl columnCapabilities = ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA);
|
||||
metricFactory = new TestColumnSelectorFactory().addCapabilities("x", columnCapabilities);
|
||||
vectorFactory = new TestVectorColumnSelectorFactory().addCapabilities("x", columnCapabilities);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGuessAggregatorHeapFootprint()
|
||||
{
|
||||
|
@ -168,4 +185,32 @@ public class SketchAggregatorFactoryTest
|
|||
Assert.assertEquals(AGGREGATOR_16384, AGGREGATOR_16384.withName("x"));
|
||||
Assert.assertEquals("newTest", AGGREGATOR_16384.withName("newTest").getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFactorizeOnUnsupportedComplexColumn()
|
||||
{
|
||||
Throwable exception = Assert.assertThrows(DruidException.class, () -> AGGREGATOR_16384.factorize(metricFactory));
|
||||
Assert.assertEquals("Unsupported input [x] of type [COMPLEX<json>] for aggregator [COMPLEX<thetaSketchBuild>].", exception.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFactorizeWithSizeOnUnsupportedComplexColumn()
|
||||
{
|
||||
Throwable exception = Assert.assertThrows(DruidException.class, () -> AGGREGATOR_16384.factorizeWithSize(metricFactory));
|
||||
Assert.assertEquals("Unsupported input [x] of type [COMPLEX<json>] for aggregator [COMPLEX<thetaSketchBuild>].", exception.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFactorizeBufferedOnUnsupportedComplexColumn()
|
||||
{
|
||||
Throwable exception = Assert.assertThrows(DruidException.class, () -> AGGREGATOR_16384.factorizeBuffered(metricFactory));
|
||||
Assert.assertEquals("Unsupported input [x] of type [COMPLEX<json>] for aggregator [COMPLEX<thetaSketchBuild>].", exception.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFactorizeVectorOnUnsupportedComplexColumn()
|
||||
{
|
||||
Throwable exception = Assert.assertThrows(DruidException.class, () -> AGGREGATOR_16384.factorizeVector(vectorFactory));
|
||||
Assert.assertEquals("Unsupported input [x] of type [COMPLEX<json>] for aggregator [COMPLEX<thetaSketchBuild>].", exception.getMessage());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,9 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.inject.Injector;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.guice.DruidInjectorBuilder;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
|
||||
|
@ -71,6 +73,7 @@ import org.apache.druid.sql.calcite.util.TestDataBuilder;
|
|||
import org.apache.druid.sql.guice.SqlModule;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.Assert;
|
||||
|
@ -158,6 +161,15 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
.size(0)
|
||||
.build(),
|
||||
index
|
||||
).add(
|
||||
DataSegment.builder()
|
||||
.dataSource(CalciteTests.WIKIPEDIA_FIRST_LAST)
|
||||
.interval(Intervals.of("2015-09-12/2015-09-13"))
|
||||
.version("1")
|
||||
.shardSpec(new NumberedShardSpec(0, 0))
|
||||
.size(0)
|
||||
.build(),
|
||||
TestDataBuilder.makeWikipediaIndexWithAggregation(tempDirProducer.newTempFolder())
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -373,6 +385,33 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testApproxCountDistinctOnUnsupportedComplexColumn()
|
||||
{
|
||||
assertQueryIsUnplannable(
|
||||
"SELECT COUNT(distinct double_first_added) FROM druid.wikipedia_first_last",
|
||||
"Query could not be planned. A possible reason is [Using APPROX_COUNT_DISTINCT() or enabling "
|
||||
+ "approximation with COUNT(DISTINCT) is not supported for column type [COMPLEX<serializablePairLongDouble>]."
|
||||
+ " You can disable approximation by setting [useApproximateCountDistinct: false] in the query context."
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testApproxCountDistinctFunctionOnUnsupportedComplexColumn()
|
||||
{
|
||||
DruidException druidException = Assert.assertThrows(
|
||||
DruidException.class,
|
||||
() -> testQuery(
|
||||
"SELECT APPROX_COUNT_DISTINCT_DS_THETA(double_first_added) FROM druid.wikipedia_first_last",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of()
|
||||
)
|
||||
);
|
||||
Assert.assertTrue(druidException.getMessage().contains(
|
||||
"Cannot apply 'APPROX_COUNT_DISTINCT_DS_THETA' to arguments of type 'APPROX_COUNT_DISTINCT_DS_THETA(<COMPLEX<SERIALIZABLEPAIRLONGDOUBLE>>)'"
|
||||
));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testThetaSketchPostAggs()
|
||||
{
|
||||
|
|
|
@ -267,7 +267,7 @@ public class CoordinatorPollingBasicAuthenticatorCacheManager implements BasicAu
|
|||
writeUserMapToDisk(prefix, userMapBytes);
|
||||
}
|
||||
} else {
|
||||
LOG.info("Empty cached serialized user map retrieved, authenticator - %s", prefix);
|
||||
LOG.debug("Empty cached serialized user map retrieved, authenticator - %s", prefix);
|
||||
}
|
||||
return userMap;
|
||||
}
|
||||
|
|
|
@ -2908,10 +2908,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
.put(
|
||||
ScanQuery.class,
|
||||
new ScanQueryRunnerFactory(
|
||||
new ScanQueryQueryToolChest(
|
||||
new ScanQueryConfig(),
|
||||
new DefaultGenericQueryMetricsFactory()
|
||||
),
|
||||
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
|
||||
new ScanQueryEngine(),
|
||||
new ScanQueryConfig()
|
||||
)
|
||||
|
|
|
@ -33,9 +33,9 @@ import org.apache.druid.initialization.DruidModule;
|
|||
import org.apache.druid.initialization.Initialization;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.realtime.ChatHandlerProvider;
|
||||
import org.apache.druid.segment.realtime.NoopChatHandlerProvider;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
|
||||
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
|
||||
import org.apache.druid.server.security.Action;
|
||||
import org.apache.druid.server.security.Resource;
|
||||
import org.apache.druid.server.security.ResourceAction;
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.msq.exec;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
@ -1357,7 +1356,10 @@ public class ControllerImpl implements Controller
|
|||
* Publish the list of segments. Additionally, if {@link DataSourceMSQDestination#isReplaceTimeChunks()},
|
||||
* also drop all other segments within the replacement intervals.
|
||||
*/
|
||||
private void publishAllSegments(final Set<DataSegment> segments) throws IOException
|
||||
private void publishAllSegments(
|
||||
final Set<DataSegment> segments,
|
||||
Function<Set<DataSegment>, Set<DataSegment>> compactionStateAnnotateFunction
|
||||
) throws IOException
|
||||
{
|
||||
final DataSourceMSQDestination destination =
|
||||
(DataSourceMSQDestination) querySpec.getDestination();
|
||||
|
@ -1413,7 +1415,7 @@ public class ControllerImpl implements Controller
|
|||
}
|
||||
performSegmentPublish(
|
||||
context.taskActionClient(),
|
||||
createOverwriteAction(taskLockType, segmentsWithTombstones)
|
||||
createOverwriteAction(taskLockType, compactionStateAnnotateFunction.apply(segmentsWithTombstones))
|
||||
);
|
||||
}
|
||||
} else if (!segments.isEmpty()) {
|
||||
|
@ -1543,6 +1545,7 @@ public class ControllerImpl implements Controller
|
|||
if (MSQControllerTask.isIngestion(querySpec)) {
|
||||
// Publish segments if needed.
|
||||
final StageId finalStageId = queryKernel.getStageId(queryDef.getFinalStageDefinition().getStageNumber());
|
||||
Function<Set<DataSegment>, Set<DataSegment>> compactionStateAnnotateFunction = Function.identity();
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
Set<DataSegment> segments = (Set<DataSegment>) queryKernel.getResultObjectForStage(finalStageId);
|
||||
|
@ -1553,7 +1556,7 @@ public class ControllerImpl implements Controller
|
|||
Tasks.DEFAULT_STORE_COMPACTION_STATE
|
||||
);
|
||||
|
||||
if (!segments.isEmpty() && storeCompactionState) {
|
||||
if (storeCompactionState) {
|
||||
DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination();
|
||||
if (!destination.isReplaceTimeChunks()) {
|
||||
// Store compaction state only for replace queries.
|
||||
|
@ -1565,20 +1568,21 @@ public class ControllerImpl implements Controller
|
|||
DataSchema dataSchema = ((SegmentGeneratorFrameProcessorFactory) queryKernel
|
||||
.getStageDefinition(finalStageId).getProcessorFactory()).getDataSchema();
|
||||
|
||||
ShardSpec shardSpec = segments.stream().findFirst().get().getShardSpec();
|
||||
ShardSpec shardSpec = segments.isEmpty() ? null : segments.stream().findFirst().get().getShardSpec();
|
||||
ClusterBy clusterBy = queryKernel.getStageDefinition(finalStageId).getClusterBy();
|
||||
|
||||
Function<Set<DataSegment>, Set<DataSegment>> compactionStateAnnotateFunction = addCompactionStateToSegments(
|
||||
compactionStateAnnotateFunction = addCompactionStateToSegments(
|
||||
querySpec,
|
||||
context.jsonMapper(),
|
||||
dataSchema,
|
||||
shardSpec,
|
||||
clusterBy,
|
||||
queryDef.getQueryId()
|
||||
);
|
||||
segments = compactionStateAnnotateFunction.apply(segments);
|
||||
}
|
||||
}
|
||||
log.info("Query [%s] publishing %d segments.", queryDef.getQueryId(), segments.size());
|
||||
publishAllSegments(segments);
|
||||
publishAllSegments(segments, compactionStateAnnotateFunction);
|
||||
} else if (MSQControllerTask.isExport(querySpec)) {
|
||||
// Write manifest file.
|
||||
ExportMSQDestination destination = (ExportMSQDestination) querySpec.getDestination();
|
||||
|
@ -1624,33 +1628,49 @@ public class ControllerImpl implements Controller
|
|||
MSQSpec querySpec,
|
||||
ObjectMapper jsonMapper,
|
||||
DataSchema dataSchema,
|
||||
ShardSpec shardSpec,
|
||||
@Nullable ShardSpec shardSpec,
|
||||
@Nullable ClusterBy clusterBy,
|
||||
String queryId
|
||||
)
|
||||
{
|
||||
final MSQTuningConfig tuningConfig = querySpec.getTuningConfig();
|
||||
PartitionsSpec partitionSpec;
|
||||
|
||||
if (Objects.equals(shardSpec.getType(), ShardSpec.Type.RANGE)) {
|
||||
List<String> partitionDimensions = ((DimensionRangeShardSpec) shardSpec).getDimensions();
|
||||
// shardSpec is absent in the absence of segments, which happens when only tombstones are generated by an
|
||||
// MSQControllerTask.
|
||||
if (shardSpec != null) {
|
||||
if (Objects.equals(shardSpec.getType(), ShardSpec.Type.RANGE)) {
|
||||
List<String> partitionDimensions = ((DimensionRangeShardSpec) shardSpec).getDimensions();
|
||||
partitionSpec = new DimensionRangePartitionsSpec(
|
||||
tuningConfig.getRowsPerSegment(),
|
||||
null,
|
||||
partitionDimensions,
|
||||
false
|
||||
);
|
||||
} else if (Objects.equals(shardSpec.getType(), ShardSpec.Type.NUMBERED)) {
|
||||
// MSQ tasks don't use maxTotalRows. Hence using LONG.MAX_VALUE.
|
||||
partitionSpec = new DynamicPartitionsSpec(tuningConfig.getRowsPerSegment(), Long.MAX_VALUE);
|
||||
} else {
|
||||
// SingleDimenionShardSpec and other shard specs are never created in MSQ.
|
||||
throw new MSQException(
|
||||
UnknownFault.forMessage(
|
||||
StringUtils.format(
|
||||
"Query[%s] cannot store compaction state in segments as shard spec of unsupported type[%s].",
|
||||
queryId,
|
||||
shardSpec.getType()
|
||||
)));
|
||||
}
|
||||
} else if (clusterBy != null && !clusterBy.getColumns().isEmpty()) {
|
||||
partitionSpec = new DimensionRangePartitionsSpec(
|
||||
tuningConfig.getRowsPerSegment(),
|
||||
null,
|
||||
partitionDimensions,
|
||||
clusterBy.getColumns()
|
||||
.stream()
|
||||
.map(KeyColumn::columnName).collect(Collectors.toList()),
|
||||
false
|
||||
);
|
||||
} else if (Objects.equals(shardSpec.getType(), ShardSpec.Type.NUMBERED)) {
|
||||
// MSQ tasks don't use maxTotalRows. Hence using LONG.MAX_VALUE.
|
||||
partitionSpec = new DynamicPartitionsSpec(tuningConfig.getRowsPerSegment(), Long.MAX_VALUE);
|
||||
} else {
|
||||
// SingleDimenionShardSpec and other shard specs are never created in MSQ.
|
||||
throw new MSQException(
|
||||
UnknownFault.forMessage(
|
||||
StringUtils.format(
|
||||
"Query[%s] cannot store compaction state in segments as shard spec of unsupported type[%s].",
|
||||
queryId,
|
||||
shardSpec.getType()
|
||||
)));
|
||||
partitionSpec = new DynamicPartitionsSpec(tuningConfig.getRowsPerSegment(), Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
Granularity segmentGranularity = ((DataSourceMSQDestination) querySpec.getDestination())
|
||||
|
@ -1671,13 +1691,26 @@ public class ControllerImpl implements Controller
|
|||
: new ClientCompactionTaskTransformSpec(
|
||||
dataSchema.getTransformSpec().getFilter()
|
||||
).asMap(jsonMapper);
|
||||
List<Object> metricsSpec = dataSchema.getAggregators() == null
|
||||
? null
|
||||
: jsonMapper.convertValue(
|
||||
dataSchema.getAggregators(),
|
||||
new TypeReference<List<Object>>() {}
|
||||
);
|
||||
List<Object> metricsSpec = Collections.emptyList();
|
||||
|
||||
if (querySpec.getQuery() instanceof GroupByQuery) {
|
||||
// For group-by queries, the aggregators are transformed to their combining factories in the dataschema, resulting
|
||||
// in a mismatch between schema in compaction spec and the one in compaction state. Sourcing the original
|
||||
// AggregatorFactory definition for aggregators in the dataSchema, therefore, directly from the querySpec.
|
||||
GroupByQuery groupByQuery = (GroupByQuery) querySpec.getQuery();
|
||||
// Collect all aggregators that are part of the current dataSchema, since a non-rollup query (isRollup() is false)
|
||||
// moves metrics columns to dimensions in the final schema.
|
||||
Set<String> aggregatorsInDataSchema = Arrays.stream(dataSchema.getAggregators())
|
||||
.map(AggregatorFactory::getName)
|
||||
.collect(
|
||||
Collectors.toSet());
|
||||
metricsSpec = new ArrayList<>(
|
||||
groupByQuery.getAggregatorSpecs()
|
||||
.stream()
|
||||
.filter(aggregatorFactory -> aggregatorsInDataSchema.contains(aggregatorFactory.getName()))
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
IndexSpec indexSpec = tuningConfig.getIndexSpec();
|
||||
|
||||
|
|
|
@ -53,7 +53,7 @@ import org.apache.druid.query.DruidMetrics;
|
|||
import org.apache.druid.query.QueryContext;
|
||||
import org.apache.druid.rpc.ServiceClientFactory;
|
||||
import org.apache.druid.rpc.indexing.OverlordClient;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandler;
|
||||
import org.apache.druid.segment.realtime.ChatHandler;
|
||||
import org.apache.druid.server.DruidNode;
|
||||
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
|
||||
|
||||
|
|
|
@ -49,7 +49,9 @@ import org.apache.druid.msq.util.MultiStageQueryContext;
|
|||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContext;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.TableDataSource;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.expression.TimestampFloorExprMacro;
|
||||
|
@ -58,11 +60,13 @@ import org.apache.druid.query.filter.DimFilter;
|
|||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.indexing.CombinedDataSchema;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.server.coordinator.CompactionConfigValidationResult;
|
||||
|
@ -123,7 +127,8 @@ public class MSQCompactionRunner implements CompactionRunner
|
|||
*/
|
||||
@Override
|
||||
public CompactionConfigValidationResult validateCompactionTask(
|
||||
CompactionTask compactionTask
|
||||
CompactionTask compactionTask,
|
||||
Map<Interval, DataSchema> intervalToDataSchemaMap
|
||||
)
|
||||
{
|
||||
List<CompactionConfigValidationResult> validationResults = new ArrayList<>();
|
||||
|
@ -139,11 +144,55 @@ public class MSQCompactionRunner implements CompactionRunner
|
|||
));
|
||||
}
|
||||
validationResults.add(ClientCompactionRunnerInfo.validateMaxNumTasksForMSQ(compactionTask.getContext()));
|
||||
validationResults.add(ClientCompactionRunnerInfo.validateMetricsSpecForMSQ(compactionTask.getMetricsSpec()));
|
||||
validationResults.add(validateRolledUpSegments(intervalToDataSchemaMap));
|
||||
return validationResults.stream()
|
||||
.filter(result -> !result.isValid())
|
||||
.findFirst()
|
||||
.orElse(new CompactionConfigValidationResult(true, null));
|
||||
.orElse(CompactionConfigValidationResult.success());
|
||||
}
|
||||
|
||||
/**
|
||||
* Valides that there are no rolled-up segments where either:
|
||||
* <ul>
|
||||
* <li>aggregator factory differs from its combining factory </li>
|
||||
* <li>input col name is different from the output name (non-idempotent)</li>
|
||||
* </ul>
|
||||
*/
|
||||
private CompactionConfigValidationResult validateRolledUpSegments(Map<Interval, DataSchema> intervalToDataSchemaMap)
|
||||
{
|
||||
for (Map.Entry<Interval, DataSchema> intervalDataSchema : intervalToDataSchemaMap.entrySet()) {
|
||||
if (intervalDataSchema.getValue() instanceof CombinedDataSchema) {
|
||||
CombinedDataSchema combinedDataSchema = (CombinedDataSchema) intervalDataSchema.getValue();
|
||||
if (combinedDataSchema.hasRolledUpSegments()) {
|
||||
for (AggregatorFactory aggregatorFactory : combinedDataSchema.getAggregators()) {
|
||||
// This is a conservative check as existing rollup may have been idempotent but the aggregator provided in
|
||||
// compaction spec isn't. This would get properly compacted yet fails in the below pre-check.
|
||||
if (
|
||||
!(
|
||||
aggregatorFactory.getClass().equals(aggregatorFactory.getCombiningFactory().getClass()) &&
|
||||
(
|
||||
aggregatorFactory.requiredFields().isEmpty() ||
|
||||
(aggregatorFactory.requiredFields().size() == 1 &&
|
||||
aggregatorFactory.requiredFields()
|
||||
.get(0)
|
||||
.equals(aggregatorFactory.getName()))
|
||||
)
|
||||
)
|
||||
) {
|
||||
// MSQ doesn't support rolling up already rolled-up segments when aggregate column name is different from
|
||||
// the aggregated column name. This is because the aggregated values would then get overwritten by new
|
||||
// values and the existing values would be lost. Note that if no rollup is specified in an index spec,
|
||||
// the default value is true.
|
||||
return CompactionConfigValidationResult.failure(
|
||||
"MSQ: Rolled-up segments in compaction interval[%s].",
|
||||
intervalDataSchema.getKey()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return CompactionConfigValidationResult.success();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -291,6 +340,10 @@ public class MSQCompactionRunner implements CompactionRunner
|
|||
for (DimensionSchema dimensionSchema : dataSchema.getDimensionsSpec().getDimensions()) {
|
||||
rowSignatureBuilder.add(dimensionSchema.getName(), ColumnType.fromString(dimensionSchema.getTypeName()));
|
||||
}
|
||||
// There can be columns that are part of metricsSpec for a datasource.
|
||||
for (AggregatorFactory aggregatorFactory : dataSchema.getAggregators()) {
|
||||
rowSignatureBuilder.add(aggregatorFactory.getName(), aggregatorFactory.getIntermediateType());
|
||||
}
|
||||
return rowSignatureBuilder.build();
|
||||
}
|
||||
|
||||
|
@ -354,15 +407,30 @@ public class MSQCompactionRunner implements CompactionRunner
|
|||
private static Query<?> buildScanQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema)
|
||||
{
|
||||
RowSignature rowSignature = getRowSignature(dataSchema);
|
||||
return new Druids.ScanQueryBuilder().dataSource(dataSchema.getDataSource())
|
||||
.columns(rowSignature.getColumnNames())
|
||||
.virtualColumns(getVirtualColumns(dataSchema, interval))
|
||||
.columnTypes(rowSignature.getColumnTypes())
|
||||
.intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval)))
|
||||
.legacy(false)
|
||||
.filters(dataSchema.getTransformSpec().getFilter())
|
||||
.context(compactionTask.getContext())
|
||||
.build();
|
||||
Druids.ScanQueryBuilder scanQueryBuilder = new Druids.ScanQueryBuilder()
|
||||
.dataSource(dataSchema.getDataSource())
|
||||
.columns(rowSignature.getColumnNames())
|
||||
.virtualColumns(getVirtualColumns(dataSchema, interval))
|
||||
.columnTypes(rowSignature.getColumnTypes())
|
||||
.intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval)))
|
||||
.filters(dataSchema.getTransformSpec().getFilter())
|
||||
.context(compactionTask.getContext());
|
||||
|
||||
if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) {
|
||||
List<OrderByColumnSpec> orderByColumnSpecs = getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec());
|
||||
|
||||
scanQueryBuilder.orderBy(
|
||||
orderByColumnSpecs
|
||||
.stream()
|
||||
.map(orderByColumnSpec ->
|
||||
new ScanQuery.OrderBy(
|
||||
orderByColumnSpec.getDimension(),
|
||||
ScanQuery.Order.fromString(orderByColumnSpec.getDirection().toString())
|
||||
))
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
return scanQueryBuilder.build();
|
||||
}
|
||||
|
||||
private static boolean isGroupBy(DataSchema dataSchema)
|
||||
|
@ -469,7 +537,10 @@ public class MSQCompactionRunner implements CompactionRunner
|
|||
);
|
||||
}
|
||||
// Similar to compaction using the native engine, don't finalize aggregations.
|
||||
// Used for writing the data schema during segment generation phase.
|
||||
context.putIfAbsent(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false);
|
||||
// Add appropriate finalization to native query context i.e. for the GroupBy query
|
||||
context.put(QueryContexts.FINALIZE_KEY, false);
|
||||
// Only scalar or array-type dimensions are allowed as grouping keys.
|
||||
context.putIfAbsent(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false);
|
||||
return context;
|
||||
|
|
|
@ -22,7 +22,7 @@ package org.apache.druid.msq.indexing.client;
|
|||
import org.apache.druid.msq.exec.Controller;
|
||||
import org.apache.druid.msq.indexing.IndexerResourcePermissionMapper;
|
||||
import org.apache.druid.msq.rpc.ControllerResource;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandler;
|
||||
import org.apache.druid.segment.realtime.ChatHandler;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
|
||||
public class ControllerChatHandler extends ControllerResource implements ChatHandler
|
||||
|
|
|
@ -33,8 +33,8 @@ import org.apache.druid.msq.kernel.StageId;
|
|||
import org.apache.druid.msq.kernel.WorkOrder;
|
||||
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
|
||||
import org.apache.druid.msq.statistics.serde.ClusterByStatisticsSnapshotSerde;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandler;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandlers;
|
||||
import org.apache.druid.segment.realtime.ChatHandler;
|
||||
import org.apache.druid.segment.realtime.ChatHandlers;
|
||||
import org.apache.druid.server.security.Action;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
|
||||
|
|
|
@ -177,9 +177,8 @@ public class SegmentGeneratorFrameProcessorFactory
|
|||
|
||||
// Create directly, without using AppenderatorsManager, because we need different memory overrides due to
|
||||
// using one Appenderator per processing thread instead of per task.
|
||||
// Note: "createOffline" ignores the batchProcessingMode and always acts like CLOSED_SEGMENTS_SINKS.
|
||||
final Appenderator appenderator =
|
||||
Appenderators.createOffline(
|
||||
Appenderators.createBatch(
|
||||
idString,
|
||||
dataSchema,
|
||||
makeAppenderatorConfig(
|
||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.druid.query.rowsandcols.semantic.ColumnSelectorFactoryMaker;
|
|||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.Cursor;
|
||||
import org.apache.druid.segment.column.NullableTypeStrategy;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -59,7 +60,6 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
|
@ -84,7 +84,10 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
private Supplier<ResultRow> rowSupplierFromFrameCursor;
|
||||
private ResultRow outputRow = null;
|
||||
private FrameWriter frameWriter = null;
|
||||
private final boolean isOverEmpty;
|
||||
|
||||
// List of type strategies to compare the partition columns across rows.
|
||||
// Type strategies are pushed in the same order as column types in frameReader.signature()
|
||||
private final NullableTypeStrategy[] typeStrategies;
|
||||
|
||||
public WindowOperatorQueryFrameProcessor(
|
||||
WindowOperatorQuery query,
|
||||
|
@ -95,7 +98,6 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
ObjectMapper jsonMapper,
|
||||
final List<OperatorFactory> operatorFactoryList,
|
||||
final RowSignature rowSignature,
|
||||
final boolean isOverEmpty,
|
||||
final int maxRowsMaterializedInWindow,
|
||||
final List<String> partitionColumnNames
|
||||
)
|
||||
|
@ -105,14 +107,18 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
this.frameWriterFactory = frameWriterFactory;
|
||||
this.operatorFactoryList = operatorFactoryList;
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.frameReader = frameReader;
|
||||
this.query = query;
|
||||
this.frameRowsAndCols = new ArrayList<>();
|
||||
this.resultRowAndCols = new ArrayList<>();
|
||||
this.objectsOfASingleRac = new ArrayList<>();
|
||||
this.isOverEmpty = isOverEmpty;
|
||||
this.maxRowsMaterialized = maxRowsMaterializedInWindow;
|
||||
this.partitionColumnNames = partitionColumnNames;
|
||||
|
||||
this.frameReader = frameReader;
|
||||
this.typeStrategies = new NullableTypeStrategy[frameReader.signature().size()];
|
||||
for (int i = 0; i < frameReader.signature().size(); i++) {
|
||||
typeStrategies[i] = frameReader.signature().getColumnType(i).get().getNullableStrategy();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -162,7 +168,7 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
*
|
||||
*
|
||||
* The flow would look like:
|
||||
* 1. Validate if the operator has an empty OVER clause
|
||||
* 1. Validate if the operator doesn't have any OVER() clause with PARTITION BY for this stage.
|
||||
* 2. If 1 is true make a giant rows and columns (R&C) using concat as shown above
|
||||
* Let all operators run amok on that R&C
|
||||
* 3. If 1 is false
|
||||
|
@ -187,14 +193,12 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
* We might think to reimplement them in the MSQ way so that we do not have to materialize so much data
|
||||
*/
|
||||
|
||||
// Phase 1 of the execution
|
||||
// eagerly validate presence of empty OVER() clause
|
||||
if (isOverEmpty) {
|
||||
// if OVER() found
|
||||
// have to bring all data to a single executor for processing
|
||||
// convert each frame to rac
|
||||
// concat all the racs to make a giant rac
|
||||
// let all operators run on the giant rac when channel is finished
|
||||
if (partitionColumnNames.isEmpty()) {
|
||||
// If we do not have any OVER() clause with PARTITION BY for this stage.
|
||||
// Bring all data to a single executor for processing.
|
||||
// Convert each frame to RAC.
|
||||
// Concatenate all the racs to make a giant RAC.
|
||||
// Let all operators run on the giant RAC until channel is finished.
|
||||
if (inputChannel.canRead()) {
|
||||
final Frame frame = inputChannel.read();
|
||||
convertRowFrameToRowsAndColumns(frame);
|
||||
|
@ -484,7 +488,7 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
|
||||
/**
|
||||
* Compare two rows based on the columns in partitionColumnNames.
|
||||
* If the partitionColumnNames is empty or null, compare entire row.
|
||||
* If the partitionColumnNames is empty, the method will end up returning true.
|
||||
* <p>
|
||||
* For example, say:
|
||||
* <ul>
|
||||
|
@ -501,17 +505,13 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
*/
|
||||
private boolean comparePartitionKeys(ResultRow row1, ResultRow row2, List<String> partitionColumnNames)
|
||||
{
|
||||
if (partitionColumnNames == null || partitionColumnNames.isEmpty()) {
|
||||
return row1.equals(row2);
|
||||
} else {
|
||||
int match = 0;
|
||||
for (String columnName : partitionColumnNames) {
|
||||
int i = frameReader.signature().indexOf(columnName);
|
||||
if (Objects.equals(row1.get(i), row2.get(i))) {
|
||||
match++;
|
||||
}
|
||||
int match = 0;
|
||||
for (String columnName : partitionColumnNames) {
|
||||
int i = frameReader.signature().indexOf(columnName);
|
||||
if (typeStrategies[i].compare(row1.get(i), row2.get(i)) == 0) {
|
||||
match++;
|
||||
}
|
||||
return match == partitionColumnNames.size();
|
||||
}
|
||||
return match == partitionColumnNames.size();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.Iterables;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectSortedMap;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.frame.processor.FrameProcessor;
|
||||
import org.apache.druid.frame.processor.OutputChannel;
|
||||
import org.apache.druid.frame.processor.OutputChannelFactory;
|
||||
|
@ -59,7 +60,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
private final WindowOperatorQuery query;
|
||||
private final List<OperatorFactory> operatorList;
|
||||
private final RowSignature stageRowSignature;
|
||||
private final boolean isEmptyOver;
|
||||
private final int maxRowsMaterializedInWindow;
|
||||
private final List<String> partitionColumnNames;
|
||||
|
||||
|
@ -68,7 +68,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
@JsonProperty("query") WindowOperatorQuery query,
|
||||
@JsonProperty("operatorList") List<OperatorFactory> operatorFactoryList,
|
||||
@JsonProperty("stageRowSignature") RowSignature stageRowSignature,
|
||||
@JsonProperty("emptyOver") boolean emptyOver,
|
||||
@JsonProperty("maxRowsMaterializedInWindow") int maxRowsMaterializedInWindow,
|
||||
@JsonProperty("partitionColumnNames") List<String> partitionColumnNames
|
||||
)
|
||||
|
@ -76,8 +75,11 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
this.query = Preconditions.checkNotNull(query, "query");
|
||||
this.operatorList = Preconditions.checkNotNull(operatorFactoryList, "bad operator");
|
||||
this.stageRowSignature = Preconditions.checkNotNull(stageRowSignature, "stageSignature");
|
||||
this.isEmptyOver = emptyOver;
|
||||
this.maxRowsMaterializedInWindow = maxRowsMaterializedInWindow;
|
||||
|
||||
if (partitionColumnNames == null) {
|
||||
throw DruidException.defensive("List of partition column names encountered as null.");
|
||||
}
|
||||
this.partitionColumnNames = partitionColumnNames;
|
||||
}
|
||||
|
||||
|
@ -105,12 +107,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
return stageRowSignature;
|
||||
}
|
||||
|
||||
@JsonProperty("emptyOver")
|
||||
public boolean isEmptyOverFound()
|
||||
{
|
||||
return isEmptyOver;
|
||||
}
|
||||
|
||||
@JsonProperty("maxRowsMaterializedInWindow")
|
||||
public int getMaxRowsMaterializedInWindow()
|
||||
{
|
||||
|
@ -166,7 +162,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
frameContext.jsonMapper(),
|
||||
operatorList,
|
||||
stageRowSignature,
|
||||
isEmptyOver,
|
||||
maxRowsMaterializedInWindow,
|
||||
partitionColumnNames
|
||||
);
|
||||
|
@ -190,8 +185,7 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
return false;
|
||||
}
|
||||
WindowOperatorQueryFrameProcessorFactory that = (WindowOperatorQueryFrameProcessorFactory) o;
|
||||
return isEmptyOver == that.isEmptyOver
|
||||
&& maxRowsMaterializedInWindow == that.maxRowsMaterializedInWindow
|
||||
return maxRowsMaterializedInWindow == that.maxRowsMaterializedInWindow
|
||||
&& Objects.equals(query, that.query)
|
||||
&& Objects.equals(operatorList, that.operatorList)
|
||||
&& Objects.equals(partitionColumnNames, that.partitionColumnNames)
|
||||
|
@ -201,6 +195,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(query, operatorList, partitionColumnNames, stageRowSignature, isEmptyOver, maxRowsMaterializedInWindow);
|
||||
return Objects.hash(query, operatorList, partitionColumnNames, stageRowSignature, maxRowsMaterializedInWindow);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.druid.segment.column.ColumnType;
|
|||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -124,8 +125,11 @@ public class WindowOperatorQueryKit implements QueryKit<WindowOperatorQuery>
|
|||
}
|
||||
|
||||
if (isEmptyOverPresent) {
|
||||
// empty over clause found
|
||||
// moving everything to a single partition
|
||||
// Move everything to a single partition since we have to load all the data on a single worker anyway to compute empty over() clause.
|
||||
log.info(
|
||||
"Empty over clause is present in the query. Creating a single stage with all operator factories [%s].",
|
||||
queryToRun.getOperators()
|
||||
);
|
||||
queryDefBuilder.add(
|
||||
StageDefinition.builder(firstStageNumber)
|
||||
.inputs(new StageInputSpec(firstStageNumber - 1))
|
||||
|
@ -136,9 +140,8 @@ public class WindowOperatorQueryKit implements QueryKit<WindowOperatorQuery>
|
|||
queryToRun,
|
||||
queryToRun.getOperators(),
|
||||
rowSignature,
|
||||
true,
|
||||
maxRowsMaterialized,
|
||||
new ArrayList<>()
|
||||
Collections.emptyList()
|
||||
))
|
||||
);
|
||||
} else {
|
||||
|
@ -237,7 +240,6 @@ public class WindowOperatorQueryKit implements QueryKit<WindowOperatorQuery>
|
|||
queryToRun,
|
||||
operatorList.get(i),
|
||||
stageRowSignature,
|
||||
false,
|
||||
maxRowsMaterialized,
|
||||
partitionColumnNames
|
||||
))
|
||||
|
@ -257,20 +259,34 @@ public class WindowOperatorQueryKit implements QueryKit<WindowOperatorQuery>
|
|||
{
|
||||
List<List<OperatorFactory>> operatorList = new ArrayList<>();
|
||||
final List<OperatorFactory> operators = originalQuery.getOperators();
|
||||
List<OperatorFactory> operatorFactoryList = new ArrayList<>();
|
||||
for (OperatorFactory of : operators) {
|
||||
operatorFactoryList.add(of);
|
||||
List<OperatorFactory> currentStage = new ArrayList<>();
|
||||
|
||||
for (int i = 0; i < operators.size(); i++) {
|
||||
OperatorFactory of = operators.get(i);
|
||||
currentStage.add(of);
|
||||
|
||||
if (of instanceof WindowOperatorFactory) {
|
||||
operatorList.add(operatorFactoryList);
|
||||
operatorFactoryList = new ArrayList<>();
|
||||
} else if (of instanceof NaivePartitioningOperatorFactory) {
|
||||
if (((NaivePartitioningOperatorFactory) of).getPartitionColumns().isEmpty()) {
|
||||
operatorList.clear();
|
||||
operatorList.add(originalQuery.getOperators());
|
||||
return operatorList;
|
||||
// Process consecutive window operators
|
||||
while (i + 1 < operators.size() && operators.get(i + 1) instanceof WindowOperatorFactory) {
|
||||
i++;
|
||||
currentStage.add(operators.get(i));
|
||||
}
|
||||
|
||||
// Finalize the current stage
|
||||
operatorList.add(new ArrayList<>(currentStage));
|
||||
currentStage.clear();
|
||||
}
|
||||
}
|
||||
|
||||
// There shouldn't be any operators left in currentStage. The last operator should always be WindowOperatorFactory.
|
||||
if (!currentStage.isEmpty()) {
|
||||
throw new ISE(
|
||||
"Found unexpected operators [%s] present in the list of operators [%s].",
|
||||
currentStage,
|
||||
operators
|
||||
);
|
||||
}
|
||||
|
||||
return operatorList;
|
||||
}
|
||||
|
||||
|
|
|
@ -149,12 +149,6 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor
|
|||
@Override
|
||||
public ReturnOrAwait<Object> runIncrementally(final IntSet readableInputs) throws IOException
|
||||
{
|
||||
final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter");
|
||||
|
||||
if (legacy) {
|
||||
throw new ISE("Cannot use this engine in legacy mode");
|
||||
}
|
||||
|
||||
if (runningCountForLimit != null
|
||||
&& runningCountForLimit.get() > query.getScanRowsOffset() + query.getScanRowsLimit()) {
|
||||
return ReturnOrAwait.returnObject(Unit.instance());
|
||||
|
|
|
@ -25,11 +25,6 @@ package org.apache.druid.msq.util;
|
|||
*/
|
||||
public enum ArrayIngestMode
|
||||
{
|
||||
/**
|
||||
* Disables the ingestion of arrays via MSQ's INSERT queries.
|
||||
*/
|
||||
NONE,
|
||||
|
||||
/**
|
||||
* String arrays are ingested as MVDs. This is to preserve the legacy behaviour of Druid and will be removed in the
|
||||
* future, since MVDs are not true array types and the behaviour is incorrect.
|
||||
|
|
|
@ -131,19 +131,9 @@ public class DimensionSchemaUtils
|
|||
} else if (queryType.getType() == ValueType.ARRAY) {
|
||||
ValueType elementType = queryType.getElementType().getType();
|
||||
if (elementType == ValueType.STRING) {
|
||||
if (arrayIngestMode == ArrayIngestMode.NONE) {
|
||||
throw InvalidInput.exception(
|
||||
"String arrays can not be ingested when '%s' is set to '%s'. Set '%s' in query context "
|
||||
+ "to 'array' to ingest the string array as an array, or ingest it as an MVD by explicitly casting the "
|
||||
+ "array to an MVD with the ARRAY_TO_MV function.",
|
||||
MultiStageQueryContext.CTX_ARRAY_INGEST_MODE,
|
||||
StringUtils.toLowerCase(arrayIngestMode.name()),
|
||||
MultiStageQueryContext.CTX_ARRAY_INGEST_MODE
|
||||
);
|
||||
} else if (arrayIngestMode == ArrayIngestMode.MVD) {
|
||||
if (arrayIngestMode == ArrayIngestMode.MVD) {
|
||||
return ColumnType.STRING;
|
||||
} else {
|
||||
assert arrayIngestMode == ArrayIngestMode.ARRAY;
|
||||
return queryType;
|
||||
}
|
||||
} else if (elementType.isNumeric()) {
|
||||
|
|
|
@ -165,7 +165,7 @@ public class MultiStageQueryContext
|
|||
public static final boolean DEFAULT_USE_AUTO_SCHEMAS = false;
|
||||
|
||||
public static final String CTX_ARRAY_INGEST_MODE = "arrayIngestMode";
|
||||
public static final ArrayIngestMode DEFAULT_ARRAY_INGEST_MODE = ArrayIngestMode.MVD;
|
||||
public static final ArrayIngestMode DEFAULT_ARRAY_INGEST_MODE = ArrayIngestMode.ARRAY;
|
||||
|
||||
public static final String NEXT_WINDOW_SHUFFLE_COL = "__windowShuffleCol";
|
||||
|
||||
|
|
|
@ -132,7 +132,7 @@ public class DataServerQueryHandlerTest
|
|||
.build();
|
||||
QueryToolChestWarehouse queryToolChestWarehouse = new MapQueryToolChestWarehouse(
|
||||
ImmutableMap.<Class<? extends Query>, QueryToolChest>builder()
|
||||
.put(ScanQuery.class, new ScanQueryQueryToolChest(null, null))
|
||||
.put(ScanQuery.class, new ScanQueryQueryToolChest(null))
|
||||
.build()
|
||||
);
|
||||
target = spy(
|
||||
|
|
|
@ -122,30 +122,7 @@ public class MSQArraysTest extends MSQTestBase
|
|||
}
|
||||
|
||||
/**
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest
|
||||
* string arrays
|
||||
*/
|
||||
@MethodSource("data")
|
||||
@ParameterizedTest(name = "{index}:with context {0}")
|
||||
public void testInsertStringArrayWithArrayIngestModeNone(String contextName, Map<String, Object> context)
|
||||
{
|
||||
|
||||
final Map<String, Object> adjustedContext = new HashMap<>(context);
|
||||
adjustedContext.put(MultiStageQueryContext.CTX_ARRAY_INGEST_MODE, "none");
|
||||
|
||||
testIngestQuery().setSql(
|
||||
"INSERT INTO foo1 SELECT MV_TO_ARRAY(dim3) AS dim3 FROM foo GROUP BY 1 PARTITIONED BY ALL TIME")
|
||||
.setQueryContext(adjustedContext)
|
||||
.setExpectedExecutionErrorMatcher(CoreMatchers.allOf(
|
||||
CoreMatchers.instanceOf(ISE.class),
|
||||
ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
|
||||
"String arrays can not be ingested when 'arrayIngestMode' is set to 'none'"))
|
||||
))
|
||||
.verifyExecutionError();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to default and the user tries to ingest
|
||||
* string arrays
|
||||
*/
|
||||
@MethodSource("data")
|
||||
|
@ -172,7 +149,7 @@ public class MSQArraysTest extends MSQTestBase
|
|||
}
|
||||
|
||||
/**
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to default and the user tries to ingest
|
||||
* string arrays
|
||||
*/
|
||||
@MethodSource("data")
|
||||
|
@ -200,7 +177,7 @@ public class MSQArraysTest extends MSQTestBase
|
|||
}
|
||||
|
||||
/**
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to default and the user tries to ingest
|
||||
* string arrays
|
||||
*/
|
||||
@MethodSource("data")
|
||||
|
@ -228,7 +205,7 @@ public class MSQArraysTest extends MSQTestBase
|
|||
}
|
||||
|
||||
/**
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to default and the user tries to ingest
|
||||
* string arrays
|
||||
*/
|
||||
@MethodSource("data")
|
||||
|
@ -277,7 +254,7 @@ public class MSQArraysTest extends MSQTestBase
|
|||
}
|
||||
|
||||
/**
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to default and the user tries to ingest
|
||||
* string arrays
|
||||
*/
|
||||
@MethodSource("data")
|
||||
|
@ -316,8 +293,7 @@ public class MSQArraysTest extends MSQTestBase
|
|||
}
|
||||
|
||||
/**
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to mvd (default) and the only array type to be
|
||||
* ingested is string array
|
||||
* Tests the behaviour of INSERT query when arrayIngestMode is set to array (default)
|
||||
*/
|
||||
@MethodSource("data")
|
||||
@ParameterizedTest(name = "{index}:with context {0}")
|
||||
|
@ -325,16 +301,32 @@ public class MSQArraysTest extends MSQTestBase
|
|||
{
|
||||
RowSignature rowSignature = RowSignature.builder()
|
||||
.add("__time", ColumnType.LONG)
|
||||
.add("dim3", ColumnType.STRING)
|
||||
.add("dim3", ColumnType.STRING_ARRAY)
|
||||
.build();
|
||||
|
||||
List<Object[]> expectedRows = new ArrayList<>(
|
||||
ImmutableList.of(
|
||||
new Object[]{0L, null},
|
||||
new Object[]{0L, new Object[]{"a", "b"}}
|
||||
)
|
||||
);
|
||||
if (!useDefault) {
|
||||
expectedRows.add(new Object[]{0L, new Object[]{""}});
|
||||
}
|
||||
expectedRows.addAll(
|
||||
ImmutableList.of(
|
||||
new Object[]{0L, new Object[]{"b", "c"}},
|
||||
new Object[]{0L, new Object[]{"d"}}
|
||||
)
|
||||
);
|
||||
|
||||
testIngestQuery().setSql(
|
||||
"INSERT INTO foo1 SELECT MV_TO_ARRAY(dim3) AS dim3 FROM foo GROUP BY 1 PARTITIONED BY ALL TIME")
|
||||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(expectedMultiValueFooRowsToArray())
|
||||
.setExpectedResultRows(expectedRows)
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -603,13 +595,6 @@ public class MSQArraysTest extends MSQTestBase
|
|||
.verifyResults();
|
||||
}
|
||||
|
||||
@MethodSource("data")
|
||||
@ParameterizedTest(name = "{index}:with context {0}")
|
||||
public void testSelectOnArraysWithArrayIngestModeAsNone(String contextName, Map<String, Object> context)
|
||||
{
|
||||
testSelectOnArrays(contextName, context, "none");
|
||||
}
|
||||
|
||||
@MethodSource("data")
|
||||
@ParameterizedTest(name = "{index}:with context {0}")
|
||||
public void testSelectOnArraysWithArrayIngestModeAsMVD(String contextName, Map<String, Object> context)
|
||||
|
@ -1128,20 +1113,4 @@ public class MSQArraysTest extends MSQTestBase
|
|||
.setExpectedResultRows(expectedRows)
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
private List<Object[]> expectedMultiValueFooRowsToArray()
|
||||
{
|
||||
List<Object[]> expectedRows = new ArrayList<>();
|
||||
expectedRows.add(new Object[]{0L, null});
|
||||
if (!useDefault) {
|
||||
expectedRows.add(new Object[]{0L, ""});
|
||||
}
|
||||
|
||||
expectedRows.addAll(ImmutableList.of(
|
||||
new Object[]{0L, ImmutableList.of("a", "b")},
|
||||
new Object[]{0L, ImmutableList.of("b", "c")},
|
||||
new Object[]{0L, "d"}
|
||||
));
|
||||
return expectedRows;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.apache.druid.msq.test.CounterSnapshotMatcher;
|
|||
import org.apache.druid.msq.test.MSQTestBase;
|
||||
import org.apache.druid.msq.test.MSQTestTaskActionClient;
|
||||
import org.apache.druid.msq.util.MultiStageQueryContext;
|
||||
import org.apache.druid.query.QueryContext;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
@ -957,6 +958,13 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
"Using RangeShardSpec to generate segments."
|
||||
)
|
||||
)
|
||||
.setExpectedLastCompactionState(expectedCompactionState(
|
||||
queryContext,
|
||||
Collections.singletonList("dim1"),
|
||||
Collections.singletonList(new StringDimensionSchema("dim1")),
|
||||
GranularityType.ALL,
|
||||
Intervals.ETERNITY
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1150,7 +1158,6 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
+ "CLUSTERED BY dim1"
|
||||
)
|
||||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(DEFAULT_MSQ_CONTEXT)
|
||||
.setExpectedShardSpec(DimensionRangeShardSpec.class)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
|
@ -1165,6 +1172,19 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
new Object[]{0L, "def", 5.0f, 1L}
|
||||
)
|
||||
)
|
||||
.setExpectedLastCompactionState(
|
||||
expectedCompactionState(
|
||||
context,
|
||||
Collections.singletonList("dim1"),
|
||||
Arrays.asList(
|
||||
new StringDimensionSchema("dim1"),
|
||||
new FloatDimensionSchema("m1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.ALL,
|
||||
Intervals.ETERNITY
|
||||
)
|
||||
)
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1571,6 +1591,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
+ "GROUP BY 1, 2 "
|
||||
+ "PARTITIONED by TIME_FLOOR(__time, 'P3M') "
|
||||
+ "CLUSTERED by dim1")
|
||||
.setQueryContext(context)
|
||||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedShardSpec(DimensionRangeShardSpec.class)
|
||||
|
@ -1582,6 +1603,16 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
)
|
||||
)
|
||||
.setExpectedResultRows(expectedResults)
|
||||
.setExpectedLastCompactionState(expectedCompactionState(
|
||||
context,
|
||||
Collections.singletonList("dim1"),
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("dim1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.QUARTER,
|
||||
Intervals.of("2000-01-01T00:00:00.000Z/2002-01-01T00:00:00.000Z")
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1652,6 +1683,16 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.setExpectedTombstoneIntervals(ImmutableSet.of(existingSegmentInterval))
|
||||
.setExpectedLastCompactionState(expectedCompactionState(
|
||||
context,
|
||||
Collections.emptyList(),
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("dim1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.DAY,
|
||||
Intervals.ETERNITY
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1688,6 +1729,16 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
Intervals.of("2016-06-27T01:00:00/2016-06-27T02:00:00")
|
||||
)
|
||||
)
|
||||
.setExpectedLastCompactionState(expectedCompactionState(
|
||||
context,
|
||||
Collections.emptyList(),
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("dim1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.HOUR,
|
||||
Intervals.of("2016-06-27T01:00:00/2016-06-27T02:00:00")
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1726,6 +1777,16 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
Intervals.of("2016-06-30T/2016-07-01T")
|
||||
)
|
||||
)
|
||||
.setExpectedLastCompactionState(expectedCompactionState(
|
||||
context,
|
||||
Collections.emptyList(),
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("dim1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.DAY,
|
||||
Intervals.of("2016-06-29T00:00:00.000Z/2016-07-03T00:00:00.000Z")
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1764,6 +1825,16 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
Intervals.of("2016-06-02T/2016-06-03T")
|
||||
)
|
||||
)
|
||||
.setExpectedLastCompactionState(expectedCompactionState(
|
||||
context,
|
||||
Collections.emptyList(),
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("dim1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.DAY,
|
||||
Intervals.of("2016-05-25T00:00:00.000Z/2016-06-03T00:00:00.000Z")
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1802,6 +1873,16 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.ETERNITY))
|
||||
.setExpectedLastCompactionState(expectedCompactionState(
|
||||
context,
|
||||
Collections.emptyList(),
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("dim1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.DAY,
|
||||
Intervals.ETERNITY
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1838,6 +1919,16 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.of("2016-06-01T/2016-09-01T")))
|
||||
.setExpectedLastCompactionState(expectedCompactionState(
|
||||
context,
|
||||
Collections.emptyList(),
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("dim1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.ALL,
|
||||
Intervals.ETERNITY
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1874,6 +1965,16 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.ETERNITY))
|
||||
.setExpectedLastCompactionState(expectedCompactionState(
|
||||
context,
|
||||
Collections.emptyList(),
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("dim1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.ALL,
|
||||
Intervals.ETERNITY
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1906,6 +2007,16 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.ETERNITY))
|
||||
.setExpectedLastCompactionState(expectedCompactionState(
|
||||
context,
|
||||
Collections.emptyList(),
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("dim1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.ALL,
|
||||
Intervals.ETERNITY
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1966,6 +2077,16 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
Intervals.of("2016-06-02T/2016-06-03T")
|
||||
)
|
||||
)
|
||||
.setExpectedLastCompactionState(expectedCompactionState(
|
||||
context,
|
||||
Collections.emptyList(),
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("dim1"),
|
||||
new LongDimensionSchema("cnt")
|
||||
),
|
||||
GranularityType.DAY,
|
||||
Intervals.of("2016-06-01T00:00:00.000Z/2016-06-03T00:00:00.000Z")
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -2049,11 +2170,16 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
}
|
||||
PartitionsSpec partitionsSpec;
|
||||
if (partitionDimensions.isEmpty()) {
|
||||
partitionsSpec = new DynamicPartitionsSpec(MultiStageQueryContext.DEFAULT_ROWS_PER_SEGMENT, Long.MAX_VALUE);
|
||||
|
||||
partitionsSpec = new DynamicPartitionsSpec(
|
||||
MultiStageQueryContext.getRowsPerSegment(QueryContext.of(context)),
|
||||
Long.MAX_VALUE
|
||||
);
|
||||
} else {
|
||||
partitionsSpec = new DimensionRangePartitionsSpec(MultiStageQueryContext.DEFAULT_ROWS_PER_SEGMENT, null,
|
||||
partitionDimensions, false
|
||||
partitionsSpec = new DimensionRangePartitionsSpec(
|
||||
MultiStageQueryContext.getRowsPerSegment(QueryContext.of(context)),
|
||||
null,
|
||||
partitionDimensions,
|
||||
false
|
||||
);
|
||||
}
|
||||
DimensionsSpec dimensionsSpec = new DimensionsSpec.Builder()
|
||||
|
|
|
@ -2594,7 +2594,6 @@ public class MSQSelectTest extends MSQTestBase
|
|||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(defaultScanQueryContext(
|
||||
context,
|
||||
resultSignature
|
||||
|
@ -2668,7 +2667,6 @@ public class MSQSelectTest extends MSQTestBase
|
|||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.filters(equality("dim2", "a", ColumnType.STRING))
|
||||
.columns("dim3")
|
||||
.context(defaultScanQueryContext(
|
||||
|
@ -2683,7 +2681,6 @@ public class MSQSelectTest extends MSQTestBase
|
|||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(defaultScanQueryContext(
|
||||
context,
|
||||
resultSignature
|
||||
|
@ -2751,7 +2748,6 @@ public class MSQSelectTest extends MSQTestBase
|
|||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(defaultScanQueryContext(
|
||||
context,
|
||||
rowSignature
|
||||
|
|
|
@ -106,7 +106,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.build();
|
||||
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "d0")
|
||||
};
|
||||
|
@ -196,7 +196,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.build();
|
||||
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "d1")
|
||||
};
|
||||
|
@ -306,7 +306,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.build();
|
||||
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "d1")
|
||||
};
|
||||
|
@ -419,7 +419,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.build();
|
||||
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "d0")
|
||||
};
|
||||
|
@ -523,7 +523,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.build();
|
||||
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "d0")
|
||||
};
|
||||
|
@ -589,7 +589,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("cc", ColumnType.DOUBLE)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "m1")
|
||||
};
|
||||
|
@ -609,7 +609,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("m1")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -655,7 +654,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("cc", ColumnType.DOUBLE)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "m1")
|
||||
};
|
||||
|
@ -678,7 +677,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("m1", "m2")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -727,7 +725,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("cc", ColumnType.DOUBLE)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "m1")
|
||||
};
|
||||
|
@ -750,7 +748,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("m1", "m2")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -796,7 +793,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("cc", ColumnType.DOUBLE)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "d1")
|
||||
};
|
||||
|
@ -881,7 +878,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("cc", ColumnType.DOUBLE)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "m1")
|
||||
};
|
||||
|
@ -896,7 +893,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.virtualColumns(expressionVirtualColumn("v0", "strlen(\"dim1\")", ColumnType.LONG))
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -955,7 +951,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("cc", ColumnType.DOUBLE)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "m1")
|
||||
};
|
||||
|
@ -969,7 +965,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("m1")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -1033,17 +1028,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("m2", ColumnType.DOUBLE)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(
|
||||
WindowFrame.PeerType.RANGE,
|
||||
true,
|
||||
0,
|
||||
false,
|
||||
0,
|
||||
ImmutableList.of(new ColumnWithDirection(
|
||||
"m1",
|
||||
ColumnWithDirection.Direction.ASC
|
||||
))
|
||||
);
|
||||
final WindowFrame theFrame = WindowFrame.forOrderBy("m1");
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "m1")
|
||||
};
|
||||
|
@ -1063,7 +1048,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("m2", "v0")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature1)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
"j0.",
|
||||
|
@ -1078,7 +1062,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("j0.m2", "m1")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -1149,14 +1132,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("m2", ColumnType.DOUBLE)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(
|
||||
WindowFrame.PeerType.ROWS,
|
||||
true,
|
||||
0,
|
||||
true,
|
||||
0,
|
||||
null
|
||||
);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "m1")
|
||||
};
|
||||
|
@ -1176,7 +1152,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("m2", "v0")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature1)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
"j0.",
|
||||
|
@ -1191,7 +1166,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("j0.m2", "m1")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -1242,7 +1216,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("cc", ColumnType.DOUBLE)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "m1")
|
||||
};
|
||||
|
@ -1265,7 +1239,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("dim2", "m1")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -1332,14 +1305,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("d3", ColumnType.STRING)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(
|
||||
WindowFrame.PeerType.ROWS,
|
||||
true,
|
||||
0,
|
||||
true,
|
||||
0,
|
||||
null
|
||||
);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "m1")
|
||||
};
|
||||
|
@ -1359,7 +1325,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("j0.unnest", "m1")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -1423,14 +1388,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("d3", ColumnType.STRING)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(
|
||||
WindowFrame.PeerType.ROWS,
|
||||
true,
|
||||
0,
|
||||
true,
|
||||
0,
|
||||
null
|
||||
);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new DoubleSumAggregatorFactory("w0", "m1")
|
||||
};
|
||||
|
@ -1450,7 +1408,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("j0.unnest", "m1")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -1808,7 +1765,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("cc", ColumnType.LONG)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new LongSumAggregatorFactory("w0", "added")
|
||||
};
|
||||
|
@ -1832,7 +1789,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("added", "cityName")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(contextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -1900,7 +1856,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.add("cc", ColumnType.LONG)
|
||||
.build();
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new LongSumAggregatorFactory("w0", "added")
|
||||
};
|
||||
|
@ -1924,7 +1880,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.columns("added", "cityName", "countryIsoCode")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(innerContextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build()),
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
context,
|
||||
|
@ -1954,7 +1909,6 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.limit(5)
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(outerContextWithRowSignature)
|
||||
.legacy(false)
|
||||
.build();
|
||||
|
||||
testSelectQuery()
|
||||
|
@ -2016,7 +1970,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
.build();
|
||||
|
||||
|
||||
final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null);
|
||||
final WindowFrame theFrame = WindowFrame.unbounded();
|
||||
final AggregatorFactory[] theAggs = {
|
||||
new LongSumAggregatorFactory("w0", "d1")
|
||||
};
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.druid.sql.calcite.schema.NamedDruidSchema;
|
|||
import org.apache.druid.sql.calcite.schema.NamedViewSchema;
|
||||
import org.apache.druid.sql.calcite.schema.ViewSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.hook.DruidHookDispatcher;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
@ -75,7 +76,8 @@ public class ResultsContextSerdeTest
|
|||
"druid",
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.TEST_AUTHORIZER_MAPPER,
|
||||
AuthConfig.newBuilder().build()
|
||||
AuthConfig.newBuilder().build(),
|
||||
new DruidHookDispatcher()
|
||||
);
|
||||
final NativeSqlEngine engine = CalciteTests.createMockSqlEngine(
|
||||
EasyMock.createMock(QuerySegmentWalker.class),
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.druid.indexing.common.task.CompactionTask;
|
|||
import org.apache.druid.indexing.common.task.TuningConfigBuilder;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.granularity.GranularityType;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
|
@ -54,10 +55,12 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
|||
import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.data.CompressionFactory;
|
||||
import org.apache.druid.segment.data.CompressionStrategy;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.indexing.CombinedDataSchema;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import org.apache.druid.segment.transform.TransformSpec;
|
||||
|
@ -73,6 +76,7 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class MSQCompactionRunnerTest
|
||||
{
|
||||
|
@ -127,7 +131,7 @@ public class MSQCompactionRunnerTest
|
|||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -140,7 +144,7 @@ public class MSQCompactionRunnerTest
|
|||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -153,7 +157,7 @@ public class MSQCompactionRunnerTest
|
|||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -166,7 +170,7 @@ public class MSQCompactionRunnerTest
|
|||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -179,7 +183,7 @@ public class MSQCompactionRunnerTest
|
|||
new ClientCompactionTaskGranularitySpec(null, Granularities.ALL, null),
|
||||
null
|
||||
);
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -192,28 +196,7 @@ public class MSQCompactionRunnerTest
|
|||
new ClientCompactionTaskGranularitySpec(null, null, false),
|
||||
AGGREGATORS.toArray(new AggregatorFactory[0])
|
||||
);
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMSQEngineWithUnsupportedMetricsSpecIsInValid()
|
||||
{
|
||||
// Aggregators having different input and ouput column names are unsupported.
|
||||
final String inputColName = "added";
|
||||
final String outputColName = "sum_added";
|
||||
CompactionTask compactionTask = createCompactionTask(
|
||||
new DynamicPartitionsSpec(3, null),
|
||||
null,
|
||||
Collections.emptyMap(),
|
||||
new ClientCompactionTaskGranularitySpec(null, null, null),
|
||||
new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)}
|
||||
);
|
||||
CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask);
|
||||
Assert.assertFalse(validationResult.isValid());
|
||||
Assert.assertEquals(
|
||||
"Different name[sum_added] and fieldName(s)[[added]] for aggregator unsupported for MSQ engine.",
|
||||
validationResult.getReason()
|
||||
);
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -288,6 +271,10 @@ public class MSQCompactionRunnerTest
|
|||
);
|
||||
Assert.assertNull(msqControllerTask.getContext().get(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY));
|
||||
Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy());
|
||||
Assert.assertEquals(PARTITION_DIMENSIONS.stream().map(col -> new ScanQuery.OrderBy(
|
||||
col,
|
||||
ScanQuery.Order.ASCENDING
|
||||
)).collect(Collectors.toList()), ((ScanQuery) actualMSQSpec.getQuery()).getOrderBys());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -358,6 +345,48 @@ public class MSQCompactionRunnerTest
|
|||
Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIntervalsWithRolledUpSegmentsAndNonIdempotentAggregatorFails()
|
||||
{
|
||||
final String inputColName = "added";
|
||||
final String outputColName = "sum_added";
|
||||
CompactionTask compactionTask = createCompactionTask(
|
||||
null,
|
||||
null,
|
||||
Collections.emptyMap(),
|
||||
null,
|
||||
new AggregatorFactory[]{
|
||||
new LongSumAggregatorFactory(
|
||||
outputColName,
|
||||
inputColName
|
||||
)
|
||||
}
|
||||
);
|
||||
CombinedDataSchema dataSchema = new CombinedDataSchema(
|
||||
DATA_SOURCE,
|
||||
new TimestampSpec(TIMESTAMP_COLUMN, null, null),
|
||||
new DimensionsSpec(DIMENSIONS),
|
||||
new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)},
|
||||
new UniformGranularitySpec(
|
||||
SEGMENT_GRANULARITY.getDefaultGranularity(),
|
||||
null,
|
||||
false,
|
||||
Collections.singletonList(COMPACTION_INTERVAL)
|
||||
),
|
||||
null,
|
||||
true
|
||||
);
|
||||
CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask(
|
||||
compactionTask,
|
||||
Collections.singletonMap(COMPACTION_INTERVAL, dataSchema)
|
||||
);
|
||||
Assert.assertFalse(validationResult.isValid());
|
||||
Assert.assertEquals(validationResult.getReason(), StringUtils.format(
|
||||
"MSQ: Rolled-up segments in compaction interval[%s].",
|
||||
COMPACTION_INTERVAL
|
||||
));
|
||||
}
|
||||
|
||||
private CompactionTask createCompactionTask(
|
||||
@Nullable PartitionsSpec partitionsSpec,
|
||||
@Nullable DimFilter dimFilter,
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue