mirror of https://github.com/apache/druid.git
Merge github.com:apache/incubator-druid into 6088-Create-Scan-Benchmark
This commit is contained in:
commit
5edbe2ae12
|
@ -147,7 +147,7 @@
|
|||
<searchConfiguration name="Suboptimal IndexedInts iteration" text="$x$ < $y$.size()" recursive="false" caseInsensitive="true" type="JAVA">
|
||||
<constraint name="__context__" target="true" within="" contains="" />
|
||||
<constraint name="x" within="" contains="" />
|
||||
<constraint name="y" nameOfExprType="IndexedInts" expressionTypes="IndexedInts" exprTypeWithinHierarchy="true" within="" contains="" />
|
||||
<constraint name="y" nameOfExprType="IndexedInts" exprTypeWithinHierarchy="true" within="" contains="" />
|
||||
</searchConfiguration>
|
||||
<searchConfiguration name="Lists.newArrayList() with a single argument. Use Collections.singletonList() instead" created="1532737126203" text="Lists.newArrayList($x$)" recursive="false" caseInsensitive="true" type="JAVA">
|
||||
<constraint name="x" nameOfExprType="java\.lang\.Iterable|java\.util\.Iterator|Object\[\]" expressionTypes="java.lang.Iterable|java.util.Iterator|Object[]" exprTypeWithinHierarchy="true" negateName="true" negateExprType="true" within="" contains="" />
|
||||
|
@ -236,6 +236,38 @@
|
|||
<constraint name="__context__" target="true" within="" contains="" />
|
||||
<constraint name="E" within="" contains="" />
|
||||
</searchConfiguration>
|
||||
<searchConfiguration name="Concurrent maps should be assigned into variables of ConcurrentMap type or more specific" text="Map<$K$, $V$> $x$ = $y$;" recursive="true" caseInsensitive="true" type="JAVA">
|
||||
<constraint name="__context__" target="true" within="" contains="" />
|
||||
<constraint name="K" within="" contains="" />
|
||||
<constraint name="V" within="" contains="" />
|
||||
<constraint name="x" within="" contains="" />
|
||||
<constraint name="y" nameOfExprType="java\.util\.concurrent\.ConcurrentMap" expressionTypes="java.util.concurrent.ConcurrentMap" exprTypeWithinHierarchy="true" within="" contains="" />
|
||||
</searchConfiguration>
|
||||
<searchConfiguration name="A ConcurrentHashMap on which compute() is called should be assinged into variables of ConcurrentHashMap type, not ConcurrentMap" text="$x$.compute($y$, $z$)" recursive="true" caseInsensitive="true" type="JAVA">
|
||||
<constraint name="__context__" target="true" within="" contains="" />
|
||||
<constraint name="x" nameOfExprType="java\.util\.concurrent\.ConcurrentMap" expressionTypes="java.util.concurrent.ConcurrentMap" within="" contains="" />
|
||||
<constraint name="y" within="" contains="" />
|
||||
<constraint name="z" within="" contains="" />
|
||||
</searchConfiguration>
|
||||
<searchConfiguration name="A ConcurrentHashMap on which computeIfAbsent() is called should be assigned into variables of ConcurrentHashMap type, not ConcurrentMap" text="$x$.computeIfAbsent($y$, $z$)" recursive="true" caseInsensitive="true" type="JAVA">
|
||||
<constraint name="__context__" target="true" within="" contains="" />
|
||||
<constraint name="x" nameOfExprType="java\.util\.concurrent\.ConcurrentMap" expressionTypes="java.util.concurrent.ConcurrentMap" within="" contains="" />
|
||||
<constraint name="y" within="" contains="" />
|
||||
<constraint name="z" within="" contains="" />
|
||||
</searchConfiguration>
|
||||
<searchConfiguration name="A ConcurrentHashMap on which computeIfPresent() is called should be assigned into variables of ConcurrentHashMap type, not ConcurrentMap" text="$x$.computeIfPresent($y$, $z$)" recursive="true" caseInsensitive="true" type="JAVA">
|
||||
<constraint name="__context__" target="true" within="" contains="" />
|
||||
<constraint name="x" nameOfExprType="java\.util\.concurrent\.ConcurrentMap" expressionTypes="java.util.concurrent.ConcurrentMap" within="" contains="" />
|
||||
<constraint name="y" within="" contains="" />
|
||||
<constraint name="z" within="" contains="" />
|
||||
</searchConfiguration>
|
||||
<searchConfiguration name="A ConcurrentHashMap on which merge() is called should be assigned into variables of ConcurrentHashMap type, not ConcurrentMap" text="$x$.merge($a$, $b$, $c$)" recursive="true" caseInsensitive="true" type="JAVA">
|
||||
<constraint name="__context__" target="true" within="" contains="" />
|
||||
<constraint name="x" nameOfExprType="java\.util\.concurrent\.ConcurrentMap" expressionTypes="java.util.concurrent.ConcurrentMap" within="" contains="" />
|
||||
<constraint name="a" within="" contains="" />
|
||||
<constraint name="b" within="" contains="" />
|
||||
<constraint name="c" within="" contains="" />
|
||||
</searchConfiguration>
|
||||
</inspection_tool>
|
||||
<inspection_tool class="SimplifyStreamApiCallChains" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
<inspection_tool class="SpellCheckingInspection" enabled="false" level="TYPO" enabled_by_default="false">
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -27,7 +27,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<prerequisites>
|
||||
|
|
|
@ -201,7 +201,7 @@ public class FilteredAggregatorBenchmark
|
|||
|
||||
factory = new TimeseriesQueryRunnerFactory(
|
||||
new TimeseriesQueryQueryToolChest(
|
||||
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
new TimeseriesQueryEngine(),
|
||||
QueryBenchmarkUtil.NOOP_QUERYWATCHER
|
||||
|
|
|
@ -407,7 +407,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
strategySelector,
|
||||
new GroupByQueryQueryToolChest(
|
||||
strategySelector,
|
||||
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -299,7 +299,10 @@ public class TopNTypeInterfaceBenchmark
|
|||
0,
|
||||
Integer.MAX_VALUE
|
||||
),
|
||||
new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()),
|
||||
new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryBenchmarkUtil.NOOP_QUERYWATCHER
|
||||
);
|
||||
}
|
||||
|
|
|
@ -495,7 +495,7 @@ public class GroupByBenchmark
|
|||
strategySelector,
|
||||
new GroupByQueryQueryToolChest(
|
||||
strategySelector,
|
||||
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -49,7 +49,7 @@ public class QueryBenchmarkUtil
|
|||
);
|
||||
}
|
||||
|
||||
public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator()
|
||||
public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
|
||||
{
|
||||
return new IntervalChunkingQueryRunnerDecorator(null, null, null) {
|
||||
@Override
|
||||
|
|
|
@ -373,7 +373,7 @@ public class SearchBenchmark
|
|||
new SearchStrategySelector(Suppliers.ofInstance(config)),
|
||||
new SearchQueryQueryToolChest(
|
||||
config,
|
||||
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryBenchmarkUtil.NOOP_QUERYWATCHER
|
||||
);
|
||||
|
|
|
@ -233,7 +233,7 @@ public class SelectBenchmark
|
|||
factory = new SelectQueryRunnerFactory(
|
||||
new SelectQueryQueryToolChest(
|
||||
JSON_MAPPER,
|
||||
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator(),
|
||||
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator(),
|
||||
selectConfigSupplier
|
||||
),
|
||||
new SelectQueryEngine(),
|
||||
|
|
|
@ -114,7 +114,7 @@ public class SqlBenchmark
|
|||
final QueryRunnerFactoryConglomerate conglomerate = conglomerateCloserPair.lhs;
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
this.walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(dataSegment, index);
|
||||
final PlannerFactory plannerFactory = new PlannerFactory(
|
||||
druidSchema,
|
||||
|
|
|
@ -296,7 +296,7 @@ public class TimeseriesBenchmark
|
|||
|
||||
factory = new TimeseriesQueryRunnerFactory(
|
||||
new TimeseriesQueryQueryToolChest(
|
||||
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
new TimeseriesQueryEngine(),
|
||||
QueryBenchmarkUtil.NOOP_QUERYWATCHER
|
||||
|
|
|
@ -278,7 +278,10 @@ public class TopNBenchmark
|
|||
0,
|
||||
Integer.MAX_VALUE
|
||||
),
|
||||
new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()),
|
||||
new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryBenchmarkUtil.NOOP_QUERYWATCHER
|
||||
);
|
||||
}
|
||||
|
|
|
@ -229,7 +229,10 @@ public class TimeCompareBenchmark
|
|||
0,
|
||||
Integer.MAX_VALUE
|
||||
),
|
||||
new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()),
|
||||
new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryBenchmarkUtil.NOOP_QUERYWATCHER
|
||||
);
|
||||
}
|
||||
|
@ -270,7 +273,7 @@ public class TimeCompareBenchmark
|
|||
|
||||
timeseriesQuery = timeseriesQueryBuilder.build();
|
||||
timeseriesFactory = new TimeseriesQueryRunnerFactory(
|
||||
new TimeseriesQueryQueryToolChest(QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()),
|
||||
new TimeseriesQueryQueryToolChest(QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()),
|
||||
new TimeseriesQueryEngine(),
|
||||
QueryBenchmarkUtil.NOOP_QUERYWATCHER
|
||||
);
|
||||
|
|
|
@ -31,7 +31,7 @@
|
|||
<parent>
|
||||
<artifactId>druid</artifactId>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<properties>
|
||||
|
|
|
@ -41,24 +41,12 @@ public class SocketUtil
|
|||
int currPort = startPort;
|
||||
|
||||
while (currPort < 0xffff) {
|
||||
ServerSocket socket = null;
|
||||
try {
|
||||
socket = new ServerSocket(currPort);
|
||||
try (ServerSocket socket = new ServerSocket(currPort)) {
|
||||
return currPort;
|
||||
}
|
||||
catch (IOException e) {
|
||||
++currPort;
|
||||
}
|
||||
finally {
|
||||
if (socket != null) {
|
||||
try {
|
||||
socket.close();
|
||||
}
|
||||
catch (IOException e) {
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
throw new ISE("Unable to find open port between [%d] and [%d]", startPort, currPort);
|
||||
|
|
|
@ -120,21 +120,19 @@ public class ParametrizedUriEmitter implements Flushable, Closeable, Emitter
|
|||
{
|
||||
try {
|
||||
URI uri = uriExtractor.apply(event);
|
||||
// get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed.
|
||||
// See https://github.com/apache/incubator-druid/pull/6898#discussion_r251384586.
|
||||
HttpPostEmitter emitter = emitters.get(uri);
|
||||
if (emitter == null) {
|
||||
try {
|
||||
emitter = emitters.computeIfAbsent(uri, u -> {
|
||||
try {
|
||||
return innerLifecycle.addMaybeStartManagedInstance(
|
||||
new HttpPostEmitter(
|
||||
config.buildHttpEmitterConfig(u.toString()),
|
||||
client,
|
||||
jsonMapper
|
||||
)
|
||||
new HttpPostEmitter(config.buildHttpEmitterConfig(u.toString()), client, jsonMapper)
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -24,13 +24,16 @@ import com.google.common.base.Throwables;
|
|||
import com.google.common.cache.CacheBuilder;
|
||||
import com.google.common.cache.CacheLoader;
|
||||
import com.google.common.cache.LoadingCache;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
|
@ -123,13 +126,23 @@ public class ResourcePool<K, V> implements Closeable
|
|||
public void close()
|
||||
{
|
||||
closed.set(true);
|
||||
final Map<K, ImmediateCreationResourceHolder<K, V>> mapView = pool.asMap();
|
||||
for (K k : ImmutableSet.copyOf(mapView.keySet())) {
|
||||
mapView.remove(k).close();
|
||||
final ConcurrentMap<K, ImmediateCreationResourceHolder<K, V>> mapView = pool.asMap();
|
||||
Closer closer = Closer.create();
|
||||
for (Iterator<Map.Entry<K, ImmediateCreationResourceHolder<K, V>>> iterator =
|
||||
mapView.entrySet().iterator(); iterator.hasNext(); ) {
|
||||
Map.Entry<K, ImmediateCreationResourceHolder<K, V>> e = iterator.next();
|
||||
iterator.remove();
|
||||
closer.register(e.getValue());
|
||||
}
|
||||
try {
|
||||
closer.close();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static class ImmediateCreationResourceHolder<K, V>
|
||||
private static class ImmediateCreationResourceHolder<K, V> implements Closeable
|
||||
{
|
||||
private final int maxSize;
|
||||
private final K key;
|
||||
|
@ -265,7 +278,8 @@ public class ResourcePool<K, V> implements Closeable
|
|||
return resourceHolderList.stream().anyMatch(a -> a.getResource().equals(object));
|
||||
}
|
||||
|
||||
void close()
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
synchronized (this) {
|
||||
closed = true;
|
||||
|
|
|
@ -31,7 +31,7 @@
|
|||
<parent>
|
||||
<artifactId>druid</artifactId>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -536,6 +536,7 @@ This deep storage doesn't do anything. There are no configs.
|
|||
#### S3 Deep Storage
|
||||
|
||||
This deep storage is used to interface with Amazon's S3. Note that the `druid-s3-extensions` extension must be loaded.
|
||||
The below table shows some important configurations for S3. See [S3 Deep Storage](../development/extensions-core/s3.html) for full configurations.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|
@ -543,7 +544,7 @@ This deep storage is used to interface with Amazon's S3. Note that the `druid-s3
|
|||
|`druid.s3.secretKey`|The secret key to use to access S3.|none|
|
||||
|`druid.storage.bucket`|S3 bucket name.|none|
|
||||
|`druid.storage.baseKey`|S3 object key prefix for storage.|none|
|
||||
|`druid.storage.disableAcl`|Boolean flag for ACL.|false|
|
||||
|`druid.storage.disableAcl`|Boolean flag for ACL. If this is set to `false`, the full control would be granted to the bucket owner. This may require to set additional permissions. See [S3 permissions settings](../development/extensions-core/s3.html#s3-permissions-settings).|false|
|
||||
|`druid.storage.archiveBucket`|S3 bucket name for archiving when running the *archive task*.|none|
|
||||
|`druid.storage.archiveBaseKey`|S3 object key prefix for archiving.|none|
|
||||
|`druid.storage.useS3aSchema`|If true, use the "s3a" filesystem when using Hadoop-based ingestion. If false, the "s3n" filesystem will be used. Only affects Hadoop-based ingestion.|false|
|
||||
|
@ -779,7 +780,9 @@ A sample Coordinator dynamic config JSON object is shown below:
|
|||
"replicantLifetime": 15,
|
||||
"replicationThrottleLimit": 10,
|
||||
"emitBalancingStats": false,
|
||||
"killDataSourceWhitelist": ["wikipedia", "testDatasource"]
|
||||
"killDataSourceWhitelist": ["wikipedia", "testDatasource"],
|
||||
"historicalNodesInMaintenance": ["localhost:8182", "localhost:8282"],
|
||||
"nodesInMaintenancePriority": 7
|
||||
}
|
||||
```
|
||||
|
||||
|
@ -799,6 +802,8 @@ Issuing a GET request at the same URL will return the spec that is currently in
|
|||
|`killAllDataSources`|Send kill tasks for ALL dataSources if property `druid.coordinator.kill.on` is true. If this is set to true then `killDataSourceWhitelist` must not be specified or be empty list.|false|
|
||||
|`killPendingSegmentsSkipList`|List of dataSources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none|
|
||||
|`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 0 (loading queue is unbounded) |0|
|
||||
|`historicalNodesInMaintenance`| List of Historical nodes in maintenance mode. Coordinator doesn't assign new segments on those nodes and moves segments from the nodes according to a specified priority.|none|
|
||||
|`nodesInMaintenancePriority`| Priority of segments from servers in maintenance. Coordinator takes ceil(maxSegmentsToMove * (priority / 10)) from servers in maitenance during balancing phase, i.e.:<br>0 - no segments from servers in maintenance will be processed during balancing<br>5 - 50% segments from servers in maintenance<br>10 - 100% segments from servers in maintenance<br>By leveraging the priority an operator can prevent general nodes from overload or decrease maitenance time instead.|7|
|
||||
|
||||
To view the audit history of Coordinator dynamic config issue a GET request to the URL -
|
||||
|
||||
|
|
|
@ -45,6 +45,7 @@ As an example, to set the region to 'us-east-1' through system properties:
|
|||
|`druid.s3.secretKey`|S3 secret key.|Must be set.|
|
||||
|`druid.storage.bucket`|Bucket to store in.|Must be set.|
|
||||
|`druid.storage.baseKey`|Base key prefix to use, i.e. what directory.|Must be set.|
|
||||
|`druid.storage.disableAcl`|Boolean flag to disable ACL. If this is set to `false`, the full control would be granted to the bucket owner. This may require to set additional permissions. See [S3 permissions settings](#s3-permissions-settings).|false|
|
||||
|`druid.storage.sse.type`|Server-side encryption type. Should be one of `s3`, `kms`, and `custom`. See the below [Server-side encryption section](#server-side-encryption) for more details.|None|
|
||||
|`druid.storage.sse.kms.keyId`|AWS KMS key ID. Can be empty if `druid.storage.sse.type` is `kms`.|None|
|
||||
|`druid.storage.sse.custom.base64EncodedKey`|Base64-encoded key. Should be specified if `druid.storage.sse.type` is `custom`.|None|
|
||||
|
@ -59,6 +60,11 @@ As an example, to set the region to 'us-east-1' through system properties:
|
|||
|`druid.s3.proxy.username`|User name to use when connecting through a proxy.|None|
|
||||
|`druid.s3.proxy.password`|Password to use when connecting through a proxy.|None|
|
||||
|
||||
### S3 permissions settings
|
||||
|
||||
`s3:GetObject` and `s3:PutObject` are basically required for pushing/loading segments to/from S3.
|
||||
If `druid.storage.disableAcl` is set to `false`, then `s3:GetBucketAcl` and `s3:PutObjectAcl` are additionally required to set ACL for objects.
|
||||
|
||||
## Server-side encryption
|
||||
|
||||
You can enable [server-side encryption](https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html) by setting
|
||||
|
|
|
@ -86,7 +86,7 @@ rollup and load your existing data as-is. Rollup in Druid is similar to creating
|
|||
(Like OpenTSDB or InfluxDB.)
|
||||
|
||||
Similar to time series databases, Druid's data model requires a timestamp. Druid is not a timeseries database, but
|
||||
it is a natural choice for storing timeseries data. Its flexible data mdoel allows it to store both timeseries and
|
||||
it is a natural choice for storing timeseries data. Its flexible data model allows it to store both timeseries and
|
||||
non-timeseries data, even in the same datasource.
|
||||
|
||||
To achieve best-case compression and query performance in Druid for timeseries data, it is important to partition and
|
||||
|
|
|
@ -119,7 +119,11 @@ Only the COUNT aggregation can accept DISTINCT.
|
|||
|`MAX(expr)`|Takes the maximum of numbers.|
|
||||
|`AVG(expr)`|Averages numbers.|
|
||||
|`APPROX_COUNT_DISTINCT(expr)`|Counts distinct values of expr, which can be a regular column or a hyperUnique column. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`.|
|
||||
|`APPROX_QUANTILE(expr, probability, [resolution])`|Computes approximate quantiles on numeric or approxHistogram exprs. The "probability" should be between 0 and 1 (exclusive). The "resolution" is the number of centroids to use for the computation. Higher resolutions will give more precise results but also have higher overhead. If not provided, the default resolution is 50. The [approximate histogram extension](../development/extensions-core/approximate-histograms.html) must be loaded to use this function.|
|
||||
|`APPROX_COUNT_DISTINCT_DS_HLL(expr, [lgK, tgtHllType])`|Counts distinct values of expr, which can be a regular column or an [HLL sketch](../development/extensions-core/datasketches-hll.html) column. The `lgK` and `tgtHllType` parameters are described in the HLL sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extensions.html) must be loaded to use this function.|
|
||||
|`APPROX_COUNT_DISTINCT_DS_THETA(expr, [size])`|Counts distinct values of expr, which can be a regular column or a [Theta sketch](../development/extensions-core/datasketches-theta.html) column. The `size` parameter is described in the Theta sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extensions.html) must be loaded to use this function.|
|
||||
|`APPROX_QUANTILE(expr, probability, [resolution])`|Computes approximate quantiles on numeric or [approxHistogram](../development/extensions-core/approximate-histograms.html#approximate-histogram-aggregator) exprs. The "probability" should be between 0 and 1 (exclusive). The "resolution" is the number of centroids to use for the computation. Higher resolutions will give more precise results but also have higher overhead. If not provided, the default resolution is 50. The [approximate histogram extension](../development/extensions-core/approximate-histograms.html) must be loaded to use this function.|
|
||||
|`APPROX_QUANTILE_DS(expr, probability, [k])`|Computes approximate quantiles on numeric or [Quantiles sketch](../development/extensions-core/datasketches-quantiles.html) exprs. The "probability" should be between 0 and 1 (exclusive). The `k` parameter is described in the Quantiles sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extensions.html) must be loaded to use this function.|
|
||||
|`APPROX_QUANTILE_FIXED_BUCKETS(expr, probability, numBuckets, lowerLimit, upperLimit, [outlierHandlingMode])`|Computes approximate quantiles on numeric or [fixed buckets histogram](../development/extensions-core/approximate-histograms.html#fixed-buckets-histogram) exprs. The "probability" should be between 0 and 1 (exclusive). The `numBuckets`, `lowerLimit`, `upperLimit`, and `outlierHandlingMode` parameters are described in the fixed buckets histogram documentation. The [approximate histogram extension](../development/extensions-core/approximate-histograms.html) must be loaded to use this function.|
|
||||
|`BLOOM_FILTER(expr, numEntries)`|Computes a bloom filter from values produced by `expr`, with `numEntries` maximum number of distinct values before false positve rate increases. See [bloom filter extension](../development/extensions-core/bloom-filter.html) documentation for additional details.|
|
||||
|
||||
### Numeric functions
|
||||
|
@ -698,6 +702,8 @@ The Druid SQL server is configured through the following properties on the Broke
|
|||
|`druid.sql.planner.useFallback`|Whether to evaluate operations on the Broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
|
||||
|`druid.sql.planner.requireTimeCondition`|Whether to require SQL to have filter conditions on __time column so that all generated native queries will have user specified intervals. If true, all queries wihout filter condition on __time column will fail|false|
|
||||
|`druid.sql.planner.sqlTimeZone`|Sets the default time zone for the server, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|UTC|
|
||||
|`druid.sql.planner.metadataSegmentCacheEnable`|Whether to keep a cache of published segments in broker. If true, broker polls coordinator in background to get segments from metadata store and maintains a local cache. If false, coordinator's REST api will be invoked when broker needs published segments info.|false|
|
||||
|`druid.sql.planner.metadataSegmentPollPeriod`|How often to poll coordinator for published segments list if `druid.sql.planner.metadataSegmentCacheEnable` is set to true. Poll period is in milliseconds. |60000|
|
||||
|
||||
## SQL Metrics
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -32,7 +32,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -25,7 +25,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -24,7 +24,7 @@
|
|||
<parent>
|
||||
<artifactId>druid</artifactId>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -24,7 +24,7 @@
|
|||
<parent>
|
||||
<artifactId>druid</artifactId>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
|
|
@ -24,7 +24,7 @@
|
|||
<parent>
|
||||
<artifactId>druid</artifactId>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
|
|
@ -58,7 +58,7 @@ public class MaterializedViewQueryQueryToolChestTest
|
|||
.put(
|
||||
TimeseriesQuery.class,
|
||||
new TimeseriesQueryQueryToolChest(
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
|
||||
)
|
||||
)
|
||||
.build()
|
||||
|
|
|
@ -31,7 +31,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<artifactId>druid</artifactId>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@
|
|||
<parent>
|
||||
<artifactId>druid</artifactId>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<artifactId>druid</artifactId>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
|
|
@ -23,7 +23,7 @@
|
|||
<parent>
|
||||
<artifactId>druid</artifactId>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -109,7 +109,7 @@ public class MapVirtualColumnGroupByTest
|
|||
strategySelector,
|
||||
new GroupByQueryQueryToolChest(
|
||||
strategySelector,
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -70,7 +70,7 @@ public class MapVirtualColumnSelectTest
|
|||
SelectQueryRunnerFactory factory = new SelectQueryRunnerFactory(
|
||||
new SelectQueryQueryToolChest(
|
||||
new DefaultObjectMapper(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
|
||||
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
|
||||
selectConfigSupplier
|
||||
),
|
||||
new SelectQueryEngine(),
|
||||
|
|
|
@ -69,7 +69,7 @@ public class MapVirtualColumnTopNTest
|
|||
new StupidPool<>("map-virtual-column-test", () -> ByteBuffer.allocate(1024)),
|
||||
new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
);
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
@ -56,6 +56,12 @@
|
|||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-sql</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
|
@ -131,6 +137,20 @@
|
|||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-sql</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
|
|
|
@ -39,11 +39,11 @@ import java.util.Objects;
|
|||
* Base class for both build and merge factories
|
||||
* @author Alexander Saydakov
|
||||
*/
|
||||
abstract class HllSketchAggregatorFactory extends AggregatorFactory
|
||||
public abstract class HllSketchAggregatorFactory extends AggregatorFactory
|
||||
{
|
||||
|
||||
static final int DEFAULT_LG_K = 12;
|
||||
static final TgtHllType DEFAULT_TGT_HLL_TYPE = TgtHllType.HLL_4;
|
||||
public static final int DEFAULT_LG_K = 12;
|
||||
public static final TgtHllType DEFAULT_TGT_HLL_TYPE = TgtHllType.HLL_4;
|
||||
|
||||
static final Comparator<HllSketch> COMPARATOR =
|
||||
Comparator.nullsFirst(Comparator.comparingDouble(HllSketch::getEstimate));
|
||||
|
|
|
@ -22,10 +22,13 @@ package org.apache.druid.query.aggregation.datasketches.hll;
|
|||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.inject.Binder;
|
||||
import com.yahoo.sketches.hll.HllSketch;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchSqlAggregator;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
import org.apache.druid.sql.guice.SqlBindings;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -48,15 +51,8 @@ public class HllSketchModule implements DruidModule
|
|||
@Override
|
||||
public void configure(final Binder binder)
|
||||
{
|
||||
if (ComplexMetrics.getSerdeForType(TYPE_NAME) == null) {
|
||||
ComplexMetrics.registerSerde(TYPE_NAME, new HllSketchMergeComplexMetricSerde());
|
||||
}
|
||||
if (ComplexMetrics.getSerdeForType(BUILD_TYPE_NAME) == null) {
|
||||
ComplexMetrics.registerSerde(BUILD_TYPE_NAME, new HllSketchBuildComplexMetricSerde());
|
||||
}
|
||||
if (ComplexMetrics.getSerdeForType(MERGE_TYPE_NAME) == null) {
|
||||
ComplexMetrics.registerSerde(MERGE_TYPE_NAME, new HllSketchMergeComplexMetricSerde());
|
||||
}
|
||||
registerSerde();
|
||||
SqlBindings.addAggregator(binder, HllSketchSqlAggregator.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -74,4 +70,17 @@ public class HllSketchModule implements DruidModule
|
|||
);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static void registerSerde()
|
||||
{
|
||||
if (ComplexMetrics.getSerdeForType(TYPE_NAME) == null) {
|
||||
ComplexMetrics.registerSerde(TYPE_NAME, new HllSketchMergeComplexMetricSerde());
|
||||
}
|
||||
if (ComplexMetrics.getSerdeForType(BUILD_TYPE_NAME) == null) {
|
||||
ComplexMetrics.registerSerde(BUILD_TYPE_NAME, new HllSketchBuildComplexMetricSerde());
|
||||
}
|
||||
if (ComplexMetrics.getSerdeForType(MERGE_TYPE_NAME) == null) {
|
||||
ComplexMetrics.registerSerde(MERGE_TYPE_NAME, new HllSketchMergeComplexMetricSerde());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,202 @@
|
|||
/*
|
||||
* 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.query.aggregation.datasketches.hll.sql;
|
||||
|
||||
import org.apache.calcite.rel.core.AggregateCall;
|
||||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.sql.SqlAggFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.type.InferTypes;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.ReturnTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
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.post.FinalizingFieldAccessPostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.sql.calcite.aggregation.Aggregation;
|
||||
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.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.table.RowSignature;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
public class HllSketchSqlAggregator implements SqlAggregator
|
||||
{
|
||||
private static final SqlAggFunction FUNCTION_INSTANCE = new HllSketchSqlAggFunction();
|
||||
private static final String NAME = "APPROX_COUNT_DISTINCT_DS_HLL";
|
||||
|
||||
@Override
|
||||
public SqlAggFunction calciteFunction()
|
||||
{
|
||||
return FUNCTION_INSTANCE;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Aggregation toDruidAggregation(
|
||||
PlannerContext plannerContext,
|
||||
RowSignature rowSignature,
|
||||
RexBuilder rexBuilder,
|
||||
String name,
|
||||
AggregateCall aggregateCall,
|
||||
Project project,
|
||||
List<Aggregation> existingAggregations,
|
||||
boolean finalizeAggregations
|
||||
)
|
||||
{
|
||||
// Don't use Aggregations.getArgumentsForSimpleAggregator, since it won't let us use direct column access
|
||||
// for string columns.
|
||||
final RexNode columnRexNode = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(0)
|
||||
);
|
||||
|
||||
final DruidExpression columnArg = Expressions.toDruidExpression(plannerContext, rowSignature, columnRexNode);
|
||||
if (columnArg == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final int logK;
|
||||
if (aggregateCall.getArgList().size() >= 2) {
|
||||
final RexNode logKarg = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(1)
|
||||
);
|
||||
|
||||
if (!logKarg.isA(SqlKind.LITERAL)) {
|
||||
// logK must be a literal in order to plan.
|
||||
return null;
|
||||
}
|
||||
|
||||
logK = ((Number) RexLiteral.value(logKarg)).intValue();
|
||||
} else {
|
||||
logK = HllSketchAggregatorFactory.DEFAULT_LG_K;
|
||||
}
|
||||
|
||||
final String tgtHllType;
|
||||
if (aggregateCall.getArgList().size() >= 3) {
|
||||
final RexNode tgtHllTypeArg = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(2)
|
||||
);
|
||||
|
||||
if (!tgtHllTypeArg.isA(SqlKind.LITERAL)) {
|
||||
// tgtHllType must be a literal in order to plan.
|
||||
return null;
|
||||
}
|
||||
|
||||
tgtHllType = RexLiteral.stringValue(tgtHllTypeArg);
|
||||
} else {
|
||||
tgtHllType = HllSketchAggregatorFactory.DEFAULT_TGT_HLL_TYPE.name();
|
||||
}
|
||||
|
||||
final List<VirtualColumn> virtualColumns = new ArrayList<>();
|
||||
final AggregatorFactory aggregatorFactory;
|
||||
final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name;
|
||||
|
||||
if (columnArg.isDirectColumnAccess() && rowSignature.getColumnType(columnArg.getDirectColumn()) == ValueType.COMPLEX) {
|
||||
aggregatorFactory = new HllSketchMergeAggregatorFactory(aggregatorName, columnArg.getDirectColumn(), logK, tgtHllType);
|
||||
} else {
|
||||
final SqlTypeName sqlTypeName = columnRexNode.getType().getSqlTypeName();
|
||||
final ValueType inputType = Calcites.getValueTypeForSqlTypeName(sqlTypeName);
|
||||
if (inputType == null) {
|
||||
throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, aggregatorName);
|
||||
}
|
||||
|
||||
final DimensionSpec dimensionSpec;
|
||||
|
||||
if (columnArg.isDirectColumnAccess()) {
|
||||
dimensionSpec = columnArg.getSimpleExtraction().toDimensionSpec(null, inputType);
|
||||
} else {
|
||||
final ExpressionVirtualColumn virtualColumn = columnArg.toVirtualColumn(
|
||||
Calcites.makePrefixedName(name, "v"),
|
||||
inputType,
|
||||
plannerContext.getExprMacroTable()
|
||||
);
|
||||
dimensionSpec = new DefaultDimensionSpec(virtualColumn.getOutputName(), null, inputType);
|
||||
virtualColumns.add(virtualColumn);
|
||||
}
|
||||
|
||||
aggregatorFactory = new HllSketchBuildAggregatorFactory(
|
||||
aggregatorName,
|
||||
dimensionSpec.getDimension(),
|
||||
logK,
|
||||
tgtHllType
|
||||
);
|
||||
}
|
||||
|
||||
return Aggregation.create(
|
||||
virtualColumns,
|
||||
Collections.singletonList(aggregatorFactory),
|
||||
finalizeAggregations ? new FinalizingFieldAccessPostAggregator(
|
||||
name,
|
||||
aggregatorFactory.getName()
|
||||
) : null
|
||||
);
|
||||
}
|
||||
|
||||
private static class HllSketchSqlAggFunction extends SqlAggFunction
|
||||
{
|
||||
private static final String SIGNATURE = "'" + NAME + "(column, lgK, tgtHllType)'\n";
|
||||
|
||||
HllSketchSqlAggFunction()
|
||||
{
|
||||
super(
|
||||
NAME,
|
||||
null,
|
||||
SqlKind.OTHER_FUNCTION,
|
||||
ReturnTypes.explicit(SqlTypeName.BIGINT),
|
||||
InferTypes.VARCHAR_1024,
|
||||
OperandTypes.or(
|
||||
OperandTypes.ANY,
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL, OperandTypes.LITERAL),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING)
|
||||
)
|
||||
),
|
||||
SqlFunctionCategory.NUMERIC,
|
||||
false,
|
||||
false
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -49,7 +49,7 @@ public class DoublesSketchAggregatorFactory extends AggregatorFactory
|
|||
public static final Comparator<DoublesSketch> COMPARATOR =
|
||||
Comparator.nullsFirst(Comparator.comparingLong(DoublesSketch::getN));
|
||||
|
||||
private static final int DEFAULT_K = 128;
|
||||
public static final int DEFAULT_K = 128;
|
||||
|
||||
// Used for sketch size estimation.
|
||||
private static final long MAX_STREAM_LENGTH = 1_000_000_000;
|
||||
|
|
|
@ -22,10 +22,13 @@ package org.apache.druid.query.aggregation.datasketches.quantiles;
|
|||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.inject.Binder;
|
||||
import com.yahoo.sketches.quantiles.DoublesSketch;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchSqlAggregator;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
import org.apache.druid.sql.guice.SqlBindings;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -44,9 +47,8 @@ public class DoublesSketchModule implements DruidModule
|
|||
@Override
|
||||
public void configure(final Binder binder)
|
||||
{
|
||||
if (ComplexMetrics.getSerdeForType(DOUBLES_SKETCH) == null) {
|
||||
ComplexMetrics.registerSerde(DOUBLES_SKETCH, new DoublesSketchComplexMetricSerde());
|
||||
}
|
||||
registerSerde();
|
||||
SqlBindings.addAggregator(binder, DoublesSketchSqlAggregator.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -65,4 +67,11 @@ public class DoublesSketchModule implements DruidModule
|
|||
);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static void registerSerde()
|
||||
{
|
||||
if (ComplexMetrics.getSerdeForType(DOUBLES_SKETCH) == null) {
|
||||
ComplexMetrics.registerSerde(DOUBLES_SKETCH, new DoublesSketchComplexMetricSerde());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,237 @@
|
|||
/*
|
||||
* 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.query.aggregation.datasketches.quantiles.sql;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.calcite.rel.core.AggregateCall;
|
||||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.sql.SqlAggFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.ReturnTypes;
|
||||
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.quantiles.DoublesSketchAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToQuantilePostAggregator;
|
||||
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.sql.calcite.aggregation.Aggregation;
|
||||
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.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.table.RowSignature;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class DoublesSketchSqlAggregator implements SqlAggregator
|
||||
{
|
||||
private static final SqlAggFunction FUNCTION_INSTANCE = new DoublesSketchSqlAggFunction();
|
||||
private static final String NAME = "APPROX_QUANTILE_DS";
|
||||
|
||||
@Override
|
||||
public SqlAggFunction calciteFunction()
|
||||
{
|
||||
return FUNCTION_INSTANCE;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Aggregation toDruidAggregation(
|
||||
final PlannerContext plannerContext,
|
||||
final RowSignature rowSignature,
|
||||
final RexBuilder rexBuilder,
|
||||
final String name,
|
||||
final AggregateCall aggregateCall,
|
||||
final Project project,
|
||||
final List<Aggregation> existingAggregations,
|
||||
final boolean finalizeAggregations
|
||||
)
|
||||
{
|
||||
final DruidExpression input = Expressions.toDruidExpression(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(0)
|
||||
)
|
||||
);
|
||||
if (input == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final AggregatorFactory aggregatorFactory;
|
||||
final String histogramName = StringUtils.format("%s:agg", name);
|
||||
final RexNode probabilityArg = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(1)
|
||||
);
|
||||
|
||||
if (!probabilityArg.isA(SqlKind.LITERAL)) {
|
||||
// Probability must be a literal in order to plan.
|
||||
return null;
|
||||
}
|
||||
|
||||
final float probability = ((Number) RexLiteral.value(probabilityArg)).floatValue();
|
||||
final int k;
|
||||
|
||||
if (aggregateCall.getArgList().size() >= 3) {
|
||||
final RexNode resolutionArg = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(2)
|
||||
);
|
||||
|
||||
if (!resolutionArg.isA(SqlKind.LITERAL)) {
|
||||
// Resolution must be a literal in order to plan.
|
||||
return null;
|
||||
}
|
||||
|
||||
k = ((Number) RexLiteral.value(resolutionArg)).intValue();
|
||||
} else {
|
||||
k = DoublesSketchAggregatorFactory.DEFAULT_K;
|
||||
}
|
||||
|
||||
// Look for existing matching aggregatorFactory.
|
||||
for (final Aggregation existing : existingAggregations) {
|
||||
for (AggregatorFactory factory : existing.getAggregatorFactories()) {
|
||||
if (factory instanceof DoublesSketchAggregatorFactory) {
|
||||
final DoublesSketchAggregatorFactory theFactory = (DoublesSketchAggregatorFactory) factory;
|
||||
|
||||
// Check input for equivalence.
|
||||
final boolean inputMatches;
|
||||
final VirtualColumn virtualInput = existing.getVirtualColumns()
|
||||
.stream()
|
||||
.filter(
|
||||
virtualColumn ->
|
||||
virtualColumn.getOutputName()
|
||||
.equals(theFactory.getFieldName())
|
||||
)
|
||||
.findFirst()
|
||||
.orElse(null);
|
||||
|
||||
if (virtualInput == null) {
|
||||
inputMatches = input.isDirectColumnAccess()
|
||||
&& input.getDirectColumn().equals(theFactory.getFieldName());
|
||||
} else {
|
||||
inputMatches = ((ExpressionVirtualColumn) virtualInput).getExpression()
|
||||
.equals(input.getExpression());
|
||||
}
|
||||
|
||||
final boolean matches = inputMatches
|
||||
&& theFactory.getK() == k;
|
||||
|
||||
if (matches) {
|
||||
// Found existing one. Use this.
|
||||
return Aggregation.create(
|
||||
ImmutableList.of(),
|
||||
new DoublesSketchToQuantilePostAggregator(
|
||||
name,
|
||||
new FieldAccessPostAggregator(
|
||||
factory.getName(),
|
||||
factory.getName()
|
||||
),
|
||||
probability
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// No existing match found. Create a new one.
|
||||
final List<VirtualColumn> virtualColumns = new ArrayList<>();
|
||||
|
||||
if (input.isDirectColumnAccess()) {
|
||||
aggregatorFactory = new DoublesSketchAggregatorFactory(
|
||||
histogramName,
|
||||
input.getDirectColumn(),
|
||||
k
|
||||
);
|
||||
} else {
|
||||
final ExpressionVirtualColumn virtualColumn = input.toVirtualColumn(
|
||||
StringUtils.format("%s:v", name),
|
||||
ValueType.FLOAT,
|
||||
plannerContext.getExprMacroTable()
|
||||
);
|
||||
virtualColumns.add(virtualColumn);
|
||||
aggregatorFactory = new DoublesSketchAggregatorFactory(
|
||||
histogramName,
|
||||
virtualColumn.getOutputName(),
|
||||
k
|
||||
);
|
||||
}
|
||||
|
||||
return Aggregation.create(
|
||||
virtualColumns,
|
||||
ImmutableList.of(aggregatorFactory),
|
||||
new DoublesSketchToQuantilePostAggregator(
|
||||
name,
|
||||
new FieldAccessPostAggregator(
|
||||
histogramName,
|
||||
histogramName
|
||||
),
|
||||
probability
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private static class DoublesSketchSqlAggFunction extends SqlAggFunction
|
||||
{
|
||||
private static final String SIGNATURE1 = "'" + NAME + "(column, probability)'\n";
|
||||
private static final String SIGNATURE2 = "'" + NAME + "(column, probability, k)'\n";
|
||||
|
||||
DoublesSketchSqlAggFunction()
|
||||
{
|
||||
super(
|
||||
NAME,
|
||||
null,
|
||||
SqlKind.OTHER_FUNCTION,
|
||||
ReturnTypes.explicit(SqlTypeName.DOUBLE),
|
||||
null,
|
||||
OperandTypes.or(
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(SIGNATURE1, OperandTypes.ANY, OperandTypes.LITERAL),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC)
|
||||
),
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(SIGNATURE2, OperandTypes.ANY, OperandTypes.LITERAL, OperandTypes.LITERAL),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.EXACT_NUMERIC)
|
||||
)
|
||||
),
|
||||
SqlFunctionCategory.NUMERIC,
|
||||
false,
|
||||
false
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
|
|||
import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
|
||||
import org.apache.druid.query.aggregation.AggregatorUtil;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -38,10 +39,10 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory
|
|||
public SketchMergeAggregatorFactory(
|
||||
@JsonProperty("name") String name,
|
||||
@JsonProperty("fieldName") String fieldName,
|
||||
@JsonProperty("size") Integer size,
|
||||
@JsonProperty("shouldFinalize") Boolean shouldFinalize,
|
||||
@JsonProperty("isInputThetaSketch") Boolean isInputThetaSketch,
|
||||
@JsonProperty("errorBoundsStdDev") Integer errorBoundsStdDev
|
||||
@Nullable @JsonProperty("size") Integer size,
|
||||
@Nullable @JsonProperty("shouldFinalize") Boolean shouldFinalize,
|
||||
@Nullable @JsonProperty("isInputThetaSketch") Boolean isInputThetaSketch,
|
||||
@Nullable @JsonProperty("errorBoundsStdDev") Integer errorBoundsStdDev
|
||||
)
|
||||
{
|
||||
super(name, fieldName, size, AggregatorUtil.SKETCH_MERGE_CACHE_TYPE_ID);
|
||||
|
|
|
@ -22,9 +22,12 @@ package org.apache.druid.query.aggregation.datasketches.theta;
|
|||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.inject.Binder;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
import org.apache.druid.query.aggregation.datasketches.theta.sql.ThetaSketchSqlAggregator;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
import org.apache.druid.sql.guice.SqlBindings;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -44,17 +47,8 @@ public class SketchModule implements DruidModule
|
|||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
if (ComplexMetrics.getSerdeForType(THETA_SKETCH) == null) {
|
||||
ComplexMetrics.registerSerde(THETA_SKETCH, new SketchMergeComplexMetricSerde());
|
||||
}
|
||||
|
||||
if (ComplexMetrics.getSerdeForType(THETA_SKETCH_MERGE_AGG) == null) {
|
||||
ComplexMetrics.registerSerde(THETA_SKETCH_MERGE_AGG, new SketchMergeComplexMetricSerde());
|
||||
}
|
||||
|
||||
if (ComplexMetrics.getSerdeForType(THETA_SKETCH_BUILD_AGG) == null) {
|
||||
ComplexMetrics.registerSerde(THETA_SKETCH_BUILD_AGG, new SketchBuildComplexMetricSerde());
|
||||
}
|
||||
registerSerde();
|
||||
SqlBindings.addAggregator(binder, ThetaSketchSqlAggregator.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -71,4 +65,20 @@ public class SketchModule implements DruidModule
|
|||
.addSerializer(SketchHolder.class, new SketchHolderJsonSerializer())
|
||||
);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static void registerSerde()
|
||||
{
|
||||
if (ComplexMetrics.getSerdeForType(THETA_SKETCH) == null) {
|
||||
ComplexMetrics.registerSerde(THETA_SKETCH, new SketchMergeComplexMetricSerde());
|
||||
}
|
||||
|
||||
if (ComplexMetrics.getSerdeForType(THETA_SKETCH_MERGE_AGG) == null) {
|
||||
ComplexMetrics.registerSerde(THETA_SKETCH_MERGE_AGG, new SketchMergeComplexMetricSerde());
|
||||
}
|
||||
|
||||
if (ComplexMetrics.getSerdeForType(THETA_SKETCH_BUILD_AGG) == null) {
|
||||
ComplexMetrics.registerSerde(THETA_SKETCH_BUILD_AGG, new SketchBuildComplexMetricSerde());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,192 @@
|
|||
/*
|
||||
* 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.query.aggregation.datasketches.theta.sql;
|
||||
|
||||
import org.apache.calcite.rel.core.AggregateCall;
|
||||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.sql.SqlAggFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.type.InferTypes;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.ReturnTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.theta.SketchAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.sql.calcite.aggregation.Aggregation;
|
||||
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.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.table.RowSignature;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
public class ThetaSketchSqlAggregator implements SqlAggregator
|
||||
{
|
||||
private static final SqlAggFunction FUNCTION_INSTANCE = new ThetaSketchSqlAggFunction();
|
||||
private static final String NAME = "APPROX_COUNT_DISTINCT_DS_THETA";
|
||||
|
||||
@Override
|
||||
public SqlAggFunction calciteFunction()
|
||||
{
|
||||
return FUNCTION_INSTANCE;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Aggregation toDruidAggregation(
|
||||
PlannerContext plannerContext,
|
||||
RowSignature rowSignature,
|
||||
RexBuilder rexBuilder,
|
||||
String name,
|
||||
AggregateCall aggregateCall,
|
||||
Project project,
|
||||
List<Aggregation> existingAggregations,
|
||||
boolean finalizeAggregations
|
||||
)
|
||||
{
|
||||
// Don't use Aggregations.getArgumentsForSimpleAggregator, since it won't let us use direct column access
|
||||
// for string columns.
|
||||
final RexNode columnRexNode = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(0)
|
||||
);
|
||||
|
||||
final DruidExpression columnArg = Expressions.toDruidExpression(plannerContext, rowSignature, columnRexNode);
|
||||
if (columnArg == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final int sketchSize;
|
||||
if (aggregateCall.getArgList().size() >= 2) {
|
||||
final RexNode sketchSizeArg = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(1)
|
||||
);
|
||||
|
||||
if (!sketchSizeArg.isA(SqlKind.LITERAL)) {
|
||||
// logK must be a literal in order to plan.
|
||||
return null;
|
||||
}
|
||||
|
||||
sketchSize = ((Number) RexLiteral.value(sketchSizeArg)).intValue();
|
||||
} else {
|
||||
sketchSize = SketchAggregatorFactory.DEFAULT_MAX_SKETCH_SIZE;
|
||||
}
|
||||
|
||||
final List<VirtualColumn> virtualColumns = new ArrayList<>();
|
||||
final AggregatorFactory aggregatorFactory;
|
||||
final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name;
|
||||
|
||||
if (columnArg.isDirectColumnAccess() && rowSignature.getColumnType(columnArg.getDirectColumn()) == ValueType.COMPLEX) {
|
||||
aggregatorFactory = new SketchMergeAggregatorFactory(
|
||||
aggregatorName,
|
||||
columnArg.getDirectColumn(),
|
||||
sketchSize,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
} else {
|
||||
final SqlTypeName sqlTypeName = columnRexNode.getType().getSqlTypeName();
|
||||
final ValueType inputType = Calcites.getValueTypeForSqlTypeName(sqlTypeName);
|
||||
if (inputType == null) {
|
||||
throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, aggregatorName);
|
||||
}
|
||||
|
||||
final DimensionSpec dimensionSpec;
|
||||
|
||||
if (columnArg.isDirectColumnAccess()) {
|
||||
dimensionSpec = columnArg.getSimpleExtraction().toDimensionSpec(null, inputType);
|
||||
} else {
|
||||
final ExpressionVirtualColumn virtualColumn = columnArg.toVirtualColumn(
|
||||
Calcites.makePrefixedName(name, "v"),
|
||||
inputType,
|
||||
plannerContext.getExprMacroTable()
|
||||
);
|
||||
dimensionSpec = new DefaultDimensionSpec(virtualColumn.getOutputName(), null, inputType);
|
||||
virtualColumns.add(virtualColumn);
|
||||
}
|
||||
|
||||
aggregatorFactory = new SketchMergeAggregatorFactory(
|
||||
aggregatorName,
|
||||
dimensionSpec.getDimension(),
|
||||
sketchSize,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
return Aggregation.create(
|
||||
virtualColumns,
|
||||
Collections.singletonList(aggregatorFactory),
|
||||
finalizeAggregations ? new FinalizingFieldAccessPostAggregator(
|
||||
name,
|
||||
aggregatorFactory.getName()
|
||||
) : null
|
||||
);
|
||||
}
|
||||
|
||||
private static class ThetaSketchSqlAggFunction extends SqlAggFunction
|
||||
{
|
||||
private static final String SIGNATURE = "'" + NAME + "(column, size)'\n";
|
||||
|
||||
ThetaSketchSqlAggFunction()
|
||||
{
|
||||
super(
|
||||
NAME,
|
||||
null,
|
||||
SqlKind.OTHER_FUNCTION,
|
||||
ReturnTypes.explicit(SqlTypeName.BIGINT),
|
||||
InferTypes.VARCHAR_1024,
|
||||
OperandTypes.or(
|
||||
OperandTypes.ANY,
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL),
|
||||
OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC)
|
||||
)
|
||||
),
|
||||
SqlFunctionCategory.NUMERIC,
|
||||
false,
|
||||
false
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.query.aggregation.datasketches.hll;
|
||||
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
||||
|
@ -48,10 +47,9 @@ public class HllSketchAggregatorTest
|
|||
|
||||
public HllSketchAggregatorTest(GroupByQueryConfig config)
|
||||
{
|
||||
DruidModule module = new HllSketchModule();
|
||||
module.configure(null);
|
||||
HllSketchModule.registerSerde();
|
||||
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||
module.getJacksonModules(), config, tempFolder);
|
||||
new HllSketchModule().getJacksonModules(), config, tempFolder);
|
||||
}
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
|
|
|
@ -0,0 +1,393 @@
|
|||
/*
|
||||
* 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.query.aggregation.datasketches.hll.sql;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
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.aggregation.post.ArithmeticPostAggregator;
|
||||
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.IndexBuilder;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.server.security.AuthTestUtils;
|
||||
import org.apache.druid.server.security.AuthenticationResult;
|
||||
import org.apache.druid.sql.SqlLifecycle;
|
||||
import org.apache.druid.sql.SqlLifecycleFactory;
|
||||
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
|
||||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class HllSketchSqlAggregatorTest extends CalciteTestBase
|
||||
{
|
||||
private static final String DATA_SOURCE = "foo";
|
||||
|
||||
private static QueryRunnerFactoryConglomerate conglomerate;
|
||||
private static Closer resourceCloser;
|
||||
private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
|
||||
private static final Map<String, Object> QUERY_CONTEXT_DEFAULT = ImmutableMap.of(
|
||||
PlannerContext.CTX_SQL_QUERY_ID, "dummy"
|
||||
);
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpClass()
|
||||
{
|
||||
final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
|
||||
.createQueryRunnerFactoryConglomerate();
|
||||
conglomerate = conglomerateCloserPair.lhs;
|
||||
resourceCloser = conglomerateCloserPair.rhs;
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownClass() throws IOException
|
||||
{
|
||||
resourceCloser.close();
|
||||
}
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
@Rule
|
||||
public QueryLogHook queryLogHook = QueryLogHook.create();
|
||||
|
||||
private SpecificSegmentsQuerySegmentWalker walker;
|
||||
private SqlLifecycleFactory sqlLifecycleFactory;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
HllSketchModule.registerSerde();
|
||||
for (Module mod : new HllSketchModule().getJacksonModules()) {
|
||||
CalciteTests.getJsonMapper().registerModule(mod);
|
||||
}
|
||||
|
||||
final QueryableIndex index = IndexBuilder.create()
|
||||
.tmpDir(temporaryFolder.newFolder())
|
||||
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new CountAggregatorFactory("cnt"),
|
||||
new DoubleSumAggregatorFactory("m1", "m1"),
|
||||
new HllSketchBuildAggregatorFactory(
|
||||
"hllsketch_dim1",
|
||||
"dim1",
|
||||
null,
|
||||
null
|
||||
)
|
||||
)
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.rows(CalciteTests.ROWS1)
|
||||
.buildMMappedIndex();
|
||||
|
||||
walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(index.getDataInterval())
|
||||
.version("1")
|
||||
.shardSpec(new LinearShardSpec(0))
|
||||
.build(),
|
||||
index
|
||||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(new HllSketchSqlAggregator()),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
systemSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
walker.close();
|
||||
walker = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testApproxCountDistinctHllSketch() throws Exception
|
||||
{
|
||||
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
|
||||
|
||||
final String sql = "SELECT\n"
|
||||
+ " SUM(cnt),\n"
|
||||
+ " APPROX_COUNT_DISTINCT_DS_HLL(dim2),\n" // uppercase
|
||||
+ " APPROX_COUNT_DISTINCT_DS_HLL(dim2) FILTER(WHERE dim2 <> ''),\n" // lowercase; also, filtered
|
||||
+ " APPROX_COUNT_DISTINCT_DS_HLL(SUBSTRING(dim2, 1, 1)),\n" // on extractionFn
|
||||
+ " APPROX_COUNT_DISTINCT_DS_HLL(SUBSTRING(dim2, 1, 1) || 'x'),\n" // on expression
|
||||
+ " APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1, 21, 'HLL_8'),\n" // on native HllSketch column
|
||||
+ " APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1)\n" // on native HllSketch column
|
||||
+ "FROM druid.foo";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults;
|
||||
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
6L,
|
||||
2L,
|
||||
2L,
|
||||
1L,
|
||||
2L,
|
||||
5L,
|
||||
5L
|
||||
}
|
||||
);
|
||||
} else {
|
||||
expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
6L,
|
||||
2L,
|
||||
2L,
|
||||
1L,
|
||||
1L,
|
||||
5L,
|
||||
5L
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.virtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"a3:v",
|
||||
"substring(\"dim2\", 0, 1)",
|
||||
ValueType.STRING,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
new ExpressionVirtualColumn(
|
||||
"a4:v",
|
||||
"concat(substring(\"dim2\", 0, 1),'x')",
|
||||
ValueType.STRING,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.aggregators(
|
||||
ImmutableList.of(
|
||||
new LongSumAggregatorFactory("a0", "cnt"),
|
||||
new HllSketchBuildAggregatorFactory(
|
||||
"a1",
|
||||
"dim2",
|
||||
null,
|
||||
null
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new HllSketchBuildAggregatorFactory(
|
||||
"a2",
|
||||
"dim2",
|
||||
null,
|
||||
null
|
||||
),
|
||||
BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null))
|
||||
),
|
||||
new HllSketchBuildAggregatorFactory(
|
||||
"a3",
|
||||
"a3:v",
|
||||
null,
|
||||
null
|
||||
),
|
||||
new HllSketchBuildAggregatorFactory(
|
||||
"a4",
|
||||
"a4:v",
|
||||
null,
|
||||
null
|
||||
),
|
||||
new HllSketchMergeAggregatorFactory("a5", "hllsketch_dim1", 21, "HLL_8"),
|
||||
new HllSketchMergeAggregatorFactory("a6", "hllsketch_dim1", null, null)
|
||||
)
|
||||
)
|
||||
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testAvgDailyCountDistinctHllSketch() throws Exception
|
||||
{
|
||||
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
|
||||
|
||||
final String sql = "SELECT\n"
|
||||
+ " AVG(u)\n"
|
||||
+ "FROM (SELECT FLOOR(__time TO DAY), APPROX_COUNT_DISTINCT_DS_HLL(cnt) AS u FROM druid.foo GROUP BY 1)";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
1L
|
||||
}
|
||||
);
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
Query expected = GroupByQuery.builder()
|
||||
.setDataSource(
|
||||
new QueryDataSource(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(
|
||||
Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"d0:v",
|
||||
"timestamp_floor(\"__time\",'P1D',null,'UTC')",
|
||||
ValueType.LONG,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.setDimensions(
|
||||
Collections.singletonList(
|
||||
new DefaultDimensionSpec(
|
||||
"d0:v",
|
||||
"d0",
|
||||
ValueType.LONG
|
||||
)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(
|
||||
Collections.singletonList(
|
||||
new HllSketchBuildAggregatorFactory(
|
||||
"a0:a",
|
||||
"cnt",
|
||||
null,
|
||||
null
|
||||
)
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new FinalizingFieldAccessPostAggregator("a0", "a0:a")
|
||||
)
|
||||
)
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setAggregatorSpecs(Arrays.asList(
|
||||
new LongSumAggregatorFactory("_a0:sum", "a0"),
|
||||
new CountAggregatorFactory("_a0:count")
|
||||
))
|
||||
.setPostAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new ArithmeticPostAggregator(
|
||||
"_a0",
|
||||
"quotient",
|
||||
ImmutableList.of(
|
||||
new FieldAccessPostAggregator(null, "_a0:sum"),
|
||||
new FieldAccessPostAggregator(null, "_a0:count")
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build();
|
||||
|
||||
Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(expected, actual);
|
||||
}
|
||||
}
|
|
@ -21,7 +21,6 @@ package org.apache.druid.query.aggregation.datasketches.quantiles;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
|
@ -55,8 +54,8 @@ public class DoublesSketchAggregatorTest
|
|||
|
||||
public DoublesSketchAggregatorTest(final GroupByQueryConfig config)
|
||||
{
|
||||
DruidModule module = new DoublesSketchModule();
|
||||
module.configure(null);
|
||||
DoublesSketchModule.registerSerde();
|
||||
DoublesSketchModule module = new DoublesSketchModule();
|
||||
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||
module.getJacksonModules(), config, tempFolder);
|
||||
timeSeriesHelper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper(
|
||||
|
|
|
@ -0,0 +1,408 @@
|
|||
/*
|
||||
* 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.query.aggregation.datasketches.quantiles.sql;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.PostAggregator;
|
||||
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule;
|
||||
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToQuantilePostAggregator;
|
||||
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
|
||||
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.IndexBuilder;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.server.security.AuthTestUtils;
|
||||
import org.apache.druid.server.security.AuthenticationResult;
|
||||
import org.apache.druid.sql.SqlLifecycle;
|
||||
import org.apache.druid.sql.SqlLifecycleFactory;
|
||||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class DoublesSketchSqlAggregatorTest extends CalciteTestBase
|
||||
{
|
||||
private static final String DATA_SOURCE = "foo";
|
||||
|
||||
private static QueryRunnerFactoryConglomerate conglomerate;
|
||||
private static Closer resourceCloser;
|
||||
private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
|
||||
private static final Map<String, Object> QUERY_CONTEXT_DEFAULT = ImmutableMap.of(
|
||||
PlannerContext.CTX_SQL_QUERY_ID, "dummy"
|
||||
);
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpClass()
|
||||
{
|
||||
final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
|
||||
.createQueryRunnerFactoryConglomerate();
|
||||
conglomerate = conglomerateCloserPair.lhs;
|
||||
resourceCloser = conglomerateCloserPair.rhs;
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownClass() throws IOException
|
||||
{
|
||||
resourceCloser.close();
|
||||
}
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
@Rule
|
||||
public QueryLogHook queryLogHook = QueryLogHook.create();
|
||||
|
||||
private SpecificSegmentsQuerySegmentWalker walker;
|
||||
private SqlLifecycleFactory sqlLifecycleFactory;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
DoublesSketchModule.registerSerde();
|
||||
for (Module mod : new DoublesSketchModule().getJacksonModules()) {
|
||||
CalciteTests.getJsonMapper().registerModule(mod);
|
||||
}
|
||||
|
||||
final QueryableIndex index = IndexBuilder.create()
|
||||
.tmpDir(temporaryFolder.newFolder())
|
||||
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new CountAggregatorFactory("cnt"),
|
||||
new DoubleSumAggregatorFactory("m1", "m1"),
|
||||
new DoublesSketchAggregatorFactory(
|
||||
"qsketch_m1",
|
||||
"m1",
|
||||
128
|
||||
)
|
||||
)
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.rows(CalciteTests.ROWS1)
|
||||
.buildMMappedIndex();
|
||||
|
||||
walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(index.getDataInterval())
|
||||
.version("1")
|
||||
.shardSpec(new LinearShardSpec(0))
|
||||
.build(),
|
||||
index
|
||||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(new DoublesSketchSqlAggregator()),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
systemSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
walker.close();
|
||||
walker = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQuantileOnFloatAndLongs() throws Exception
|
||||
{
|
||||
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
|
||||
final String sql = "SELECT\n"
|
||||
+ "APPROX_QUANTILE_DS(m1, 0.01),\n"
|
||||
+ "APPROX_QUANTILE_DS(m1, 0.5, 64),\n"
|
||||
+ "APPROX_QUANTILE_DS(m1, 0.98, 256),\n"
|
||||
+ "APPROX_QUANTILE_DS(m1, 0.99),\n"
|
||||
+ "APPROX_QUANTILE_DS(m1 * 2, 0.97),\n"
|
||||
+ "APPROX_QUANTILE_DS(m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE_DS(m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
|
||||
+ "APPROX_QUANTILE_DS(m1, 0.999) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE_DS(cnt, 0.5)\n"
|
||||
+ "FROM foo";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
1.0,
|
||||
4.0,
|
||||
6.0,
|
||||
6.0,
|
||||
12.0,
|
||||
6.0,
|
||||
5.0,
|
||||
6.0,
|
||||
1.0
|
||||
}
|
||||
);
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.virtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"a4:v",
|
||||
"(\"m1\" * 2)",
|
||||
ValueType.FLOAT,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.aggregators(ImmutableList.of(
|
||||
new DoublesSketchAggregatorFactory("a0:agg", "m1", null),
|
||||
new DoublesSketchAggregatorFactory("a1:agg", "m1", 64),
|
||||
new DoublesSketchAggregatorFactory("a2:agg", "m1", 256),
|
||||
new DoublesSketchAggregatorFactory("a4:agg", "a4:v", null),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a5:agg", "m1", null),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a6:agg", "m1", null),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
),
|
||||
new DoublesSketchAggregatorFactory("a8:agg", "cnt", null)
|
||||
))
|
||||
.postAggregators(
|
||||
new DoublesSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.01f),
|
||||
new DoublesSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.50f),
|
||||
new DoublesSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a2:agg"), 0.98f),
|
||||
new DoublesSketchToQuantilePostAggregator("a3", makeFieldAccessPostAgg("a0:agg"), 0.99f),
|
||||
new DoublesSketchToQuantilePostAggregator("a4", makeFieldAccessPostAgg("a4:agg"), 0.97f),
|
||||
new DoublesSketchToQuantilePostAggregator("a5", makeFieldAccessPostAgg("a5:agg"), 0.99f),
|
||||
new DoublesSketchToQuantilePostAggregator("a6", makeFieldAccessPostAgg("a6:agg"), 0.999f),
|
||||
new DoublesSketchToQuantilePostAggregator("a7", makeFieldAccessPostAgg("a5:agg"), 0.999f),
|
||||
new DoublesSketchToQuantilePostAggregator("a8", makeFieldAccessPostAgg("a8:agg"), 0.50f)
|
||||
)
|
||||
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQuantileOnComplexColumn() throws Exception
|
||||
{
|
||||
SqlLifecycle lifecycle = sqlLifecycleFactory.factorize();
|
||||
final String sql = "SELECT\n"
|
||||
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.01),\n"
|
||||
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.5, 64),\n"
|
||||
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.98, 256),\n"
|
||||
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.99),\n"
|
||||
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
|
||||
+ "APPROX_QUANTILE_DS(qsketch_m1, 0.999) FILTER(WHERE dim1 = 'abc')\n"
|
||||
+ "FROM foo";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = lifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
1.0,
|
||||
4.0,
|
||||
6.0,
|
||||
6.0,
|
||||
6.0,
|
||||
5.0,
|
||||
6.0
|
||||
}
|
||||
);
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(ImmutableList.of(
|
||||
new DoublesSketchAggregatorFactory("a0:agg", "qsketch_m1", null),
|
||||
new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", 64),
|
||||
new DoublesSketchAggregatorFactory("a2:agg", "qsketch_m1", 256),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a4:agg", "qsketch_m1", null),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new DoublesSketchAggregatorFactory("a5:agg", "qsketch_m1", null),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
)
|
||||
))
|
||||
.postAggregators(
|
||||
new DoublesSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.01f),
|
||||
new DoublesSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.50f),
|
||||
new DoublesSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a2:agg"), 0.98f),
|
||||
new DoublesSketchToQuantilePostAggregator("a3", makeFieldAccessPostAgg("a0:agg"), 0.99f),
|
||||
new DoublesSketchToQuantilePostAggregator("a4", makeFieldAccessPostAgg("a4:agg"), 0.99f),
|
||||
new DoublesSketchToQuantilePostAggregator("a5", makeFieldAccessPostAgg("a5:agg"), 0.999f),
|
||||
new DoublesSketchToQuantilePostAggregator("a6", makeFieldAccessPostAgg("a4:agg"), 0.999f)
|
||||
)
|
||||
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQuantileOnInnerQuery() throws Exception
|
||||
{
|
||||
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
|
||||
final String sql = "SELECT AVG(x), APPROX_QUANTILE_DS(x, 0.98)\n"
|
||||
+ "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
expectedResults = ImmutableList.of(new Object[]{7.0, 11.0});
|
||||
} else {
|
||||
expectedResults = ImmutableList.of(new Object[]{5.25, 8.0});
|
||||
}
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(
|
||||
new QueryDataSource(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(new DefaultDimensionSpec("dim2", "d0"))
|
||||
.setAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new DoubleSumAggregatorFactory("a0", "m1")
|
||||
)
|
||||
)
|
||||
.setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setAggregatorSpecs(
|
||||
new DoubleSumAggregatorFactory("_a0:sum", "a0"),
|
||||
new CountAggregatorFactory("_a0:count"),
|
||||
new DoublesSketchAggregatorFactory(
|
||||
"_a1:agg",
|
||||
"a0",
|
||||
null
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new ArithmeticPostAggregator(
|
||||
"_a0",
|
||||
"quotient",
|
||||
ImmutableList.of(
|
||||
new FieldAccessPostAggregator(null, "_a0:sum"),
|
||||
new FieldAccessPostAggregator(null, "_a0:count")
|
||||
)
|
||||
),
|
||||
new DoublesSketchToQuantilePostAggregator("_a1", makeFieldAccessPostAgg("_a1:agg"), 0.98f)
|
||||
)
|
||||
)
|
||||
.setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
|
||||
private static PostAggregator makeFieldAccessPostAgg(String name)
|
||||
{
|
||||
return new FieldAccessPostAggregator(name, name);
|
||||
}
|
||||
}
|
|
@ -71,10 +71,9 @@ public class SketchAggregationTest
|
|||
|
||||
public SketchAggregationTest(final GroupByQueryConfig config)
|
||||
{
|
||||
SketchModule sm = new SketchModule();
|
||||
sm.configure(null);
|
||||
SketchModule.registerSerde();
|
||||
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||
sm.getJacksonModules(),
|
||||
new SketchModule().getJacksonModules(),
|
||||
config,
|
||||
tempFolder
|
||||
);
|
||||
|
|
|
@ -83,8 +83,8 @@ public class SketchAggregationWithSimpleDataTest
|
|||
@Before
|
||||
public void setup() throws Exception
|
||||
{
|
||||
SketchModule.registerSerde();
|
||||
sm = new SketchModule();
|
||||
sm.configure(null);
|
||||
try (
|
||||
final AggregationTestHelper toolchest = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||
sm.getJacksonModules(),
|
||||
|
@ -264,8 +264,8 @@ public class SketchAggregationWithSimpleDataTest
|
|||
@Test
|
||||
public void testSimpleDataIngestAndSelectQuery() throws Exception
|
||||
{
|
||||
SketchModule.registerSerde();
|
||||
SketchModule sm = new SketchModule();
|
||||
sm.configure(null);
|
||||
AggregationTestHelper selectQueryAggregationTestHelper = AggregationTestHelper.createSelectQueryAggregationTestHelper(
|
||||
sm.getJacksonModules(),
|
||||
tempFolder
|
||||
|
|
|
@ -0,0 +1,402 @@
|
|||
/*
|
||||
* 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.query.aggregation.datasketches.theta.sql;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.datasketches.theta.SketchModule;
|
||||
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
|
||||
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.IndexBuilder;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.server.security.AuthTestUtils;
|
||||
import org.apache.druid.server.security.AuthenticationResult;
|
||||
import org.apache.druid.sql.SqlLifecycle;
|
||||
import org.apache.druid.sql.SqlLifecycleFactory;
|
||||
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
|
||||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class ThetaSketchSqlAggregatorTest extends CalciteTestBase
|
||||
{
|
||||
private static final String DATA_SOURCE = "foo";
|
||||
|
||||
private static QueryRunnerFactoryConglomerate conglomerate;
|
||||
private static Closer resourceCloser;
|
||||
private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
|
||||
private static final Map<String, Object> QUERY_CONTEXT_DEFAULT = ImmutableMap.of(
|
||||
PlannerContext.CTX_SQL_QUERY_ID, "dummy"
|
||||
);
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpClass()
|
||||
{
|
||||
final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
|
||||
.createQueryRunnerFactoryConglomerate();
|
||||
conglomerate = conglomerateCloserPair.lhs;
|
||||
resourceCloser = conglomerateCloserPair.rhs;
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownClass() throws IOException
|
||||
{
|
||||
resourceCloser.close();
|
||||
}
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
@Rule
|
||||
public QueryLogHook queryLogHook = QueryLogHook.create();
|
||||
|
||||
private SpecificSegmentsQuerySegmentWalker walker;
|
||||
private SqlLifecycleFactory sqlLifecycleFactory;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
SketchModule.registerSerde();
|
||||
for (Module mod : new SketchModule().getJacksonModules()) {
|
||||
CalciteTests.getJsonMapper().registerModule(mod);
|
||||
}
|
||||
|
||||
final QueryableIndex index = IndexBuilder.create()
|
||||
.tmpDir(temporaryFolder.newFolder())
|
||||
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new CountAggregatorFactory("cnt"),
|
||||
new DoubleSumAggregatorFactory("m1", "m1"),
|
||||
new SketchMergeAggregatorFactory(
|
||||
"thetasketch_dim1",
|
||||
"dim1",
|
||||
null,
|
||||
false,
|
||||
false,
|
||||
null
|
||||
)
|
||||
)
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.rows(CalciteTests.ROWS1)
|
||||
.buildMMappedIndex();
|
||||
|
||||
walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(index.getDataInterval())
|
||||
.version("1")
|
||||
.shardSpec(new LinearShardSpec(0))
|
||||
.build(),
|
||||
index
|
||||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(new ThetaSketchSqlAggregator()),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
systemSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
walker.close();
|
||||
walker = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testApproxCountDistinctThetaSketch() throws Exception
|
||||
{
|
||||
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
|
||||
final String sql = "SELECT\n"
|
||||
+ " SUM(cnt),\n"
|
||||
+ " APPROX_COUNT_DISTINCT_DS_THETA(dim2),\n" // uppercase
|
||||
+ " APPROX_COUNT_DISTINCT_DS_THETA(dim2) FILTER(WHERE dim2 <> ''),\n" // lowercase; also, filtered
|
||||
+ " APPROX_COUNT_DISTINCT_DS_THETA(SUBSTRING(dim2, 1, 1)),\n" // on extractionFn
|
||||
+ " APPROX_COUNT_DISTINCT_DS_THETA(SUBSTRING(dim2, 1, 1) || 'x'),\n" // on expression
|
||||
+ " APPROX_COUNT_DISTINCT_DS_THETA(thetasketch_dim1, 32768),\n" // on native theta sketch column
|
||||
+ " APPROX_COUNT_DISTINCT_DS_THETA(thetasketch_dim1)\n" // on native theta sketch column
|
||||
+ "FROM druid.foo";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults;
|
||||
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
6L,
|
||||
2L,
|
||||
2L,
|
||||
1L,
|
||||
2L,
|
||||
5L,
|
||||
5L
|
||||
}
|
||||
);
|
||||
} else {
|
||||
expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
6L,
|
||||
2L,
|
||||
2L,
|
||||
1L,
|
||||
1L,
|
||||
5L,
|
||||
5L
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.virtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"a3:v",
|
||||
"substring(\"dim2\", 0, 1)",
|
||||
ValueType.STRING,
|
||||
TestExprMacroTable.INSTANCE
|
||||
),
|
||||
new ExpressionVirtualColumn(
|
||||
"a4:v",
|
||||
"concat(substring(\"dim2\", 0, 1),'x')",
|
||||
ValueType.STRING,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.aggregators(
|
||||
ImmutableList.of(
|
||||
new LongSumAggregatorFactory("a0", "cnt"),
|
||||
new SketchMergeAggregatorFactory(
|
||||
"a1",
|
||||
"dim2",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new SketchMergeAggregatorFactory(
|
||||
"a2",
|
||||
"dim2",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
),
|
||||
BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null))
|
||||
),
|
||||
new SketchMergeAggregatorFactory(
|
||||
"a3",
|
||||
"a3:v",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
),
|
||||
new SketchMergeAggregatorFactory(
|
||||
"a4",
|
||||
"a4:v",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
),
|
||||
new SketchMergeAggregatorFactory("a5", "thetasketch_dim1", 32768, null, null, null),
|
||||
new SketchMergeAggregatorFactory("a6", "thetasketch_dim1", null, null, null, null)
|
||||
)
|
||||
)
|
||||
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAvgDailyCountDistinctThetaSketch() throws Exception
|
||||
{
|
||||
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
|
||||
|
||||
final String sql = "SELECT\n"
|
||||
+ " AVG(u)\n"
|
||||
+ "FROM (SELECT FLOOR(__time TO DAY), APPROX_COUNT_DISTINCT_DS_THETA(cnt) AS u FROM druid.foo GROUP BY 1)";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
1L
|
||||
}
|
||||
);
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
Query expected = GroupByQuery.builder()
|
||||
.setDataSource(
|
||||
new QueryDataSource(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(
|
||||
Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"d0:v",
|
||||
"timestamp_floor(\"__time\",'P1D',null,'UTC')",
|
||||
ValueType.LONG,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.setDimensions(
|
||||
Collections.singletonList(
|
||||
new DefaultDimensionSpec(
|
||||
"d0:v",
|
||||
"d0",
|
||||
ValueType.LONG
|
||||
)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(
|
||||
Collections.singletonList(
|
||||
new SketchMergeAggregatorFactory(
|
||||
"a0:a",
|
||||
"cnt",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new FinalizingFieldAccessPostAggregator("a0", "a0:a")
|
||||
)
|
||||
)
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setAggregatorSpecs(Arrays.asList(
|
||||
new LongSumAggregatorFactory("_a0:sum", "a0"),
|
||||
new CountAggregatorFactory("_a0:count")
|
||||
))
|
||||
.setPostAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new ArithmeticPostAggregator(
|
||||
"_a0",
|
||||
"quotient",
|
||||
ImmutableList.of(
|
||||
new FieldAccessPostAggregator(null, "_a0:sum"),
|
||||
new FieldAccessPostAggregator(null, "_a0:count")
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build();
|
||||
|
||||
Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(expected, actual);
|
||||
}
|
||||
}
|
|
@ -32,7 +32,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -192,7 +192,7 @@ public class BloomFilterSqlAggregatorTest
|
|||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(new BloomFilterSqlAggregator()),
|
||||
ImmutableSet.of()
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -21,9 +21,11 @@ package org.apache.druid.query.aggregation.histogram;
|
|||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
import org.apache.druid.query.aggregation.histogram.sql.FixedBucketsHistogramQuantileSqlAggregator;
|
||||
import org.apache.druid.query.aggregation.histogram.sql.QuantileSqlAggregator;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
import org.apache.druid.sql.guice.SqlBindings;
|
||||
|
@ -55,6 +57,14 @@ public class ApproximateHistogramDruidModule implements DruidModule
|
|||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
registerSerde();
|
||||
SqlBindings.addAggregator(binder, QuantileSqlAggregator.class);
|
||||
SqlBindings.addAggregator(binder, FixedBucketsHistogramQuantileSqlAggregator.class);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static void registerSerde()
|
||||
{
|
||||
if (ComplexMetrics.getSerdeForType("approximateHistogram") == null) {
|
||||
ComplexMetrics.registerSerde("approximateHistogram", new ApproximateHistogramFoldingSerde());
|
||||
|
@ -63,10 +73,5 @@ public class ApproximateHistogramDruidModule implements DruidModule
|
|||
if (ComplexMetrics.getSerdeForType(FixedBucketsHistogramAggregator.TYPE_NAME) == null) {
|
||||
ComplexMetrics.registerSerde(FixedBucketsHistogramAggregator.TYPE_NAME, new FixedBucketsHistogramSerde());
|
||||
}
|
||||
|
||||
if (binder != null) {
|
||||
// Binder is null in some tests.
|
||||
SqlBindings.addAggregator(binder, QuantileSqlAggregator.class);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,322 @@
|
|||
/*
|
||||
* 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.query.aggregation.histogram.sql;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.calcite.rel.core.AggregateCall;
|
||||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.sql.SqlAggFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.ReturnTypes;
|
||||
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.histogram.FixedBucketsHistogram;
|
||||
import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogramAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.histogram.QuantilePostAggregator;
|
||||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.sql.calcite.aggregation.Aggregation;
|
||||
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.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.table.RowSignature;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class FixedBucketsHistogramQuantileSqlAggregator implements SqlAggregator
|
||||
{
|
||||
private static final SqlAggFunction FUNCTION_INSTANCE = new FixedBucketsHistogramQuantileSqlAggFunction();
|
||||
private static final String NAME = "APPROX_QUANTILE_FIXED_BUCKETS";
|
||||
|
||||
@Override
|
||||
public SqlAggFunction calciteFunction()
|
||||
{
|
||||
return FUNCTION_INSTANCE;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Aggregation toDruidAggregation(
|
||||
PlannerContext plannerContext,
|
||||
RowSignature rowSignature,
|
||||
RexBuilder rexBuilder,
|
||||
String name,
|
||||
AggregateCall aggregateCall,
|
||||
Project project,
|
||||
List<Aggregation> existingAggregations,
|
||||
boolean finalizeAggregations
|
||||
)
|
||||
{
|
||||
final DruidExpression input = Expressions.toDruidExpression(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(0)
|
||||
)
|
||||
);
|
||||
if (input == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final AggregatorFactory aggregatorFactory;
|
||||
final String histogramName = StringUtils.format("%s:agg", name);
|
||||
final RexNode probabilityArg = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(1)
|
||||
);
|
||||
|
||||
if (!probabilityArg.isA(SqlKind.LITERAL)) {
|
||||
// Probability must be a literal in order to plan.
|
||||
return null;
|
||||
}
|
||||
|
||||
final float probability = ((Number) RexLiteral.value(probabilityArg)).floatValue();
|
||||
|
||||
final int numBuckets;
|
||||
if (aggregateCall.getArgList().size() >= 3) {
|
||||
final RexNode numBucketsArg = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(2)
|
||||
);
|
||||
|
||||
if (!numBucketsArg.isA(SqlKind.LITERAL)) {
|
||||
// Resolution must be a literal in order to plan.
|
||||
return null;
|
||||
}
|
||||
|
||||
numBuckets = ((Number) RexLiteral.value(numBucketsArg)).intValue();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
||||
final double lowerLimit;
|
||||
if (aggregateCall.getArgList().size() >= 4) {
|
||||
final RexNode lowerLimitArg = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(3)
|
||||
);
|
||||
|
||||
if (!lowerLimitArg.isA(SqlKind.LITERAL)) {
|
||||
// Resolution must be a literal in order to plan.
|
||||
return null;
|
||||
}
|
||||
|
||||
lowerLimit = ((Number) RexLiteral.value(lowerLimitArg)).doubleValue();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
||||
final double upperLimit;
|
||||
if (aggregateCall.getArgList().size() >= 5) {
|
||||
final RexNode upperLimitArg = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(4)
|
||||
);
|
||||
|
||||
if (!upperLimitArg.isA(SqlKind.LITERAL)) {
|
||||
// Resolution must be a literal in order to plan.
|
||||
return null;
|
||||
}
|
||||
|
||||
upperLimit = ((Number) RexLiteral.value(upperLimitArg)).doubleValue();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
||||
final FixedBucketsHistogram.OutlierHandlingMode outlierHandlingMode;
|
||||
if (aggregateCall.getArgList().size() >= 6) {
|
||||
final RexNode outlierHandlingModeArg = Expressions.fromFieldAccess(
|
||||
rowSignature,
|
||||
project,
|
||||
aggregateCall.getArgList().get(5)
|
||||
);
|
||||
|
||||
if (!outlierHandlingModeArg.isA(SqlKind.LITERAL)) {
|
||||
// Resolution must be a literal in order to plan.
|
||||
return null;
|
||||
}
|
||||
|
||||
outlierHandlingMode = FixedBucketsHistogram.OutlierHandlingMode.fromString(
|
||||
RexLiteral.stringValue(outlierHandlingModeArg)
|
||||
);
|
||||
} else {
|
||||
outlierHandlingMode = FixedBucketsHistogram.OutlierHandlingMode.IGNORE;
|
||||
}
|
||||
|
||||
// Look for existing matching aggregatorFactory.
|
||||
for (final Aggregation existing : existingAggregations) {
|
||||
for (AggregatorFactory factory : existing.getAggregatorFactories()) {
|
||||
if (factory instanceof FixedBucketsHistogramAggregatorFactory) {
|
||||
final FixedBucketsHistogramAggregatorFactory theFactory = (FixedBucketsHistogramAggregatorFactory) factory;
|
||||
|
||||
// Check input for equivalence.
|
||||
final boolean inputMatches;
|
||||
final VirtualColumn virtualInput = existing.getVirtualColumns()
|
||||
.stream()
|
||||
.filter(
|
||||
virtualColumn ->
|
||||
virtualColumn.getOutputName()
|
||||
.equals(theFactory.getFieldName())
|
||||
)
|
||||
.findFirst()
|
||||
.orElse(null);
|
||||
|
||||
if (virtualInput == null) {
|
||||
inputMatches = input.isDirectColumnAccess()
|
||||
&& input.getDirectColumn().equals(theFactory.getFieldName());
|
||||
} else {
|
||||
inputMatches = ((ExpressionVirtualColumn) virtualInput).getExpression()
|
||||
.equals(input.getExpression());
|
||||
}
|
||||
|
||||
final boolean matches = inputMatches
|
||||
&& theFactory.getOutlierHandlingMode() == outlierHandlingMode
|
||||
&& theFactory.getNumBuckets() == numBuckets
|
||||
&& theFactory.getLowerLimit() == lowerLimit
|
||||
&& theFactory.getUpperLimit() == upperLimit;
|
||||
|
||||
if (matches) {
|
||||
// Found existing one. Use this.
|
||||
return Aggregation.create(
|
||||
ImmutableList.of(),
|
||||
new QuantilePostAggregator(name, factory.getName(), probability)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// No existing match found. Create a new one.
|
||||
final List<VirtualColumn> virtualColumns = new ArrayList<>();
|
||||
|
||||
if (input.isDirectColumnAccess()) {
|
||||
aggregatorFactory = new FixedBucketsHistogramAggregatorFactory(
|
||||
histogramName,
|
||||
input.getDirectColumn(),
|
||||
numBuckets,
|
||||
lowerLimit,
|
||||
upperLimit,
|
||||
outlierHandlingMode
|
||||
);
|
||||
} else {
|
||||
final ExpressionVirtualColumn virtualColumn = input.toVirtualColumn(
|
||||
StringUtils.format("%s:v", name),
|
||||
ValueType.FLOAT,
|
||||
plannerContext.getExprMacroTable()
|
||||
);
|
||||
virtualColumns.add(virtualColumn);
|
||||
aggregatorFactory = new FixedBucketsHistogramAggregatorFactory(
|
||||
histogramName,
|
||||
virtualColumn.getOutputName(),
|
||||
numBuckets,
|
||||
lowerLimit,
|
||||
upperLimit,
|
||||
outlierHandlingMode
|
||||
);
|
||||
}
|
||||
|
||||
return Aggregation.create(
|
||||
virtualColumns,
|
||||
ImmutableList.of(aggregatorFactory),
|
||||
new QuantilePostAggregator(name, histogramName, probability)
|
||||
);
|
||||
}
|
||||
|
||||
private static class FixedBucketsHistogramQuantileSqlAggFunction extends SqlAggFunction
|
||||
{
|
||||
private static final String SIGNATURE1 =
|
||||
"'"
|
||||
+ NAME
|
||||
+ "(column, probability, numBuckets, lowerLimit, upperLimit)'\n";
|
||||
private static final String SIGNATURE2 =
|
||||
"'"
|
||||
+ NAME
|
||||
+ "(column, probability, numBuckets, lowerLimit, upperLimit, outlierHandlingMode)'\n";
|
||||
|
||||
FixedBucketsHistogramQuantileSqlAggFunction()
|
||||
{
|
||||
super(
|
||||
NAME,
|
||||
null,
|
||||
SqlKind.OTHER_FUNCTION,
|
||||
ReturnTypes.explicit(SqlTypeName.DOUBLE),
|
||||
null,
|
||||
OperandTypes.or(
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(
|
||||
SIGNATURE1,
|
||||
OperandTypes.ANY,
|
||||
OperandTypes.LITERAL,
|
||||
OperandTypes.LITERAL,
|
||||
OperandTypes.LITERAL,
|
||||
OperandTypes.LITERAL
|
||||
),
|
||||
OperandTypes.family(
|
||||
SqlTypeFamily.ANY,
|
||||
SqlTypeFamily.NUMERIC,
|
||||
SqlTypeFamily.NUMERIC,
|
||||
SqlTypeFamily.NUMERIC,
|
||||
SqlTypeFamily.NUMERIC
|
||||
)
|
||||
),
|
||||
OperandTypes.and(
|
||||
OperandTypes.sequence(
|
||||
SIGNATURE2,
|
||||
OperandTypes.ANY,
|
||||
OperandTypes.LITERAL,
|
||||
OperandTypes.LITERAL,
|
||||
OperandTypes.LITERAL,
|
||||
OperandTypes.LITERAL,
|
||||
OperandTypes.LITERAL
|
||||
),
|
||||
OperandTypes.family(
|
||||
SqlTypeFamily.ANY,
|
||||
SqlTypeFamily.NUMERIC,
|
||||
SqlTypeFamily.NUMERIC,
|
||||
SqlTypeFamily.NUMERIC,
|
||||
SqlTypeFamily.NUMERIC,
|
||||
SqlTypeFamily.STRING
|
||||
)
|
||||
)
|
||||
),
|
||||
SqlFunctionCategory.NUMERIC,
|
||||
false,
|
||||
false
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -52,10 +52,9 @@ public class ApproximateHistogramAggregationTest
|
|||
|
||||
public ApproximateHistogramAggregationTest(final GroupByQueryConfig config)
|
||||
{
|
||||
ApproximateHistogramDruidModule module = new ApproximateHistogramDruidModule();
|
||||
module.configure(null);
|
||||
ApproximateHistogramDruidModule.registerSerde();
|
||||
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||
Lists.newArrayList(module.getJacksonModules()),
|
||||
Lists.newArrayList(new ApproximateHistogramDruidModule().getJacksonModules()),
|
||||
config,
|
||||
tempFolder
|
||||
);
|
||||
|
|
|
@ -145,9 +145,7 @@ public class ApproximateHistogramGroupByQueryTest
|
|||
{
|
||||
this.factory = factory;
|
||||
this.runner = runner;
|
||||
|
||||
//Note: this is needed in order to properly register the serde for Histogram.
|
||||
new ApproximateHistogramDruidModule().configure(null);
|
||||
ApproximateHistogramDruidModule.registerSerde();
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
|
@ -81,7 +81,7 @@ public class ApproximateHistogramTopNQueryTest
|
|||
defaultPool,
|
||||
new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
|
@ -91,7 +91,7 @@ public class ApproximateHistogramTopNQueryTest
|
|||
customPool,
|
||||
new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
|
|
|
@ -52,10 +52,9 @@ public class FixedBucketsHistogramAggregationTest
|
|||
|
||||
public FixedBucketsHistogramAggregationTest(final GroupByQueryConfig config)
|
||||
{
|
||||
ApproximateHistogramDruidModule module = new ApproximateHistogramDruidModule();
|
||||
module.configure(null);
|
||||
ApproximateHistogramDruidModule.registerSerde();
|
||||
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||
Lists.newArrayList(module.getJacksonModules()),
|
||||
Lists.newArrayList(new ApproximateHistogramDruidModule().getJacksonModules()),
|
||||
config,
|
||||
tempFolder
|
||||
);
|
||||
|
|
|
@ -146,8 +146,7 @@ public class FixedBucketsHistogramGroupByQueryTest
|
|||
this.factory = factory;
|
||||
this.runner = runner;
|
||||
|
||||
//Note: this is needed in order to properly register the serde for Histogram.
|
||||
new ApproximateHistogramDruidModule().configure(null);
|
||||
ApproximateHistogramDruidModule.registerSerde();
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
|
@ -81,7 +81,7 @@ public class FixedBucketsHistogramTopNQueryTest
|
|||
defaultPool,
|
||||
new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
|
@ -91,7 +91,7 @@ public class FixedBucketsHistogramTopNQueryTest
|
|||
customPool,
|
||||
new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
|
|
|
@ -0,0 +1,452 @@
|
|||
/*
|
||||
* 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.query.aggregation.histogram.sql;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.histogram.ApproximateHistogramDruidModule;
|
||||
import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogram;
|
||||
import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogramAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.histogram.QuantilePostAggregator;
|
||||
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
|
||||
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.IndexBuilder;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.server.security.AuthTestUtils;
|
||||
import org.apache.druid.server.security.AuthenticationResult;
|
||||
import org.apache.druid.sql.SqlLifecycle;
|
||||
import org.apache.druid.sql.SqlLifecycleFactory;
|
||||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.schema.SystemSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.QueryLogHook;
|
||||
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class FixedBucketsHistogramQuantileSqlAggregatorTest extends CalciteTestBase
|
||||
{
|
||||
private static final String DATA_SOURCE = "foo";
|
||||
|
||||
private static QueryRunnerFactoryConglomerate conglomerate;
|
||||
private static Closer resourceCloser;
|
||||
private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
|
||||
private static final Map<String, Object> QUERY_CONTEXT_DEFAULT = ImmutableMap.of(
|
||||
PlannerContext.CTX_SQL_QUERY_ID, "dummy"
|
||||
);
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpClass()
|
||||
{
|
||||
final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
|
||||
.createQueryRunnerFactoryConglomerate();
|
||||
conglomerate = conglomerateCloserPair.lhs;
|
||||
resourceCloser = conglomerateCloserPair.rhs;
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownClass() throws IOException
|
||||
{
|
||||
resourceCloser.close();
|
||||
}
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
@Rule
|
||||
public QueryLogHook queryLogHook = QueryLogHook.create();
|
||||
|
||||
private SpecificSegmentsQuerySegmentWalker walker;
|
||||
private SqlLifecycleFactory sqlLifecycleFactory;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
ApproximateHistogramDruidModule.registerSerde();
|
||||
for (Module mod : new ApproximateHistogramDruidModule().getJacksonModules()) {
|
||||
CalciteTests.getJsonMapper().registerModule(mod);
|
||||
}
|
||||
|
||||
final QueryableIndex index = IndexBuilder.create()
|
||||
.tmpDir(temporaryFolder.newFolder())
|
||||
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new CountAggregatorFactory("cnt"),
|
||||
new DoubleSumAggregatorFactory("m1", "m1"),
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"fbhist_m1",
|
||||
"m1",
|
||||
20,
|
||||
0,
|
||||
10,
|
||||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE
|
||||
)
|
||||
)
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.rows(CalciteTests.ROWS1)
|
||||
.buildMMappedIndex();
|
||||
|
||||
walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(index.getDataInterval())
|
||||
.version("1")
|
||||
.shardSpec(new LinearShardSpec(0))
|
||||
.build(),
|
||||
index
|
||||
);
|
||||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(new QuantileSqlAggregator(), new FixedBucketsHistogramQuantileSqlAggregator()),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
|
||||
new PlannerFactory(
|
||||
druidSchema,
|
||||
systemSchema,
|
||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||
operatorTable,
|
||||
CalciteTests.createExprMacroTable(),
|
||||
plannerConfig,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
walker.close();
|
||||
walker = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQuantileOnFloatAndLongs() throws Exception
|
||||
{
|
||||
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
|
||||
final String sql = "SELECT\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.01, 20, 0.0, 10.0),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.5, 20, 0.0, 10.0),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.98, 20, 0.0, 10.0),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.99, 20, 0.0, 10.0),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(m1 * 2, 0.97, 40, 0.0, 20.0),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.99, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 <> 'abc'),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(cnt, 0.5, 20, 0.0, 10.0)\n"
|
||||
+ "FROM foo";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
1.0299999713897705,
|
||||
3.5,
|
||||
6.440000057220459,
|
||||
6.470000267028809,
|
||||
12.40999984741211,
|
||||
6.494999885559082,
|
||||
5.497499942779541,
|
||||
6.499499797821045,
|
||||
1.25
|
||||
}
|
||||
);
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
|
||||
Query expected = Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.virtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"a4:v",
|
||||
"(\"m1\" * 2)",
|
||||
ValueType.FLOAT,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.aggregators(ImmutableList.of(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"a0:agg", "m1", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE
|
||||
),
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"a4:agg", "a4:v", 40, 0.0d, 20.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"a5:agg", "m1", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE
|
||||
),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"a6:agg", "m1", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE
|
||||
),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
),
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"a8:agg", "cnt", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE
|
||||
)
|
||||
))
|
||||
.postAggregators(
|
||||
new QuantilePostAggregator("a0", "a0:agg", 0.01f),
|
||||
new QuantilePostAggregator("a1", "a0:agg", 0.50f),
|
||||
new QuantilePostAggregator("a2", "a0:agg", 0.98f),
|
||||
new QuantilePostAggregator("a3", "a0:agg", 0.99f),
|
||||
new QuantilePostAggregator("a4", "a4:agg", 0.97f),
|
||||
new QuantilePostAggregator("a5", "a5:agg", 0.99f),
|
||||
new QuantilePostAggregator("a6", "a6:agg", 0.999f),
|
||||
new QuantilePostAggregator("a7", "a5:agg", 0.999f),
|
||||
new QuantilePostAggregator("a8", "a8:agg", 0.50f)
|
||||
)
|
||||
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build();
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
expected,
|
||||
actual
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQuantileOnComplexColumn() throws Exception
|
||||
{
|
||||
SqlLifecycle lifecycle = sqlLifecycleFactory.factorize();
|
||||
final String sql = "SELECT\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.01, 20, 0.0, 10.0),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.5, 20, 0.0, 10.0),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.98, 30, 0.0, 10.0),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.99, 20, 0.0, 10.0),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.99, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 <> 'abc'),\n"
|
||||
+ "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc')\n"
|
||||
+ "FROM foo";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = lifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
1.0299999713897705,
|
||||
3.5,
|
||||
6.293333530426025,
|
||||
6.470000267028809,
|
||||
6.494999885559082,
|
||||
5.497499942779541,
|
||||
6.499499797821045
|
||||
}
|
||||
);
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
|
||||
Query expected = Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(ImmutableList.of(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"a0:agg",
|
||||
"fbhist_m1",
|
||||
20,
|
||||
0.0,
|
||||
10.0,
|
||||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE
|
||||
),
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"a2:agg",
|
||||
"fbhist_m1",
|
||||
30,
|
||||
0.0,
|
||||
10.0,
|
||||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"a4:agg",
|
||||
"fbhist_m1",
|
||||
20,
|
||||
0.0,
|
||||
10.0,
|
||||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE
|
||||
),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"a5:agg",
|
||||
"fbhist_m1",
|
||||
20,
|
||||
0.0,
|
||||
10.0,
|
||||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE
|
||||
),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
)
|
||||
))
|
||||
.postAggregators(
|
||||
new QuantilePostAggregator("a0", "a0:agg", 0.01f),
|
||||
new QuantilePostAggregator("a1", "a0:agg", 0.50f),
|
||||
new QuantilePostAggregator("a2", "a2:agg", 0.98f),
|
||||
new QuantilePostAggregator("a3", "a0:agg", 0.99f),
|
||||
new QuantilePostAggregator("a4", "a4:agg", 0.99f),
|
||||
new QuantilePostAggregator("a5", "a5:agg", 0.999f),
|
||||
new QuantilePostAggregator("a6", "a4:agg", 0.999f)
|
||||
)
|
||||
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build();
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(expected, actual);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQuantileOnInnerQuery() throws Exception
|
||||
{
|
||||
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
|
||||
final String sql = "SELECT AVG(x), APPROX_QUANTILE_FIXED_BUCKETS(x, 0.98, 100, 0.0, 100.0)\n"
|
||||
+ "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
expectedResults = ImmutableList.of(new Object[]{7.0, 11.940000534057617});
|
||||
} else {
|
||||
expectedResults = ImmutableList.of(new Object[]{5.25, 8.920000076293945});
|
||||
}
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries());
|
||||
Query expected = GroupByQuery.builder()
|
||||
.setDataSource(
|
||||
new QueryDataSource(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(
|
||||
Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(new DefaultDimensionSpec("dim2", "d0"))
|
||||
.setAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new DoubleSumAggregatorFactory("a0", "m1")
|
||||
)
|
||||
)
|
||||
.setContext(ImmutableMap.of(
|
||||
PlannerContext.CTX_SQL_QUERY_ID,
|
||||
"dummy"
|
||||
))
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setAggregatorSpecs(
|
||||
new DoubleSumAggregatorFactory("_a0:sum", "a0"),
|
||||
new CountAggregatorFactory("_a0:count"),
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
"_a1:agg",
|
||||
"a0",
|
||||
100,
|
||||
0,
|
||||
100.0d,
|
||||
FixedBucketsHistogram.OutlierHandlingMode.IGNORE
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new ArithmeticPostAggregator(
|
||||
"_a0",
|
||||
"quotient",
|
||||
ImmutableList.of(
|
||||
new FieldAccessPostAggregator(null, "_a0:sum"),
|
||||
new FieldAccessPostAggregator(null, "_a0:count")
|
||||
)
|
||||
),
|
||||
new QuantilePostAggregator("_a1", "_a1:agg", 0.98f)
|
||||
)
|
||||
)
|
||||
.setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build();
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(expected, actual);
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.query.aggregation.histogram.sql;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
@ -119,8 +120,10 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase
|
|||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
// Note: this is needed in order to properly register the serde for Histogram.
|
||||
new ApproximateHistogramDruidModule().configure(null);
|
||||
ApproximateHistogramDruidModule.registerSerde();
|
||||
for (Module mod : new ApproximateHistogramDruidModule().getJacksonModules()) {
|
||||
CalciteTests.getJsonMapper().registerModule(mod);
|
||||
}
|
||||
|
||||
final QueryableIndex index = IndexBuilder.create()
|
||||
.tmpDir(temporaryFolder.newFolder())
|
||||
|
@ -157,7 +160,7 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase
|
|||
|
||||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
|
||||
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
|
||||
final DruidOperatorTable operatorTable = new DruidOperatorTable(
|
||||
ImmutableSet.of(new QuantileSqlAggregator()),
|
||||
ImmutableSet.of()
|
||||
|
@ -188,221 +191,210 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase
|
|||
public void testQuantileOnFloatAndLongs() throws Exception
|
||||
{
|
||||
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
|
||||
try {
|
||||
final String sql = "SELECT\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.01),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.5, 50),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.98, 200),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.99),\n"
|
||||
+ "APPROX_QUANTILE(m1 * 2, 0.97),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE(cnt, 0.5)\n"
|
||||
+ "FROM foo";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
1.0,
|
||||
3.0,
|
||||
5.880000114440918,
|
||||
5.940000057220459,
|
||||
11.640000343322754,
|
||||
6.0,
|
||||
4.994999885559082,
|
||||
6.0,
|
||||
1.0
|
||||
}
|
||||
);
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
final String sql = "SELECT\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.01),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.5, 50),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.98, 200),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.99),\n"
|
||||
+ "APPROX_QUANTILE(m1 * 2, 0.97),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
|
||||
+ "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE(cnt, 0.5)\n"
|
||||
+ "FROM foo";
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.virtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"a4:v",
|
||||
"(\"m1\" * 2)",
|
||||
ValueType.FLOAT,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.aggregators(ImmutableList.of(
|
||||
new ApproximateHistogramAggregatorFactory("a0:agg", "m1", null, null, null, null),
|
||||
new ApproximateHistogramAggregatorFactory("a2:agg", "m1", 200, null, null, null),
|
||||
new ApproximateHistogramAggregatorFactory("a4:agg", "a4:v", null, null, null, null),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramAggregatorFactory("a5:agg", "m1", null, null, null, null),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramAggregatorFactory("a6:agg", "m1", null, null, null, null),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
),
|
||||
new ApproximateHistogramAggregatorFactory("a8:agg", "cnt", null, null, null, null)
|
||||
))
|
||||
.postAggregators(
|
||||
new QuantilePostAggregator("a0", "a0:agg", 0.01f),
|
||||
new QuantilePostAggregator("a1", "a0:agg", 0.50f),
|
||||
new QuantilePostAggregator("a2", "a2:agg", 0.98f),
|
||||
new QuantilePostAggregator("a3", "a0:agg", 0.99f),
|
||||
new QuantilePostAggregator("a4", "a4:agg", 0.97f),
|
||||
new QuantilePostAggregator("a5", "a5:agg", 0.99f),
|
||||
new QuantilePostAggregator("a6", "a6:agg", 0.999f),
|
||||
new QuantilePostAggregator("a7", "a5:agg", 0.999f),
|
||||
new QuantilePostAggregator("a8", "a8:agg", 0.50f)
|
||||
)
|
||||
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw e;
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{
|
||||
1.0,
|
||||
3.0,
|
||||
5.880000114440918,
|
||||
5.940000057220459,
|
||||
11.640000343322754,
|
||||
6.0,
|
||||
4.994999885559082,
|
||||
6.0,
|
||||
1.0
|
||||
}
|
||||
);
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.virtualColumns(
|
||||
new ExpressionVirtualColumn(
|
||||
"a4:v",
|
||||
"(\"m1\" * 2)",
|
||||
ValueType.FLOAT,
|
||||
TestExprMacroTable.INSTANCE
|
||||
)
|
||||
)
|
||||
.aggregators(ImmutableList.of(
|
||||
new ApproximateHistogramAggregatorFactory("a0:agg", "m1", null, null, null, null),
|
||||
new ApproximateHistogramAggregatorFactory("a2:agg", "m1", 200, null, null, null),
|
||||
new ApproximateHistogramAggregatorFactory("a4:agg", "a4:v", null, null, null, null),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramAggregatorFactory("a5:agg", "m1", null, null, null, null),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramAggregatorFactory("a6:agg", "m1", null, null, null, null),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
),
|
||||
new ApproximateHistogramAggregatorFactory("a8:agg", "cnt", null, null, null, null)
|
||||
))
|
||||
.postAggregators(
|
||||
new QuantilePostAggregator("a0", "a0:agg", 0.01f),
|
||||
new QuantilePostAggregator("a1", "a0:agg", 0.50f),
|
||||
new QuantilePostAggregator("a2", "a2:agg", 0.98f),
|
||||
new QuantilePostAggregator("a3", "a0:agg", 0.99f),
|
||||
new QuantilePostAggregator("a4", "a4:agg", 0.97f),
|
||||
new QuantilePostAggregator("a5", "a5:agg", 0.99f),
|
||||
new QuantilePostAggregator("a6", "a6:agg", 0.999f),
|
||||
new QuantilePostAggregator("a7", "a5:agg", 0.999f),
|
||||
new QuantilePostAggregator("a8", "a8:agg", 0.50f)
|
||||
)
|
||||
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQuantileOnComplexColumn() throws Exception
|
||||
{
|
||||
SqlLifecycle lifecycle = sqlLifecycleFactory.factorize();
|
||||
try {
|
||||
final String sql = "SELECT\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.01),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.5, 50),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.98, 200),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.99),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 = 'abc')\n"
|
||||
+ "FROM foo";
|
||||
final String sql = "SELECT\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.01),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.5, 50),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.98, 200),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.99),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n"
|
||||
+ "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 = 'abc')\n"
|
||||
+ "FROM foo";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = lifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{1.0, 3.0, 5.880000114440918, 5.940000057220459, 6.0, 4.994999885559082, 6.0}
|
||||
);
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
// Verify results
|
||||
final List<Object[]> results = lifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{1.0, 3.0, 5.880000114440918, 5.940000057220459, 6.0, 4.994999885559082, 6.0}
|
||||
);
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(ImmutableList.of(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null),
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a2:agg", "hist_m1", 200, null, null, null),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a4:agg", "hist_m1", null, null, null, null),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a5:agg", "hist_m1", null, null, null, null),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
)
|
||||
))
|
||||
.postAggregators(
|
||||
new QuantilePostAggregator("a0", "a0:agg", 0.01f),
|
||||
new QuantilePostAggregator("a1", "a0:agg", 0.50f),
|
||||
new QuantilePostAggregator("a2", "a2:agg", 0.98f),
|
||||
new QuantilePostAggregator("a3", "a0:agg", 0.99f),
|
||||
new QuantilePostAggregator("a4", "a4:agg", 0.99f),
|
||||
new QuantilePostAggregator("a5", "a5:agg", 0.999f),
|
||||
new QuantilePostAggregator("a6", "a4:agg", 0.999f)
|
||||
)
|
||||
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw e;
|
||||
}
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(ImmutableList.of(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null),
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a2:agg", "hist_m1", 200, null, null, null),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a4:agg", "hist_m1", null, null, null, null),
|
||||
new SelectorDimFilter("dim1", "abc", null)
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a5:agg", "hist_m1", null, null, null, null),
|
||||
new NotDimFilter(new SelectorDimFilter("dim1", "abc", null))
|
||||
)
|
||||
))
|
||||
.postAggregators(
|
||||
new QuantilePostAggregator("a0", "a0:agg", 0.01f),
|
||||
new QuantilePostAggregator("a1", "a0:agg", 0.50f),
|
||||
new QuantilePostAggregator("a2", "a2:agg", 0.98f),
|
||||
new QuantilePostAggregator("a3", "a0:agg", 0.99f),
|
||||
new QuantilePostAggregator("a4", "a4:agg", 0.99f),
|
||||
new QuantilePostAggregator("a5", "a5:agg", 0.999f),
|
||||
new QuantilePostAggregator("a6", "a4:agg", 0.999f)
|
||||
)
|
||||
.context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQuantileOnInnerQuery() throws Exception
|
||||
{
|
||||
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
|
||||
try {
|
||||
final String sql = "SELECT AVG(x), APPROX_QUANTILE(x, 0.98)\n"
|
||||
+ "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)";
|
||||
final String sql = "SELECT AVG(x), APPROX_QUANTILE(x, 0.98)\n"
|
||||
+ "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)";
|
||||
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
expectedResults = ImmutableList.of(new Object[]{7.0, 8.26386833190918});
|
||||
} else {
|
||||
expectedResults = ImmutableList.of(new Object[]{5.25, 6.59091854095459});
|
||||
}
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
// Verify results
|
||||
final List<Object[]> results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList();
|
||||
final List<Object[]> expectedResults;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
expectedResults = ImmutableList.of(new Object[]{7.0, 8.26386833190918});
|
||||
} else {
|
||||
expectedResults = ImmutableList.of(new Object[]{5.25, 6.59091854095459});
|
||||
}
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
}
|
||||
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(
|
||||
new QueryDataSource(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(new DefaultDimensionSpec("dim2", "d0"))
|
||||
.setAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new DoubleSumAggregatorFactory("a0", "m1")
|
||||
)
|
||||
)
|
||||
.setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setAggregatorSpecs(new DoubleSumAggregatorFactory("_a0:sum", "a0"),
|
||||
new CountAggregatorFactory("_a0:count"),
|
||||
new ApproximateHistogramAggregatorFactory("_a1:agg",
|
||||
"a0",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
))
|
||||
.setPostAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new ArithmeticPostAggregator(
|
||||
"_a0",
|
||||
"quotient",
|
||||
ImmutableList.of(
|
||||
new FieldAccessPostAggregator(null, "_a0:sum"),
|
||||
new FieldAccessPostAggregator(null, "_a0:count")
|
||||
)
|
||||
),
|
||||
new QuantilePostAggregator("_a1", "_a1:agg", 0.98f)
|
||||
)
|
||||
)
|
||||
.setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw e;
|
||||
}
|
||||
// Verify query
|
||||
Assert.assertEquals(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(
|
||||
new QueryDataSource(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(new DefaultDimensionSpec("dim2", "d0"))
|
||||
.setAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new DoubleSumAggregatorFactory("a0", "m1")
|
||||
)
|
||||
)
|
||||
.setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setAggregatorSpecs(
|
||||
new DoubleSumAggregatorFactory("_a0:sum", "a0"),
|
||||
new CountAggregatorFactory("_a0:count"),
|
||||
new ApproximateHistogramAggregatorFactory(
|
||||
"_a1:agg",
|
||||
"a0",
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new ArithmeticPostAggregator(
|
||||
"_a0",
|
||||
"quotient",
|
||||
ImmutableList.of(
|
||||
new FieldAccessPostAggregator(null, "_a0:sum"),
|
||||
new FieldAccessPostAggregator(null, "_a0:count")
|
||||
)
|
||||
),
|
||||
new QuantilePostAggregator("_a1", "_a1:agg", 0.98f)
|
||||
)
|
||||
)
|
||||
.setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy"))
|
||||
.build(),
|
||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -54,6 +54,7 @@ import java.util.Map;
|
|||
import java.util.Properties;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.CancellationException;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -184,7 +185,7 @@ public class KafkaLookupExtractorFactory implements LookupExtractorFactory
|
|||
final String topic = getKafkaTopic();
|
||||
LOG.debug("About to listen to topic [%s] with group.id [%s]", topic, factoryId);
|
||||
cacheHandler = cacheManager.createCache();
|
||||
final Map<String, String> map = cacheHandler.getCache();
|
||||
final ConcurrentMap<String, String> map = cacheHandler.getCache();
|
||||
mapRef.set(map);
|
||||
// Enable publish-subscribe
|
||||
kafkaProperties.setProperty("auto.offset.reset", "smallest");
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -104,6 +104,7 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -123,8 +124,8 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
|||
private static final EmittingLogger log = new EmittingLogger(LegacyKafkaIndexTaskRunner.class);
|
||||
private static final String METADATA_NEXT_PARTITIONS = "nextPartitions";
|
||||
|
||||
private final Map<Integer, Long> endOffsets = new ConcurrentHashMap<>();
|
||||
private final Map<Integer, Long> nextOffsets = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<Integer, Long> endOffsets = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<Integer, Long> nextOffsets = new ConcurrentHashMap<>();
|
||||
|
||||
// The pause lock and associated conditions are to support coordination between the Jetty threads and the main
|
||||
// ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully
|
||||
|
|
|
@ -50,7 +50,6 @@ import org.apache.druid.discovery.DruidNodeAnnouncer;
|
|||
import org.apache.druid.discovery.LookupNodeService;
|
||||
import org.apache.druid.indexer.TaskState;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.Counters;
|
||||
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
|
||||
import org.apache.druid.indexing.common.SegmentLoaderFactory;
|
||||
import org.apache.druid.indexing.common.TaskLock;
|
||||
|
@ -2393,8 +2392,7 @@ public class KafkaIndexTaskTest
|
|||
);
|
||||
return true;
|
||||
}
|
||||
},
|
||||
new Counters()
|
||||
}
|
||||
);
|
||||
final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
|
||||
taskStorage,
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -68,6 +68,7 @@ import java.util.Set;
|
|||
import java.util.UUID;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
|
@ -341,7 +342,8 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String>
|
|||
|
||||
private ScheduledExecutorService scheduledExec;
|
||||
|
||||
private final Map<StreamPartition<String>, PartitionResource> partitionResources = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<StreamPartition<String>, PartitionResource> partitionResources =
|
||||
new ConcurrentHashMap<>();
|
||||
private BlockingQueue<OrderedPartitionableRecord<String, String>> records;
|
||||
|
||||
private volatile boolean checkPartitionsStarted = false;
|
||||
|
|
|
@ -55,7 +55,6 @@ import org.apache.druid.discovery.DruidNodeAnnouncer;
|
|||
import org.apache.druid.discovery.LookupNodeService;
|
||||
import org.apache.druid.indexer.TaskState;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.Counters;
|
||||
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
|
||||
import org.apache.druid.indexing.common.SegmentLoaderFactory;
|
||||
import org.apache.druid.indexing.common.TaskLock;
|
||||
|
@ -2705,8 +2704,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
|||
);
|
||||
return true;
|
||||
}
|
||||
},
|
||||
new Counters()
|
||||
}
|
||||
);
|
||||
final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
|
||||
taskStorage,
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.druid.query.lookup.namespace.ExtractionNamespace;
|
|||
import sun.misc.Cleaner;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CancellationException;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
@ -424,7 +425,9 @@ public final class CacheScheduler
|
|||
NamespaceExtractionCacheManager cacheManager
|
||||
)
|
||||
{
|
||||
this.namespaceGeneratorMap = namespaceGeneratorMap;
|
||||
// Accesses to IdentityHashMap should be faster than to HashMap or ImmutableMap.
|
||||
// Class doesn't override Object.equals().
|
||||
this.namespaceGeneratorMap = new IdentityHashMap<>(namespaceGeneratorMap);
|
||||
this.cacheManager = cacheManager;
|
||||
cacheManager.scheduledExecutorService().scheduleAtFixedRate(
|
||||
new Runnable()
|
||||
|
@ -506,6 +509,7 @@ public final class CacheScheduler
|
|||
|
||||
public <T extends ExtractionNamespace> Entry schedule(final T namespace)
|
||||
{
|
||||
@SuppressWarnings("unchecked")
|
||||
final CacheGenerator<T> generator = (CacheGenerator<T>) namespaceGeneratorMap.get(namespace.getClass());
|
||||
if (generator == null) {
|
||||
throw new ISE("Cannot find generator for namespace [%s]", namespace);
|
||||
|
|
|
@ -114,7 +114,7 @@ public class OnHeapNamespaceExtractionCacheManager extends NamespaceExtractionCa
|
|||
long size = 0;
|
||||
expungeCollectedCaches();
|
||||
for (WeakReference<ConcurrentMap<String, String>> cacheRef : caches) {
|
||||
final Map<String, String> cache = cacheRef.get();
|
||||
final ConcurrentMap<String, String> cache = cacheRef.get();
|
||||
if (cache == null) {
|
||||
continue;
|
||||
}
|
||||
|
|
|
@ -29,8 +29,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
|
|||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
|
||||
import org.apache.druid.query.lookup.namespace.CacheGenerator;
|
||||
import org.apache.druid.query.lookup.namespace.ExtractionNamespace;
|
||||
import org.apache.druid.query.lookup.namespace.UriExtractionNamespace;
|
||||
import org.apache.druid.query.lookup.namespace.UriExtractionNamespaceTest;
|
||||
import org.apache.druid.segment.loading.LocalFileTimestampVersionFinder;
|
||||
|
@ -64,7 +62,6 @@ import java.nio.charset.StandardCharsets;
|
|||
import java.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -246,20 +243,14 @@ public class UriCacheGeneratorTest
|
|||
Function<Lifecycle, NamespaceExtractionCacheManager> cacheManagerCreator
|
||||
)
|
||||
{
|
||||
final Map<Class<? extends ExtractionNamespace>, CacheGenerator<?>> namespaceFunctionFactoryMap = new HashMap<>();
|
||||
this.suffix = suffix;
|
||||
this.outStreamSupplier = outStreamSupplier;
|
||||
this.lifecycle = new Lifecycle();
|
||||
this.scheduler = new CacheScheduler(
|
||||
new NoopServiceEmitter(),
|
||||
namespaceFunctionFactoryMap,
|
||||
ImmutableMap.of(UriExtractionNamespace.class, new UriCacheGenerator(FINDERS)),
|
||||
cacheManagerCreator.apply(lifecycle)
|
||||
);
|
||||
namespaceFunctionFactoryMap.put(
|
||||
UriExtractionNamespace.class,
|
||||
|
||||
new UriCacheGenerator(FINDERS)
|
||||
);
|
||||
}
|
||||
|
||||
@Rule
|
||||
|
|
|
@ -29,7 +29,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@
|
|||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.13.0-incubating-SNAPSHOT</version>
|
||||
<version>0.14.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue