mirror of https://github.com/apache/druid.git
Merge remote-tracking branch 'apache/master' into quidem-runner-extension-submit
This commit is contained in:
commit
9578953678
|
@ -1,4 +1,5 @@
|
|||
dist
|
||||
longCompress
|
||||
target
|
||||
*.iml
|
||||
*.ipr
|
||||
|
|
|
@ -37,6 +37,8 @@ import org.apache.druid.frame.processor.FrameProcessorExecutor;
|
|||
import org.apache.druid.frame.read.FrameReader;
|
||||
import org.apache.druid.frame.testutil.FrameSequenceBuilder;
|
||||
import org.apache.druid.frame.write.FrameWriters;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.NonnullPair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -47,6 +49,7 @@ import org.apache.druid.segment.RowBasedSegment;
|
|||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.nested.StructuredData;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
|
@ -82,6 +85,7 @@ public class FrameChannelMergerBenchmark
|
|||
{
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
}
|
||||
|
||||
private static final String KEY = "key";
|
||||
|
@ -99,6 +103,9 @@ public class FrameChannelMergerBenchmark
|
|||
@Param({"100"})
|
||||
private int rowLength;
|
||||
|
||||
@Param({"string", "nested"})
|
||||
private String columnType;
|
||||
|
||||
/**
|
||||
* Linked to {@link KeyGenerator}.
|
||||
*/
|
||||
|
@ -121,13 +128,20 @@ public class FrameChannelMergerBenchmark
|
|||
*/
|
||||
RANDOM {
|
||||
@Override
|
||||
public String generateKey(int rowNumber, int keyLength)
|
||||
public Comparable generateKey(int rowNumber, int keyLength, String columnType)
|
||||
{
|
||||
final StringBuilder builder = new StringBuilder(keyLength);
|
||||
for (int i = 0; i < keyLength; i++) {
|
||||
builder.append((char) ('a' + ThreadLocalRandom.current().nextInt(26)));
|
||||
}
|
||||
return builder.toString();
|
||||
String str = builder.toString();
|
||||
if ("string".equals(columnType)) {
|
||||
return str;
|
||||
} else if ("nested".equals(columnType)) {
|
||||
return StructuredData.wrap(str);
|
||||
} else {
|
||||
throw new IAE("unsupported column type");
|
||||
}
|
||||
}
|
||||
},
|
||||
|
||||
|
@ -136,13 +150,20 @@ public class FrameChannelMergerBenchmark
|
|||
*/
|
||||
SEQUENTIAL {
|
||||
@Override
|
||||
public String generateKey(int rowNumber, int keyLength)
|
||||
public Comparable generateKey(int rowNumber, int keyLength, String columnType)
|
||||
{
|
||||
return StringUtils.format("%0" + keyLength + "d", rowNumber);
|
||||
String str = StringUtils.format("%0" + keyLength + "d", rowNumber);
|
||||
if ("string".equals(columnType)) {
|
||||
return str;
|
||||
} else if ("nested".equals(columnType)) {
|
||||
return StructuredData.wrap(str);
|
||||
} else {
|
||||
throw new IAE("unsupported column type");
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
public abstract String generateKey(int rowNumber, int keyLength);
|
||||
public abstract Comparable generateKey(int rowNumber, int keyLength, String columnType);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -176,13 +197,9 @@ public class FrameChannelMergerBenchmark
|
|||
public abstract int getChannelNumber(int rowNumber, int numRows, int numChannels);
|
||||
}
|
||||
|
||||
private final RowSignature signature =
|
||||
RowSignature.builder()
|
||||
.add(KEY, ColumnType.STRING)
|
||||
.add(VALUE, ColumnType.STRING)
|
||||
.build();
|
||||
private RowSignature signature;
|
||||
private FrameReader frameReader;
|
||||
|
||||
private final FrameReader frameReader = FrameReader.create(signature);
|
||||
private final List<KeyColumn> sortKey = ImmutableList.of(new KeyColumn(KEY, KeyOrder.ASCENDING));
|
||||
|
||||
private List<List<Frame>> channelFrames;
|
||||
|
@ -200,6 +217,14 @@ public class FrameChannelMergerBenchmark
|
|||
@Setup(Level.Trial)
|
||||
public void setupTrial()
|
||||
{
|
||||
signature =
|
||||
RowSignature.builder()
|
||||
.add(KEY, createKeyColumnTypeFromTypeString(columnType))
|
||||
.add(VALUE, ColumnType.STRING)
|
||||
.build();
|
||||
|
||||
frameReader = FrameReader.create(signature);
|
||||
|
||||
exec = new FrameProcessorExecutor(
|
||||
MoreExecutors.listeningDecorator(
|
||||
Execs.singleThreaded(StringUtils.encodeForFormat(getClass().getSimpleName()))
|
||||
|
@ -211,14 +236,15 @@ public class FrameChannelMergerBenchmark
|
|||
ChannelDistribution.valueOf(StringUtils.toUpperCase(channelDistributionString));
|
||||
|
||||
// Create channelRows which holds rows for each channel.
|
||||
final List<List<NonnullPair<String, String>>> channelRows = new ArrayList<>();
|
||||
final List<List<NonnullPair<Comparable, String>>> channelRows = new ArrayList<>();
|
||||
channelFrames = new ArrayList<>();
|
||||
for (int channelNumber = 0; channelNumber < numChannels; channelNumber++) {
|
||||
channelRows.add(new ArrayList<>());
|
||||
channelFrames.add(new ArrayList<>());
|
||||
}
|
||||
|
||||
// Create "valueString", a string full of spaces to pad out the row.
|
||||
// Create "valueString", a string full of spaces to pad out the row. Nested columns wrap up strings with the
|
||||
// corresponding keyLength, therefore the padding works out for the nested types as well.
|
||||
final StringBuilder valueStringBuilder = new StringBuilder();
|
||||
for (int i = 0; i < rowLength - keyLength; i++) {
|
||||
valueStringBuilder.append(' ');
|
||||
|
@ -227,20 +253,20 @@ public class FrameChannelMergerBenchmark
|
|||
|
||||
// Populate "channelRows".
|
||||
for (int rowNumber = 0; rowNumber < numRows; rowNumber++) {
|
||||
final String keyString = keyGenerator.generateKey(rowNumber, keyLength);
|
||||
final NonnullPair<String, String> row = new NonnullPair<>(keyString, valueString);
|
||||
final Comparable keyObject = keyGenerator.generateKey(rowNumber, keyLength, columnType);
|
||||
final NonnullPair<Comparable, String> row = new NonnullPair<>(keyObject, valueString);
|
||||
channelRows.get(channelDistribution.getChannelNumber(rowNumber, numRows, numChannels)).add(row);
|
||||
}
|
||||
|
||||
// Sort each "channelRows".
|
||||
for (List<NonnullPair<String, String>> rows : channelRows) {
|
||||
for (List<NonnullPair<Comparable, String>> rows : channelRows) {
|
||||
rows.sort(Comparator.comparing(row -> row.lhs));
|
||||
}
|
||||
|
||||
// Populate each "channelFrames".
|
||||
for (int channelNumber = 0; channelNumber < numChannels; channelNumber++) {
|
||||
final List<NonnullPair<String, String>> rows = channelRows.get(channelNumber);
|
||||
final RowBasedSegment<NonnullPair<String, String>> segment =
|
||||
final List<NonnullPair<Comparable, String>> rows = channelRows.get(channelNumber);
|
||||
final RowBasedSegment<NonnullPair<Comparable, String>> segment =
|
||||
new RowBasedSegment<>(
|
||||
SegmentId.dummy("__dummy"),
|
||||
Sequences.simple(rows),
|
||||
|
@ -350,4 +376,14 @@ public class FrameChannelMergerBenchmark
|
|||
throw new ISE("Incorrect numRows[%s], expected[%s]", FutureUtils.getUncheckedImmediately(retVal), numRows);
|
||||
}
|
||||
}
|
||||
|
||||
private ColumnType createKeyColumnTypeFromTypeString(final String columnTypeString)
|
||||
{
|
||||
if ("string".equals(columnTypeString)) {
|
||||
return ColumnType.STRING;
|
||||
} else if ("nested".equals(columnTypeString)) {
|
||||
return ColumnType.NESTED_DATA;
|
||||
}
|
||||
throw new IAE("Unsupported type [%s]", columnTypeString);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,6 +54,7 @@ org.apache.calcite.sql.type.OperandTypes#NULLABLE_LITERAL @ Create an instance o
|
|||
org.apache.commons.io.FileUtils#getTempDirectory() @ Use org.junit.rules.TemporaryFolder for tests instead
|
||||
org.apache.commons.io.FileUtils#deleteDirectory(java.io.File) @ Use org.apache.druid.java.util.common.FileUtils#deleteDirectory()
|
||||
org.apache.commons.io.FileUtils#forceMkdir(java.io.File) @ Use org.apache.druid.java.util.common.FileUtils.mkdirp instead
|
||||
org.apache.datasketches.memory.Memory#wrap(byte[], int, int, java.nio.ByteOrder) @ The implementation isn't correct in datasketches-memory-2.2.0. Please refer to https://github.com/apache/datasketches-memory/issues/178. Use wrap(byte[]) and modify the offset by the callers instead
|
||||
java.lang.Class#getCanonicalName() @ Class.getCanonicalName can return null for anonymous types, use Class.getName instead.
|
||||
java.util.concurrent.Executors#newFixedThreadPool(int) @ Executor is non-daemon and can prevent JVM shutdown, use org.apache.druid.java.util.common.concurrent.Execs#multiThreaded(int, java.lang.String) instead.
|
||||
|
||||
|
|
|
@ -138,8 +138,8 @@ then
|
|||
setKey _common druid.zk.service.host "${ZOOKEEPER}"
|
||||
fi
|
||||
|
||||
DRUID_SET_HOST=${DRUID_SET_HOST:-1}
|
||||
if [ "${DRUID_SET_HOST}" = "1" ]
|
||||
DRUID_SET_HOST_IP=${DRUID_SET_HOST_IP:-0}
|
||||
if [ "${DRUID_SET_HOST_IP}" = "1" ]
|
||||
then
|
||||
setKey $SERVICE druid.host $(ip r get 1 | awk '{print $7;exit}')
|
||||
fi
|
||||
|
|
|
@ -97,8 +97,8 @@ then
|
|||
setKey _common druid.zk.service.host "${ZOOKEEPER}"
|
||||
fi
|
||||
|
||||
DRUID_SET_HOST=${DRUID_SET_HOST:-1}
|
||||
if [ "${DRUID_SET_HOST}" = "1" ]
|
||||
DRUID_SET_HOST_IP=${DRUID_SET_HOST_IP:-0}
|
||||
if [ "${DRUID_SET_HOST_IP}" = "1" ]
|
||||
then
|
||||
setKey $SERVICE druid.host $(ip r get 1 | awk '{print $7;exit}')
|
||||
fi
|
||||
|
|
|
@ -99,7 +99,7 @@ This feature uses Confluent's Protobuf provider which is not included in the Dru
|
|||
- https://repo1.maven.org/maven2/org/jetbrains/kotlin/kotlin-stdlib/1.4.0/kotlin-stdlib-1.4.0.jar
|
||||
- https://repo1.maven.org/maven2/com/squareup/wire/wire-schema/3.2.2/wire-schema-3.2.2.jar
|
||||
|
||||
Copy or symlink those files inside the folder `extensions/protobuf-extensions` under the distribution root directory.
|
||||
Copy or symlink those files inside the folder `extensions-core/protobuf-extensions` under the distribution root directory.
|
||||
|
||||
## Create Kafka Supervisor
|
||||
|
||||
|
@ -251,7 +251,7 @@ Important supervisor properties
|
|||
If necessary, from your Kafka installation directory run the following command to create the Kafka topic
|
||||
|
||||
```
|
||||
./bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic metrics_pb
|
||||
./bin/kafka-topics.sh --create --bootstrap-server localhost:9092 --replication-factor 1 --partitions 1 --topic metrics_pb
|
||||
```
|
||||
|
||||
This example script requires `protobuf` and `kafka-python` modules. With the topic in place, messages can be inserted running the following command from your Druid installation directory
|
||||
|
@ -263,7 +263,7 @@ This example script requires `protobuf` and `kafka-python` modules. With the top
|
|||
You can confirm that data has been inserted to your Kafka topic using the following command from your Kafka installation directory
|
||||
|
||||
```
|
||||
./bin/kafka-console-consumer --zookeeper localhost --topic metrics_pb
|
||||
./bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic metrics_pb --from-beginning
|
||||
```
|
||||
|
||||
which should print messages like this
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
---
|
||||
id: tutorial-jdbc
|
||||
title: Use the JDBC driver to query Druid
|
||||
sidebar_label: JDBC connector
|
||||
sidebar_label: JDBC connector tutorial
|
||||
---
|
||||
|
||||
<!--
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
---
|
||||
id: tutorial-sketches-theta
|
||||
title: Approximations with Theta sketches
|
||||
sidebar_label: Theta sketches
|
||||
sidebar_label: Theta sketches tutorial
|
||||
---
|
||||
|
||||
<!--
|
||||
|
|
|
@ -128,7 +128,6 @@
|
|||
<dependency>
|
||||
<groupId>org.apache.curator</groupId>
|
||||
<artifactId>curator-client</artifactId>
|
||||
<version>5.5.0</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
|
|
|
@ -67,6 +67,7 @@ public class ClusterByStatisticsCollectorImpl implements ClusterByStatisticsColl
|
|||
|
||||
private ClusterByStatisticsCollectorImpl(
|
||||
final ClusterBy clusterBy,
|
||||
final RowSignature rowSignature,
|
||||
final RowKeyReader keyReader,
|
||||
final KeyCollectorFactory<?, ?> keyCollectorFactory,
|
||||
final long maxRetainedBytes,
|
||||
|
@ -78,7 +79,7 @@ public class ClusterByStatisticsCollectorImpl implements ClusterByStatisticsColl
|
|||
this.keyReader = keyReader;
|
||||
this.keyCollectorFactory = keyCollectorFactory;
|
||||
this.maxRetainedBytes = maxRetainedBytes;
|
||||
this.buckets = new TreeMap<>(clusterBy.bucketComparator());
|
||||
this.buckets = new TreeMap<>(clusterBy.bucketComparator(rowSignature));
|
||||
this.maxBuckets = maxBuckets;
|
||||
this.checkHasMultipleValues = checkHasMultipleValues;
|
||||
this.hasMultipleValues = checkHasMultipleValues ? new boolean[clusterBy.getColumns().size()] : null;
|
||||
|
@ -98,10 +99,12 @@ public class ClusterByStatisticsCollectorImpl implements ClusterByStatisticsColl
|
|||
)
|
||||
{
|
||||
final RowKeyReader keyReader = clusterBy.keyReader(signature);
|
||||
final KeyCollectorFactory<?, ?> keyCollectorFactory = KeyCollectors.makeStandardFactory(clusterBy, aggregate);
|
||||
final KeyCollectorFactory<?, ?> keyCollectorFactory =
|
||||
KeyCollectors.makeStandardFactory(clusterBy, aggregate, signature);
|
||||
|
||||
return new ClusterByStatisticsCollectorImpl(
|
||||
clusterBy,
|
||||
signature,
|
||||
keyReader,
|
||||
keyCollectorFactory,
|
||||
maxRetainedBytes,
|
||||
|
@ -209,6 +212,12 @@ public class ClusterByStatisticsCollectorImpl implements ClusterByStatisticsColl
|
|||
return count;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
long getTotalRetainedBytes()
|
||||
{
|
||||
return totalRetainedBytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues(final int keyPosition)
|
||||
{
|
||||
|
@ -411,7 +420,7 @@ public class ClusterByStatisticsCollectorImpl implements ClusterByStatisticsColl
|
|||
void downSample()
|
||||
{
|
||||
long newTotalRetainedBytes = totalRetainedBytes;
|
||||
final long targetTotalRetainedBytes = totalRetainedBytes / 2;
|
||||
final long targetTotalRetainedBytes = Math.min(totalRetainedBytes / 2, maxRetainedBytes);
|
||||
|
||||
final List<Pair<Long, BucketHolder>> sortedHolders = new ArrayList<>(buckets.size());
|
||||
final RowKeyReader trimmedRowReader = keyReader.trimmedKeyReader(clusterBy.getBucketByCount());
|
||||
|
|
|
@ -23,6 +23,7 @@ import it.unimi.dsi.fastutil.objects.Object2LongRBTreeMap;
|
|||
import org.apache.druid.collections.SerializablePair;
|
||||
import org.apache.druid.frame.key.ClusterBy;
|
||||
import org.apache.druid.frame.key.RowKey;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.stream.Collectors;
|
||||
|
@ -36,9 +37,9 @@ public class DistinctKeyCollectorFactory implements KeyCollectorFactory<Distinct
|
|||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
static DistinctKeyCollectorFactory create(final ClusterBy clusterBy)
|
||||
static DistinctKeyCollectorFactory create(final ClusterBy clusterBy, RowSignature rowSignature)
|
||||
{
|
||||
return new DistinctKeyCollectorFactory(clusterBy.keyComparator());
|
||||
return new DistinctKeyCollectorFactory(clusterBy.keyComparator(rowSignature));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.msq.statistics;
|
||||
|
||||
import org.apache.druid.frame.key.ClusterBy;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
public class KeyCollectors
|
||||
{
|
||||
|
@ -33,19 +34,20 @@ public class KeyCollectors
|
|||
*/
|
||||
public static KeyCollectorFactory<?, ?> makeStandardFactory(
|
||||
final ClusterBy clusterBy,
|
||||
final boolean aggregate
|
||||
final boolean aggregate,
|
||||
final RowSignature rowSignature
|
||||
)
|
||||
{
|
||||
final KeyCollectorFactory<?, ?> baseFactory;
|
||||
|
||||
if (aggregate) {
|
||||
baseFactory = DistinctKeyCollectorFactory.create(clusterBy);
|
||||
baseFactory = DistinctKeyCollectorFactory.create(clusterBy, rowSignature);
|
||||
} else {
|
||||
baseFactory = QuantilesSketchKeyCollectorFactory.create(clusterBy);
|
||||
baseFactory = QuantilesSketchKeyCollectorFactory.create(clusterBy, rowSignature);
|
||||
}
|
||||
|
||||
// Wrap in DelegateOrMinKeyCollectorFactory, so we are guaranteed to be able to downsample to a single key. This
|
||||
// is important because it allows us to better handle large numbers of small buckets.
|
||||
return new DelegateOrMinKeyCollectorFactory<>(clusterBy.keyComparator(), baseFactory);
|
||||
return new DelegateOrMinKeyCollectorFactory<>(clusterBy.keyComparator(rowSignature), baseFactory);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.datasketches.memory.WritableMemory;
|
|||
import org.apache.datasketches.quantiles.ItemsSketch;
|
||||
import org.apache.druid.frame.key.ClusterBy;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import java.nio.ByteOrder;
|
||||
import java.util.Arrays;
|
||||
|
@ -46,9 +47,9 @@ public class QuantilesSketchKeyCollectorFactory
|
|||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
static QuantilesSketchKeyCollectorFactory create(final ClusterBy clusterBy)
|
||||
static QuantilesSketchKeyCollectorFactory create(final ClusterBy clusterBy, final RowSignature rowSignature)
|
||||
{
|
||||
return new QuantilesSketchKeyCollectorFactory(clusterBy.byteKeyComparator());
|
||||
return new QuantilesSketchKeyCollectorFactory(clusterBy.byteKeyComparator(rowSignature));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,419 @@
|
|||
/*
|
||||
* 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.msq.exec;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||
import org.apache.druid.data.input.impl.LocalInputSource;
|
||||
import org.apache.druid.data.input.impl.systemfield.SystemFields;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.msq.indexing.MSQSpec;
|
||||
import org.apache.druid.msq.indexing.MSQTuningConfig;
|
||||
import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination;
|
||||
import org.apache.druid.msq.test.MSQTestBase;
|
||||
import org.apache.druid.msq.util.MultiStageQueryContext;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.NestedDataTestUtils;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.nested.StructuredData;
|
||||
import org.apache.druid.sql.calcite.external.ExternalDataSource;
|
||||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMapping;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.apache.druid.utils.CompressionUtils;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.StandardCopyOption;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class MSQComplexGroupByTest extends MSQTestBase
|
||||
{
|
||||
static {
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
}
|
||||
|
||||
private String dataFileNameJsonString;
|
||||
private String dataFileSignatureJsonString;
|
||||
private DataSource dataFileExternalDataSource;
|
||||
|
||||
public static Collection<Object[]> data()
|
||||
{
|
||||
Object[][] data = new Object[][]{
|
||||
{DEFAULT, DEFAULT_MSQ_CONTEXT},
|
||||
{DURABLE_STORAGE, DURABLE_STORAGE_MSQ_CONTEXT},
|
||||
{FAULT_TOLERANCE, FAULT_TOLERANCE_MSQ_CONTEXT},
|
||||
{PARALLEL_MERGE, PARALLEL_MERGE_MSQ_CONTEXT}
|
||||
};
|
||||
return Arrays.asList(data);
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws IOException
|
||||
{
|
||||
File dataFile = newTempFile("dataFile");
|
||||
final InputStream resourceStream = this.getClass().getClassLoader()
|
||||
.getResourceAsStream(NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE);
|
||||
final InputStream decompressing = CompressionUtils.decompress(
|
||||
resourceStream,
|
||||
"nested-all-types-test-data.json"
|
||||
);
|
||||
Files.copy(decompressing, dataFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
|
||||
decompressing.close();
|
||||
|
||||
dataFileNameJsonString = queryFramework().queryJsonMapper().writeValueAsString(dataFile);
|
||||
|
||||
RowSignature dataFileSignature = RowSignature.builder()
|
||||
.add("timestamp", ColumnType.STRING)
|
||||
.add("obj", ColumnType.NESTED_DATA)
|
||||
.build();
|
||||
dataFileSignatureJsonString = queryFramework().queryJsonMapper().writeValueAsString(dataFileSignature);
|
||||
|
||||
dataFileExternalDataSource = new ExternalDataSource(
|
||||
new LocalInputSource(null, null, ImmutableList.of(dataFile), SystemFields.none()),
|
||||
new JsonInputFormat(null, null, null, null, null),
|
||||
dataFileSignature
|
||||
);
|
||||
|
||||
objectMapper.registerModules(NestedDataModule.getJacksonModulesList());
|
||||
}
|
||||
|
||||
@MethodSource("data")
|
||||
@ParameterizedTest(name = "{index}:with context {0}")
|
||||
public void testInsertWithoutRollupOnNestedData(String contextName, Map<String, Object> context)
|
||||
{
|
||||
testIngestQuery().setSql("INSERT INTO foo1 SELECT\n"
|
||||
+ " obj,\n"
|
||||
+ " COUNT(*) as cnt\n"
|
||||
+ "FROM TABLE(\n"
|
||||
+ " EXTERN(\n"
|
||||
+ " '{ \"files\": [" + dataFileNameJsonString + "],\"type\":\"local\"}',\n"
|
||||
+ " '{\"type\": \"json\"}',\n"
|
||||
+ " '[{\"name\": \"timestamp\", \"type\": \"STRING\"}, {\"name\": \"obj\", \"type\": \"COMPLEX<json>\"}]'\n"
|
||||
+ " )\n"
|
||||
+ " )\n"
|
||||
+ " GROUP BY 1\n"
|
||||
+ " PARTITIONED BY ALL")
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(RowSignature.builder()
|
||||
.add("__time", ColumnType.LONG)
|
||||
.add("obj", ColumnType.NESTED_DATA)
|
||||
.add("cnt", ColumnType.LONG)
|
||||
.build())
|
||||
.setExpectedResultRows(ImmutableList.of(
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 500,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "e",
|
||||
"z", ImmutableList.of(1, 2, 3, 4)
|
||||
),
|
||||
"v", "a"
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 100,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "a",
|
||||
"y", 1.1,
|
||||
"z", ImmutableList.of(1, 2, 3, 4)
|
||||
),
|
||||
"v", Collections.emptyList()
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 700,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "g",
|
||||
"y", 1.1,
|
||||
"z", Arrays.asList(9, null, 9, 9)
|
||||
),
|
||||
"v", Collections.emptyList()
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 200,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "b",
|
||||
"y", 1.1,
|
||||
"z", ImmutableList.of(2, 4, 6)
|
||||
),
|
||||
"v", Collections.emptyList()
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 600,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "f",
|
||||
"y", 1.1,
|
||||
"z", ImmutableList.of(6, 7, 8, 9)
|
||||
),
|
||||
"v", "b"
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 400,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "d",
|
||||
"y", 1.1,
|
||||
"z", ImmutableList.of(3, 4)
|
||||
),
|
||||
"v", Collections.emptyList()
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(ImmutableMap.of("a", 300)),
|
||||
1L
|
||||
}
|
||||
))
|
||||
.setQueryContext(context)
|
||||
.verifyResults();
|
||||
|
||||
}
|
||||
|
||||
@MethodSource("data")
|
||||
@ParameterizedTest(name = "{index}:with context {0}")
|
||||
public void testInsertWithRollupOnNestedData(String contextName, Map<String, Object> context)
|
||||
{
|
||||
final Map<String, Object> adjustedContext = new HashMap<>(context);
|
||||
adjustedContext.put(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false);
|
||||
adjustedContext.put(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false);
|
||||
testIngestQuery().setSql("INSERT INTO foo1 SELECT\n"
|
||||
+ " obj,\n"
|
||||
+ " COUNT(*) as cnt\n"
|
||||
+ "FROM TABLE(\n"
|
||||
+ " EXTERN(\n"
|
||||
+ " '{ \"files\": [" + dataFileNameJsonString + "],\"type\":\"local\"}',\n"
|
||||
+ " '{\"type\": \"json\"}',\n"
|
||||
+ " '[{\"name\": \"timestamp\", \"type\": \"STRING\"}, {\"name\": \"obj\", \"type\": \"COMPLEX<json>\"}]'\n"
|
||||
+ " )\n"
|
||||
+ " )\n"
|
||||
+ " GROUP BY 1\n"
|
||||
+ " PARTITIONED BY ALL")
|
||||
.setQueryContext(adjustedContext)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(RowSignature.builder()
|
||||
.add("__time", ColumnType.LONG)
|
||||
.add("obj", ColumnType.NESTED_DATA)
|
||||
.add("cnt", ColumnType.LONG)
|
||||
.build())
|
||||
.addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt"))
|
||||
.setExpectedResultRows(ImmutableList.of(
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 500,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "e",
|
||||
"z", ImmutableList.of(1, 2, 3, 4)
|
||||
),
|
||||
"v", "a"
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 100,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "a",
|
||||
"y", 1.1,
|
||||
"z", ImmutableList.of(1, 2, 3, 4)
|
||||
),
|
||||
"v", Collections.emptyList()
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 700,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "g",
|
||||
"y", 1.1,
|
||||
"z", Arrays.asList(9, null, 9, 9)
|
||||
),
|
||||
"v", Collections.emptyList()
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 200,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "b",
|
||||
"y", 1.1,
|
||||
"z", ImmutableList.of(2, 4, 6)
|
||||
),
|
||||
"v", Collections.emptyList()
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 600,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "f",
|
||||
"y", 1.1,
|
||||
"z", ImmutableList.of(6, 7, 8, 9)
|
||||
),
|
||||
"v", "b"
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(
|
||||
ImmutableMap.of(
|
||||
"a", 400,
|
||||
"b", ImmutableMap.of(
|
||||
"x", "d",
|
||||
"y", 1.1,
|
||||
"z", ImmutableList.of(3, 4)
|
||||
),
|
||||
"v", Collections.emptyList()
|
||||
)
|
||||
),
|
||||
1L
|
||||
},
|
||||
new Object[]{
|
||||
0L,
|
||||
StructuredData.wrap(ImmutableMap.of("a", 300)),
|
||||
1L
|
||||
}
|
||||
))
|
||||
.setExpectedRollUp(true)
|
||||
.setQueryContext(adjustedContext)
|
||||
.verifyResults();
|
||||
|
||||
}
|
||||
|
||||
@MethodSource("data")
|
||||
@ParameterizedTest(name = "{index}:with context {0}")
|
||||
public void testSortingOnNestedData(String contextName, Map<String, Object> context)
|
||||
{
|
||||
RowSignature rowSignature = RowSignature.builder()
|
||||
.add("obj", ColumnType.NESTED_DATA)
|
||||
.build();
|
||||
testSelectQuery().setSql("SELECT\n"
|
||||
+ " obj\n"
|
||||
+ "FROM TABLE(\n"
|
||||
+ " EXTERN(\n"
|
||||
+ " '{ \"files\": [" + dataFileNameJsonString + "],\"type\":\"local\"}',\n"
|
||||
+ " '{\"type\": \"json\"}',\n"
|
||||
+ " '[{\"name\": \"timestamp\", \"type\": \"STRING\"}, {\"name\": \"obj\", \"type\": \"COMPLEX<json>\"}]'\n"
|
||||
|
||||
+ " )\n"
|
||||
+ " )\n"
|
||||
+ " ORDER BY 1")
|
||||
.setQueryContext(ImmutableMap.of())
|
||||
.setExpectedMSQSpec(MSQSpec
|
||||
.builder()
|
||||
.query(newScanQueryBuilder()
|
||||
.dataSource(dataFileExternalDataSource)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("obj")
|
||||
.context(defaultScanQueryContext(context, rowSignature))
|
||||
.orderBy(Collections.singletonList(new ScanQuery.OrderBy("obj", ScanQuery.Order.ASCENDING)))
|
||||
.build()
|
||||
)
|
||||
.columnMappings(new ColumnMappings(ImmutableList.of(
|
||||
new ColumnMapping("obj", "obj")
|
||||
)))
|
||||
.tuningConfig(MSQTuningConfig.defaultConfig())
|
||||
.destination(TaskReportMSQDestination.INSTANCE)
|
||||
.build()
|
||||
)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedResultRows(ImmutableList.of(
|
||||
new Object[]{"{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]},\"v\":\"a\"}"},
|
||||
new Object[]{"{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]},\"v\":[]}"},
|
||||
new Object[]{"{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]},\"v\":[]}"},
|
||||
new Object[]{"{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]},\"v\":[]}"},
|
||||
new Object[]{"{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]},\"v\":\"b\"}"},
|
||||
new Object[]{"{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]},\"v\":[]}"},
|
||||
new Object[]{"{\"a\":300}"}
|
||||
))
|
||||
.verifyResults();
|
||||
}
|
||||
}
|
|
@ -41,7 +41,7 @@ public class ControllerChatHandlerTest
|
|||
final Controller controller = Mockito.mock(Controller.class);
|
||||
|
||||
TaskReport.ReportMap reportMap = new TaskReport.ReportMap();
|
||||
reportMap.put("killUnusedSegments", new KillTaskReport("kill_1", new KillTaskReport.Stats(1, 2, 3)));
|
||||
reportMap.put("killUnusedSegments", new KillTaskReport("kill_1", new KillTaskReport.Stats(1, 2)));
|
||||
|
||||
Mockito.when(controller.liveReports())
|
||||
.thenReturn(reportMap);
|
||||
|
|
|
@ -108,7 +108,7 @@ public class ClusterByStatisticsCollectorImplTest extends InitializedNullHandlin
|
|||
.iterator();
|
||||
|
||||
final NavigableMap<RowKey, List<Integer>> sortedKeyWeights =
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator());
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE));
|
||||
|
||||
doTest(
|
||||
clusterBy,
|
||||
|
@ -157,7 +157,7 @@ public class ClusterByStatisticsCollectorImplTest extends InitializedNullHandlin
|
|||
}
|
||||
|
||||
final NavigableMap<RowKey, List<Integer>> sortedKeyWeights =
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator());
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE));
|
||||
|
||||
doTest(
|
||||
clusterBy,
|
||||
|
@ -208,7 +208,7 @@ public class ClusterByStatisticsCollectorImplTest extends InitializedNullHandlin
|
|||
}
|
||||
|
||||
final NavigableMap<RowKey, List<Integer>> sortedKeyWeights =
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator());
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE));
|
||||
|
||||
doTest(
|
||||
clusterBy,
|
||||
|
@ -267,7 +267,7 @@ public class ClusterByStatisticsCollectorImplTest extends InitializedNullHandlin
|
|||
}
|
||||
|
||||
final NavigableMap<RowKey, List<Integer>> sortedKeyWeights =
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator());
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE));
|
||||
|
||||
doTest(
|
||||
clusterBy,
|
||||
|
@ -331,7 +331,7 @@ public class ClusterByStatisticsCollectorImplTest extends InitializedNullHandlin
|
|||
}
|
||||
|
||||
final NavigableMap<RowKey, List<Integer>> sortedKeyWeights =
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator());
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE));
|
||||
|
||||
doTest(
|
||||
clusterBy,
|
||||
|
@ -402,7 +402,7 @@ public class ClusterByStatisticsCollectorImplTest extends InitializedNullHandlin
|
|||
}
|
||||
|
||||
final NavigableMap<RowKey, List<Integer>> sortedKeyWeights =
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator());
|
||||
computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE));
|
||||
|
||||
doTest(
|
||||
clusterBy,
|
||||
|
@ -451,6 +451,26 @@ public class ClusterByStatisticsCollectorImplTest extends InitializedNullHandlin
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testShouldDownsampleSingleBucket()
|
||||
{
|
||||
ClusterByStatisticsCollectorImpl clusterByStatisticsCollector =
|
||||
(ClusterByStatisticsCollectorImpl) ClusterByStatisticsCollectorImpl.create(
|
||||
CLUSTER_BY_XYZ_BUCKET_BY_X,
|
||||
SIGNATURE,
|
||||
35000,
|
||||
500,
|
||||
false,
|
||||
false
|
||||
);
|
||||
|
||||
clusterByStatisticsCollector.add(createKey(CLUSTER_BY_XYZ_BUCKET_BY_X, 2, 1, "value1"), 1);
|
||||
clusterByStatisticsCollector.add(createKey(CLUSTER_BY_XYZ_BUCKET_BY_X, 2, 3, "value2"), 1);
|
||||
clusterByStatisticsCollector.add(createKey(CLUSTER_BY_XYZ_BUCKET_BY_X, 1, 1, "Extremely long key string for unit test; Extremely long key string for unit test;"), 500);
|
||||
|
||||
Assert.assertTrue(clusterByStatisticsCollector.getTotalRetainedBytes() <= 35000);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBucketDownsampledToSingleKeyFinishesCorrectly()
|
||||
{
|
||||
|
@ -551,7 +571,7 @@ public class ClusterByStatisticsCollectorImplTest extends InitializedNullHandlin
|
|||
final BiConsumer<String, ClusterByStatisticsCollectorImpl> testFn
|
||||
)
|
||||
{
|
||||
final Comparator<RowKey> comparator = clusterBy.keyComparator();
|
||||
final Comparator<RowKey> comparator = clusterBy.keyComparator(SIGNATURE);
|
||||
|
||||
// Load into single collector, sorted order.
|
||||
final ClusterByStatisticsCollectorImpl sortedCollector = makeCollector(clusterBy, aggregate);
|
||||
|
@ -649,7 +669,7 @@ public class ClusterByStatisticsCollectorImplTest extends InitializedNullHandlin
|
|||
testName,
|
||||
partitions,
|
||||
sortedKeyWeights.firstKey(),
|
||||
clusterBy.keyComparator()
|
||||
clusterBy.keyComparator(SIGNATURE)
|
||||
);
|
||||
verifyPartitionWeights(testName, clusterBy, partitions, sortedKeyWeights, aggregate, expectedPartitionSize);
|
||||
}
|
||||
|
|
|
@ -41,7 +41,7 @@ public class DelegateOrMinKeyCollectorTest
|
|||
{
|
||||
private final ClusterBy clusterBy = new ClusterBy(ImmutableList.of(new KeyColumn("x", KeyOrder.ASCENDING)), 0);
|
||||
private final RowSignature signature = RowSignature.builder().add("x", ColumnType.LONG).build();
|
||||
private final Comparator<RowKey> comparator = clusterBy.keyComparator();
|
||||
private final Comparator<RowKey> comparator = clusterBy.keyComparator(signature);
|
||||
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
|
@ -53,7 +53,7 @@ public class DelegateOrMinKeyCollectorTest
|
|||
final DelegateOrMinKeyCollector<QuantilesSketchKeyCollector> collector =
|
||||
new DelegateOrMinKeyCollectorFactory<>(
|
||||
comparator,
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy)
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, signature)
|
||||
).newKeyCollector();
|
||||
|
||||
Assert.assertTrue(collector.getDelegate().isPresent());
|
||||
|
@ -69,8 +69,8 @@ public class DelegateOrMinKeyCollectorTest
|
|||
{
|
||||
ClusterBy clusterBy = ClusterBy.none();
|
||||
new DelegateOrMinKeyCollector<>(
|
||||
clusterBy.keyComparator(),
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy).newKeyCollector(),
|
||||
clusterBy.keyComparator(RowSignature.empty()),
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, RowSignature.empty()).newKeyCollector(),
|
||||
RowKey.empty()
|
||||
);
|
||||
}
|
||||
|
@ -81,7 +81,7 @@ public class DelegateOrMinKeyCollectorTest
|
|||
final DelegateOrMinKeyCollector<QuantilesSketchKeyCollector> collector =
|
||||
new DelegateOrMinKeyCollectorFactory<>(
|
||||
comparator,
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy)
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, signature)
|
||||
).newKeyCollector();
|
||||
|
||||
RowKey key = createKey(1L);
|
||||
|
@ -100,7 +100,7 @@ public class DelegateOrMinKeyCollectorTest
|
|||
final DelegateOrMinKeyCollector<QuantilesSketchKeyCollector> collector =
|
||||
new DelegateOrMinKeyCollectorFactory<>(
|
||||
comparator,
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy)
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, signature)
|
||||
).newKeyCollector();
|
||||
|
||||
RowKey key = createKey(1L);
|
||||
|
@ -128,7 +128,7 @@ public class DelegateOrMinKeyCollectorTest
|
|||
final DelegateOrMinKeyCollector<QuantilesSketchKeyCollector> collector =
|
||||
new DelegateOrMinKeyCollectorFactory<>(
|
||||
comparator,
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy)
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, signature)
|
||||
).newKeyCollector();
|
||||
|
||||
RowKey key = createKey(1L);
|
||||
|
|
|
@ -28,6 +28,8 @@ import org.apache.druid.frame.key.KeyColumn;
|
|||
import org.apache.druid.frame.key.KeyOrder;
|
||||
import org.apache.druid.frame.key.RowKey;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.hamcrest.MatcherAssert;
|
||||
import org.hamcrest.Matchers;
|
||||
import org.junit.Assert;
|
||||
|
@ -42,7 +44,8 @@ import java.util.NoSuchElementException;
|
|||
public class DistinctKeyCollectorTest
|
||||
{
|
||||
private final ClusterBy clusterBy = new ClusterBy(ImmutableList.of(new KeyColumn("x", KeyOrder.ASCENDING)), 0);
|
||||
private final Comparator<RowKey> comparator = clusterBy.keyComparator();
|
||||
private final RowSignature signature = RowSignature.builder().add("x", ColumnType.LONG).build();
|
||||
private final Comparator<RowKey> comparator = clusterBy.keyComparator(signature);
|
||||
private final int numKeys = 500_000;
|
||||
|
||||
static {
|
||||
|
@ -53,7 +56,7 @@ public class DistinctKeyCollectorTest
|
|||
public void test_empty()
|
||||
{
|
||||
KeyCollectorTestUtils.doTest(
|
||||
DistinctKeyCollectorFactory.create(clusterBy),
|
||||
DistinctKeyCollectorFactory.create(clusterBy, signature),
|
||||
Collections.emptyList(),
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
@ -77,7 +80,7 @@ public class DistinctKeyCollectorTest
|
|||
ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator);
|
||||
|
||||
KeyCollectorTestUtils.doTest(
|
||||
DistinctKeyCollectorFactory.create(clusterBy),
|
||||
DistinctKeyCollectorFactory.create(clusterBy, signature),
|
||||
keyWeights,
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
@ -95,7 +98,7 @@ public class DistinctKeyCollectorTest
|
|||
ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator);
|
||||
|
||||
KeyCollectorTestUtils.doTest(
|
||||
DistinctKeyCollectorFactory.create(clusterBy),
|
||||
DistinctKeyCollectorFactory.create(clusterBy, signature),
|
||||
keyWeights,
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
@ -113,7 +116,7 @@ public class DistinctKeyCollectorTest
|
|||
@Test(expected = IllegalArgumentException.class)
|
||||
public void test_generateWithNegativeTargetWeight_throwsException()
|
||||
{
|
||||
DistinctKeyCollector distinctKeyCollector = DistinctKeyCollectorFactory.create(clusterBy).newKeyCollector();
|
||||
DistinctKeyCollector distinctKeyCollector = DistinctKeyCollectorFactory.create(clusterBy, signature).newKeyCollector();
|
||||
distinctKeyCollector.generatePartitionsWithTargetWeight(-1);
|
||||
}
|
||||
|
||||
|
@ -125,7 +128,7 @@ public class DistinctKeyCollectorTest
|
|||
ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator).firstKey();
|
||||
|
||||
KeyCollectorTestUtils.doTest(
|
||||
DistinctKeyCollectorFactory.create(clusterBy),
|
||||
DistinctKeyCollectorFactory.create(clusterBy, signature),
|
||||
keyWeights,
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
@ -161,7 +164,7 @@ public class DistinctKeyCollectorTest
|
|||
ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator);
|
||||
|
||||
KeyCollectorTestUtils.doTest(
|
||||
DistinctKeyCollectorFactory.create(clusterBy),
|
||||
DistinctKeyCollectorFactory.create(clusterBy, signature),
|
||||
keyWeights,
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
@ -184,7 +187,7 @@ public class DistinctKeyCollectorTest
|
|||
ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator);
|
||||
|
||||
KeyCollectorTestUtils.doTest(
|
||||
DistinctKeyCollectorFactory.create(clusterBy),
|
||||
DistinctKeyCollectorFactory.create(clusterBy, signature),
|
||||
keyWeights,
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
@ -211,7 +214,7 @@ public class DistinctKeyCollectorTest
|
|||
ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator);
|
||||
|
||||
KeyCollectorTestUtils.doTest(
|
||||
DistinctKeyCollectorFactory.create(clusterBy),
|
||||
DistinctKeyCollectorFactory.create(clusterBy, signature),
|
||||
keyWeights,
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
|
|
@ -43,7 +43,8 @@ import java.util.NoSuchElementException;
|
|||
public class QuantilesSketchKeyCollectorTest
|
||||
{
|
||||
private final ClusterBy clusterBy = new ClusterBy(ImmutableList.of(new KeyColumn("x", KeyOrder.ASCENDING)), 0);
|
||||
private final Comparator<RowKey> comparator = clusterBy.keyComparator();
|
||||
private final RowSignature signature = RowSignature.builder().add("x", ColumnType.LONG).build();
|
||||
private final Comparator<RowKey> comparator = clusterBy.keyComparator(signature);
|
||||
private final int numKeys = 500_000;
|
||||
|
||||
static {
|
||||
|
@ -54,7 +55,7 @@ public class QuantilesSketchKeyCollectorTest
|
|||
public void test_empty()
|
||||
{
|
||||
KeyCollectorTestUtils.doTest(
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy),
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, signature),
|
||||
Collections.emptyList(),
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
@ -78,7 +79,7 @@ public class QuantilesSketchKeyCollectorTest
|
|||
ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator);
|
||||
|
||||
KeyCollectorTestUtils.doTest(
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy),
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, signature),
|
||||
keyWeights,
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
@ -96,7 +97,7 @@ public class QuantilesSketchKeyCollectorTest
|
|||
ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator);
|
||||
|
||||
KeyCollectorTestUtils.doTest(
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy),
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, signature),
|
||||
keyWeights,
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
@ -114,7 +115,7 @@ public class QuantilesSketchKeyCollectorTest
|
|||
ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator).firstKey();
|
||||
|
||||
KeyCollectorTestUtils.doTest(
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy),
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, signature),
|
||||
keyWeights,
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
@ -147,7 +148,7 @@ public class QuantilesSketchKeyCollectorTest
|
|||
ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator);
|
||||
|
||||
KeyCollectorTestUtils.doTest(
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy),
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, signature),
|
||||
keyWeights,
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
@ -169,8 +170,8 @@ public class QuantilesSketchKeyCollectorTest
|
|||
public void testAverageKeyLength()
|
||||
{
|
||||
final QuantilesSketchKeyCollector collector =
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy).newKeyCollector();
|
||||
final QuantilesSketchKeyCollector other = QuantilesSketchKeyCollectorFactory.create(clusterBy).newKeyCollector();
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, signature).newKeyCollector();
|
||||
final QuantilesSketchKeyCollector other = QuantilesSketchKeyCollectorFactory.create(clusterBy, signature).newKeyCollector();
|
||||
|
||||
RowSignature smallKeySignature = KeyTestUtils.createKeySignature(
|
||||
new ClusterBy(ImmutableList.of(new KeyColumn("x", KeyOrder.ASCENDING)), 0).getColumns(),
|
||||
|
@ -219,7 +220,7 @@ public class QuantilesSketchKeyCollectorTest
|
|||
ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator);
|
||||
|
||||
KeyCollectorTestUtils.doTest(
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy),
|
||||
QuantilesSketchKeyCollectorFactory.create(clusterBy, signature),
|
||||
keyWeights,
|
||||
comparator,
|
||||
(testName, collector) -> {
|
||||
|
|
|
@ -77,6 +77,7 @@ import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFacto
|
|||
import org.apache.druid.server.SegmentManager;
|
||||
import org.apache.druid.server.coordination.DataSegmentAnnouncer;
|
||||
import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer;
|
||||
import org.apache.druid.sql.calcite.CalciteNestedDataQueryTest;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.apache.druid.sql.calcite.util.TestDataBuilder;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
@ -292,6 +293,101 @@ public class CalciteMSQTestsHelper
|
|||
.inputTmpDir(tempFolderProducer.apply("tmpDir"))
|
||||
.buildMMappedIndex();
|
||||
break;
|
||||
case CalciteNestedDataQueryTest.DATA_SOURCE:
|
||||
case CalciteNestedDataQueryTest.DATA_SOURCE_MIXED:
|
||||
if (segmentId.getPartitionNum() == 0) {
|
||||
index = IndexBuilder.create()
|
||||
.tmpDir(tempFolderProducer.apply("tmpDir"))
|
||||
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new CountAggregatorFactory("cnt")
|
||||
)
|
||||
.withDimensionsSpec(CalciteNestedDataQueryTest.ALL_JSON_COLUMNS.getDimensionsSpec())
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.rows(CalciteNestedDataQueryTest.ROWS)
|
||||
.buildMMappedIndex();
|
||||
} else if (segmentId.getPartitionNum() == 1) {
|
||||
index = IndexBuilder.create()
|
||||
.tmpDir(tempFolderProducer.apply("tmpDir"))
|
||||
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new CountAggregatorFactory("cnt")
|
||||
)
|
||||
.withDimensionsSpec(CalciteNestedDataQueryTest.JSON_AND_SCALAR_MIX.getDimensionsSpec())
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.rows(CalciteNestedDataQueryTest.ROWS_MIX)
|
||||
.buildMMappedIndex();
|
||||
} else {
|
||||
throw new ISE("Cannot query segment %s in test runner", segmentId);
|
||||
}
|
||||
break;
|
||||
case CalciteNestedDataQueryTest.DATA_SOURCE_MIXED_2:
|
||||
if (segmentId.getPartitionNum() == 0) {
|
||||
index = IndexBuilder.create()
|
||||
.tmpDir(tempFolderProducer.apply("tmpDir"))
|
||||
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new CountAggregatorFactory("cnt")
|
||||
)
|
||||
.withDimensionsSpec(CalciteNestedDataQueryTest.JSON_AND_SCALAR_MIX.getDimensionsSpec())
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.rows(CalciteNestedDataQueryTest.ROWS_MIX)
|
||||
.buildMMappedIndex();
|
||||
} else if (segmentId.getPartitionNum() == 1) {
|
||||
index = IndexBuilder.create()
|
||||
.tmpDir(tempFolderProducer.apply("tmpDir"))
|
||||
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new CountAggregatorFactory("cnt")
|
||||
)
|
||||
.withDimensionsSpec(CalciteNestedDataQueryTest.ALL_JSON_COLUMNS.getDimensionsSpec())
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.rows(CalciteNestedDataQueryTest.ROWS)
|
||||
.buildMMappedIndex();
|
||||
} else {
|
||||
throw new ISE("Cannot query segment %s in test runner", segmentId);
|
||||
}
|
||||
break;
|
||||
case CalciteNestedDataQueryTest.DATA_SOURCE_ALL:
|
||||
index = IndexBuilder.create()
|
||||
.tmpDir(tempFolderProducer.apply("tmpDir"))
|
||||
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withTimestampSpec(NestedDataTestUtils.AUTO_SCHEMA.getTimestampSpec())
|
||||
.withDimensionsSpec(NestedDataTestUtils.AUTO_SCHEMA.getDimensionsSpec())
|
||||
.withMetrics(
|
||||
new CountAggregatorFactory("cnt")
|
||||
)
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.inputSource(
|
||||
ResourceInputSource.of(
|
||||
NestedDataTestUtils.class.getClassLoader(),
|
||||
NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE
|
||||
)
|
||||
)
|
||||
.inputFormat(TestDataBuilder.DEFAULT_JSON_INPUT_FORMAT)
|
||||
.inputTmpDir(tempFolderProducer.apply("tmpDir"))
|
||||
.buildMMappedIndex();
|
||||
break;
|
||||
case CalciteTests.WIKIPEDIA_FIRST_LAST:
|
||||
index = TestDataBuilder.makeWikipediaIndexWithAggregation(tempFolderProducer.apply("tmpDir"));
|
||||
break;
|
||||
|
|
|
@ -0,0 +1,112 @@
|
|||
/*
|
||||
* 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.msq.test;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Module;
|
||||
import org.apache.druid.guice.DruidInjectorBuilder;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.msq.exec.WorkerMemoryParameters;
|
||||
import org.apache.druid.msq.sql.MSQTaskSqlEngine;
|
||||
import org.apache.druid.query.groupby.TestGroupByBuffers;
|
||||
import org.apache.druid.server.QueryLifecycleFactory;
|
||||
import org.apache.druid.sql.calcite.CalciteNestedDataQueryTest;
|
||||
import org.apache.druid.sql.calcite.QueryTestBuilder;
|
||||
import org.apache.druid.sql.calcite.SqlTestFrameworkConfig;
|
||||
import org.apache.druid.sql.calcite.TempDirProducer;
|
||||
import org.apache.druid.sql.calcite.run.SqlEngine;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
/**
|
||||
* Runs {@link CalciteNestedDataQueryTest} but with MSQ engine
|
||||
*/
|
||||
@SqlTestFrameworkConfig.ComponentSupplier(CalciteNestedDataQueryMSQTest.NestedDataQueryMSQComponentSupplier.class)
|
||||
public class CalciteNestedDataQueryMSQTest extends CalciteNestedDataQueryTest
|
||||
{
|
||||
|
||||
public static class NestedDataQueryMSQComponentSupplier extends NestedComponentSupplier
|
||||
{
|
||||
public NestedDataQueryMSQComponentSupplier(TempDirProducer tempFolderProducer)
|
||||
{
|
||||
super(tempFolderProducer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configureGuice(DruidInjectorBuilder builder)
|
||||
{
|
||||
super.configureGuice(builder);
|
||||
builder.addModules(
|
||||
CalciteMSQTestsHelper.fetchModules(tempDirProducer::newTempFolder, TestGroupByBuffers.createDefault()).toArray(new Module[0])
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SqlEngine createEngine(
|
||||
QueryLifecycleFactory qlf,
|
||||
ObjectMapper queryJsonMapper,
|
||||
Injector injector
|
||||
)
|
||||
{
|
||||
final WorkerMemoryParameters workerMemoryParameters =
|
||||
WorkerMemoryParameters.createInstance(
|
||||
WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50,
|
||||
2,
|
||||
10,
|
||||
2,
|
||||
0,
|
||||
0
|
||||
);
|
||||
final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient(
|
||||
queryJsonMapper,
|
||||
injector,
|
||||
new MSQTestTaskActionClient(queryJsonMapper, injector),
|
||||
workerMemoryParameters,
|
||||
ImmutableList.of()
|
||||
);
|
||||
return new MSQTaskSqlEngine(indexingServiceClient, queryJsonMapper);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected QueryTestBuilder testBuilder()
|
||||
{
|
||||
return new QueryTestBuilder(new CalciteTestConfig(true))
|
||||
.addCustomRunner(new ExtractResultsFactory(() -> (MSQTestOverlordServiceClient) ((MSQTaskSqlEngine) queryFramework().engine()).overlordClient()))
|
||||
.skipVectorize(true)
|
||||
.verifyNativeQueries(new VerifyMSQSupportedNativeQueriesPredicate());
|
||||
}
|
||||
|
||||
@Override
|
||||
@Test
|
||||
public void testJoinOnNestedColumnThrows()
|
||||
{
|
||||
Assertions.assertThrows(ISE.class, () -> {
|
||||
testQuery(
|
||||
"SELECT * FROM druid.nested a INNER JOIN druid.nested b ON a.nester = b.nester",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of()
|
||||
);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -20,8 +20,10 @@
|
|||
package org.apache.druid.msq.test;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.msq.indexing.report.MSQResultsReport;
|
||||
import org.apache.druid.msq.indexing.report.MSQTaskReport;
|
||||
import org.apache.druid.msq.indexing.report.MSQTaskReportPayload;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.sql.calcite.QueryTestBuilder;
|
||||
import org.apache.druid.sql.calcite.QueryTestRunner;
|
||||
import org.junit.Assert;
|
||||
|
@ -55,6 +57,7 @@ public class ExtractResultsFactory implements QueryTestRunner.QueryRunStepFactor
|
|||
return new QueryTestRunner.BaseExecuteQuery(builder)
|
||||
{
|
||||
final List<QueryTestRunner.QueryResults> extractedResults = new ArrayList<>();
|
||||
final RowSignature resultsSignature = null;
|
||||
|
||||
final MSQTestOverlordServiceClient overlordClient = overlordClientSupplier.get();
|
||||
|
||||
|
@ -99,7 +102,10 @@ public class ExtractResultsFactory implements QueryTestRunner.QueryRunStepFactor
|
|||
if (resultRows == null) {
|
||||
throw new ISE("Results report not present in the task's report payload");
|
||||
}
|
||||
extractedResults.add(results.withResults(resultRows));
|
||||
extractedResults.add(
|
||||
results.withSignatureAndResults(
|
||||
convertColumnAndTypeToRowSignature(payload.getResults().getSignature()), resultRows)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -108,6 +114,15 @@ public class ExtractResultsFactory implements QueryTestRunner.QueryRunStepFactor
|
|||
{
|
||||
return extractedResults;
|
||||
}
|
||||
|
||||
private RowSignature convertColumnAndTypeToRowSignature(final List<MSQResultsReport.ColumnAndType> columnAndTypes)
|
||||
{
|
||||
final RowSignature.Builder builder = RowSignature.builder();
|
||||
for (MSQResultsReport.ColumnAndType columnAndType : columnAndTypes) {
|
||||
builder.add(columnAndType.getName(), columnAndType.getType());
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1230,7 +1230,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
// assert rollup
|
||||
Assert.assertEquals(expectedRollUp, queryableIndex.getMetadata().isRollup());
|
||||
|
||||
// asset query granularity
|
||||
// assert query granularity
|
||||
Assert.assertEquals(expectedQueryGranularity, queryableIndex.getMetadata().getQueryGranularity());
|
||||
|
||||
// assert aggregator factories
|
||||
|
@ -1262,7 +1262,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
.collect(Collectors.toList());
|
||||
|
||||
log.info(
|
||||
"Found rows which are sorted forcefully %s",
|
||||
"Found rows which are sorted forcefully\n%s",
|
||||
transformedOutputRows.stream().map(Arrays::deepToString).collect(Collectors.joining("\n"))
|
||||
);
|
||||
|
||||
|
|
|
@ -92,6 +92,10 @@ public class S3Utils
|
|||
} else if (e instanceof SdkClientException && e.getMessage().contains("Unable to execute HTTP request")) {
|
||||
// This is likely due to a temporary DNS issue and can be retried.
|
||||
return true;
|
||||
} else if (e instanceof SdkClientException && e.getMessage().contains("Unable to find a region via the region provider chain")) {
|
||||
// This can happen sometimes when AWS isn't able to obtain the credentials for some service:
|
||||
// https://github.com/aws/aws-sdk-java/issues/2285
|
||||
return true;
|
||||
} else if (e instanceof AmazonClientException) {
|
||||
return AWSClientUtil.isClientExceptionRecoverable((AmazonClientException) e);
|
||||
} else {
|
||||
|
@ -101,8 +105,8 @@ public class S3Utils
|
|||
};
|
||||
|
||||
/**
|
||||
* Retries S3 operations that fail due to io-related exceptions. Service-level exceptions (access denied, file not
|
||||
* found, etc) are not retried.
|
||||
* Retries S3 operations that fail intermittently (due to io-related exceptions, during obtaining credentials, etc).
|
||||
* Service-level exceptions (access denied, file not found, etc) are not retried.
|
||||
*/
|
||||
public static <T> T retryS3Operation(Task<T> f) throws Exception
|
||||
{
|
||||
|
@ -110,8 +114,9 @@ public class S3Utils
|
|||
}
|
||||
|
||||
/**
|
||||
* Retries S3 operations that fail due to io-related exceptions. Service-level exceptions (access denied, file not
|
||||
* found, etc) are not retried. Also provide a way to set maxRetries that can be useful, i.e. for testing.
|
||||
* Retries S3 operations that fail intermittently (due to io-related exceptions, during obtaining credentials, etc).
|
||||
* Service-level exceptions (access denied, file not found, etc) are not retried.
|
||||
* Also provide a way to set maxRetries that can be useful, i.e. for testing.
|
||||
*/
|
||||
public static <T> T retryS3Operation(Task<T> f, int maxRetries) throws Exception
|
||||
{
|
||||
|
|
|
@ -204,7 +204,15 @@ public class ServerSideEncryptingAmazonS3
|
|||
throw new ISE("S3StorageConfig cannot be null!");
|
||||
}
|
||||
|
||||
return new ServerSideEncryptingAmazonS3(amazonS3ClientBuilder.build(), s3StorageConfig.getServerSideEncryption());
|
||||
AmazonS3 amazonS3Client;
|
||||
try {
|
||||
amazonS3Client = S3Utils.retryS3Operation(() -> amazonS3ClientBuilder.build());
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
return new ServerSideEncryptingAmazonS3(amazonS3Client, s3StorageConfig.getServerSideEncryption());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.storage.s3;
|
||||
|
||||
import com.amazonaws.SdkClientException;
|
||||
import com.amazonaws.services.s3.model.AmazonS3Exception;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -108,4 +109,25 @@ public class S3UtilsTest
|
|||
);
|
||||
Assert.assertEquals(maxRetries, count.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRetryWithSdkClientException() throws Exception
|
||||
{
|
||||
final int maxRetries = 3;
|
||||
final AtomicInteger count = new AtomicInteger();
|
||||
S3Utils.retryS3Operation(
|
||||
() -> {
|
||||
if (count.incrementAndGet() >= maxRetries) {
|
||||
return "hey";
|
||||
} else {
|
||||
throw new SdkClientException(
|
||||
"Unable to find a region via the region provider chain. "
|
||||
+ "Must provide an explicit region in the builder or setup environment to supply a region."
|
||||
);
|
||||
}
|
||||
},
|
||||
maxRetries
|
||||
);
|
||||
Assert.assertEquals(maxRetries, count.get());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,85 +0,0 @@
|
|||
/*
|
||||
* 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.indexing.common.actions;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
/**
|
||||
* TaskAction to release a {@link org.apache.druid.indexing.common.SegmentLock}.
|
||||
* Used by batch tasks when they fail to acquire all necessary locks.
|
||||
*/
|
||||
public class SegmentLockReleaseAction implements TaskAction<Void>
|
||||
{
|
||||
private final Interval interval;
|
||||
private final int partitionId;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentLockReleaseAction(@JsonProperty Interval interval, @JsonProperty int partitionId)
|
||||
{
|
||||
this.interval = interval;
|
||||
this.partitionId = partitionId;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getPartitionId()
|
||||
{
|
||||
return partitionId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Void> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<Void>()
|
||||
{
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Void perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
toolbox.getTaskLockbox().unlock(task, interval, partitionId);
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAudited()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "SegmentLockReleaseAction{" +
|
||||
"interval=" + interval +
|
||||
", partitionId=" + partitionId +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -32,13 +32,14 @@ import org.apache.druid.indexer.TaskStatus;
|
|||
import org.apache.druid.indexer.report.KillTaskReport;
|
||||
import org.apache.druid.indexer.report.TaskReport;
|
||||
import org.apache.druid.indexing.common.TaskLock;
|
||||
import org.apache.druid.indexing.common.TaskLockType;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.actions.MarkSegmentsAsUnusedAction;
|
||||
import org.apache.druid.indexing.common.actions.RetrieveUnusedSegmentsAction;
|
||||
import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction;
|
||||
import org.apache.druid.indexing.common.actions.SegmentNukeAction;
|
||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||
import org.apache.druid.indexing.common.actions.TaskLocks;
|
||||
import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
|
||||
import org.apache.druid.indexing.overlord.Segments;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -46,7 +47,6 @@ import org.apache.druid.java.util.common.logger.Logger;
|
|||
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
|
||||
import org.apache.druid.server.security.ResourceAction;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.utils.CollectionUtils;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
@ -66,9 +66,6 @@ import java.util.stream.Collectors;
|
|||
* The client representation of this task is {@link ClientKillUnusedSegmentsTaskQuery}.
|
||||
* JSON serialization fields of this class must correspond to those of {@link
|
||||
* ClientKillUnusedSegmentsTaskQuery}, except for {@link #id} and {@link #context} fields.
|
||||
* <p>
|
||||
* The field {@link #isMarkAsUnused()} is now deprecated.
|
||||
* </p>
|
||||
*/
|
||||
public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
||||
{
|
||||
|
@ -91,8 +88,6 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
|||
@Nullable
|
||||
private final List<String> versions;
|
||||
|
||||
@Deprecated
|
||||
private final boolean markAsUnused;
|
||||
/**
|
||||
* Split processing to try and keep each nuke operation relatively short, in the case that either
|
||||
* the database or the storage layer is particularly slow.
|
||||
|
@ -117,7 +112,6 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
|||
@JsonProperty("interval") Interval interval,
|
||||
@JsonProperty("versions") @Nullable List<String> versions,
|
||||
@JsonProperty("context") Map<String, Object> context,
|
||||
@JsonProperty("markAsUnused") @Deprecated Boolean markAsUnused,
|
||||
@JsonProperty("batchSize") Integer batchSize,
|
||||
@JsonProperty("limit") @Nullable Integer limit,
|
||||
@JsonProperty("maxUsedStatusLastUpdatedTime") @Nullable DateTime maxUsedStatusLastUpdatedTime
|
||||
|
@ -129,7 +123,6 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
|||
interval,
|
||||
context
|
||||
);
|
||||
this.markAsUnused = markAsUnused != null && markAsUnused;
|
||||
this.batchSize = (batchSize != null) ? batchSize : DEFAULT_SEGMENT_NUKE_BATCH_SIZE;
|
||||
if (this.batchSize <= 0) {
|
||||
throw InvalidInput.exception("batchSize[%d] must be a positive integer.", batchSize);
|
||||
|
@ -137,14 +130,6 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
|||
if (limit != null && limit <= 0) {
|
||||
throw InvalidInput.exception("limit[%d] must be a positive integer.", limit);
|
||||
}
|
||||
if (Boolean.TRUE.equals(markAsUnused)) {
|
||||
if (limit != null) {
|
||||
throw InvalidInput.exception("limit[%d] cannot be provided when markAsUnused is enabled.", limit);
|
||||
}
|
||||
if (!CollectionUtils.isNullOrEmpty(versions)) {
|
||||
throw InvalidInput.exception("versions[%s] cannot be provided when markAsUnused is enabled.", versions);
|
||||
}
|
||||
}
|
||||
this.versions = versions;
|
||||
this.limit = limit;
|
||||
this.maxUsedStatusLastUpdatedTime = maxUsedStatusLastUpdatedTime;
|
||||
|
@ -158,21 +143,6 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
|||
return versions;
|
||||
}
|
||||
|
||||
/**
|
||||
* This field has been deprecated as "kill" tasks should not be responsible for
|
||||
* marking segments as unused. Instead, users should call the Coordinator API
|
||||
* {@code /{dataSourceName}/markUnused} to explicitly mark segments as unused.
|
||||
* Segments may also be marked unused by the Coordinator if they become overshadowed
|
||||
* or have a {@code DropRule} applied to them.
|
||||
*/
|
||||
@Deprecated
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||
public boolean isMarkAsUnused()
|
||||
{
|
||||
return markAsUnused;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||
public int getBatchSize()
|
||||
|
@ -214,16 +184,6 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
|||
// Track stats for reporting
|
||||
int numSegmentsKilled = 0;
|
||||
int numBatchesProcessed = 0;
|
||||
final int numSegmentsMarkedAsUnused;
|
||||
if (markAsUnused) {
|
||||
numSegmentsMarkedAsUnused = toolbox.getTaskActionClient().submit(
|
||||
new MarkSegmentsAsUnusedAction(getDataSource(), getInterval())
|
||||
);
|
||||
LOG.info("Marked [%d] segments of datasource[%s] in interval[%s] as unused.",
|
||||
numSegmentsMarkedAsUnused, getDataSource(), getInterval());
|
||||
} else {
|
||||
numSegmentsMarkedAsUnused = 0;
|
||||
}
|
||||
|
||||
// List unused segments
|
||||
int nextBatchSize = computeNextBatchSize(numSegmentsKilled);
|
||||
|
@ -304,7 +264,7 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
|||
);
|
||||
|
||||
final KillTaskReport.Stats stats =
|
||||
new KillTaskReport.Stats(numSegmentsKilled, numBatchesProcessed, numSegmentsMarkedAsUnused);
|
||||
new KillTaskReport.Stats(numSegmentsKilled, numBatchesProcessed);
|
||||
toolbox.getTaskReportFileWriter().write(
|
||||
taskId,
|
||||
TaskReport.buildTaskReports(new KillTaskReport(taskId, stats))
|
||||
|
@ -346,4 +306,41 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
|||
{
|
||||
return LookupLoadingSpec.NONE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isReady(TaskActionClient taskActionClient) throws Exception
|
||||
{
|
||||
final boolean useConcurrentLocks = Boolean.TRUE.equals(
|
||||
getContextValue(
|
||||
Tasks.USE_CONCURRENT_LOCKS,
|
||||
Tasks.DEFAULT_USE_CONCURRENT_LOCKS
|
||||
)
|
||||
);
|
||||
|
||||
TaskLockType actualLockType = determineLockType(useConcurrentLocks);
|
||||
|
||||
final TaskLock lock = taskActionClient.submit(
|
||||
new TimeChunkLockTryAcquireAction(
|
||||
actualLockType,
|
||||
getInterval()
|
||||
)
|
||||
);
|
||||
if (lock == null) {
|
||||
return false;
|
||||
}
|
||||
lock.assertNotRevoked();
|
||||
return true;
|
||||
}
|
||||
|
||||
private TaskLockType determineLockType(boolean useConcurrentLocks)
|
||||
{
|
||||
TaskLockType actualLockType;
|
||||
if (useConcurrentLocks) {
|
||||
actualLockType = TaskLockType.REPLACE;
|
||||
} else {
|
||||
actualLockType = getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE);
|
||||
}
|
||||
return actualLockType;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.google.common.base.MoreObjects;
|
|||
import com.google.common.base.Objects;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ComparisonChain;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Ordering;
|
||||
|
@ -63,13 +62,13 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
|
@ -766,30 +765,12 @@ public class TaskLockbox
|
|||
giant.lock();
|
||||
|
||||
try {
|
||||
return action.perform(isTaskLocksValid(task, intervals));
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check all locks task acquired are still valid.
|
||||
* It doesn't check other semantics like acquired locks are enough to overwrite existing segments.
|
||||
* This kind of semantic should be checked in each caller of {@link #doInCriticalSection}.
|
||||
*/
|
||||
private boolean isTaskLocksValid(Task task, Set<Interval> intervals)
|
||||
{
|
||||
giant.lock();
|
||||
try {
|
||||
return intervals
|
||||
.stream()
|
||||
.allMatch(interval -> {
|
||||
final List<TaskLockPosse> lockPosses = getOnlyTaskLockPosseContainingInterval(task, interval);
|
||||
return lockPosses.stream().map(TaskLockPosse::getTaskLock).noneMatch(
|
||||
TaskLock::isRevoked
|
||||
);
|
||||
});
|
||||
// Check if any of the locks held by this task have been revoked
|
||||
final boolean areTaskLocksValid = intervals.stream().noneMatch(interval -> {
|
||||
Optional<TaskLockPosse> lockPosse = getOnlyTaskLockPosseContainingInterval(task, interval);
|
||||
return lockPosse.isPresent() && lockPosse.get().getTaskLock().isRevoked();
|
||||
});
|
||||
return action.perform(areTaskLocksValid);
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
|
@ -801,7 +782,7 @@ public class TaskLockbox
|
|||
giant.lock();
|
||||
|
||||
try {
|
||||
lockPosse.forEachTask(taskId -> revokeLock(taskId, lockPosse.getTaskLock()));
|
||||
lockPosse.taskIds.forEach(taskId -> revokeLock(taskId, lockPosse.getTaskLock()));
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
|
@ -1083,22 +1064,20 @@ public class TaskLockbox
|
|||
* @param task task to unlock
|
||||
* @param interval interval to unlock
|
||||
*/
|
||||
public void unlock(final Task task, final Interval interval, @Nullable Integer partitionId)
|
||||
private void unlock(final Task task, final Interval interval, @Nullable Integer partitionId)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
final String dataSource = task.getDataSource();
|
||||
final NavigableMap<DateTime, SortedMap<Interval, List<TaskLockPosse>>> dsRunning = running.get(
|
||||
task.getDataSource()
|
||||
);
|
||||
|
||||
if (dsRunning == null || dsRunning.isEmpty()) {
|
||||
final NavigableMap<DateTime, SortedMap<Interval, List<TaskLockPosse>>> locksForDatasource
|
||||
= running.get(task.getDataSource());
|
||||
if (locksForDatasource == null || locksForDatasource.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
final SortedMap<Interval, List<TaskLockPosse>> intervalToPosses = dsRunning.get(interval.getStart());
|
||||
|
||||
final SortedMap<Interval, List<TaskLockPosse>> intervalToPosses
|
||||
= locksForDatasource.get(interval.getStart());
|
||||
if (intervalToPosses == null || intervalToPosses.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
@ -1126,18 +1105,15 @@ public class TaskLockbox
|
|||
final boolean removed = taskLockPosse.removeTask(task);
|
||||
|
||||
if (taskLockPosse.isTasksEmpty()) {
|
||||
log.info("TaskLock is now empty: %s", taskLock);
|
||||
log.info("TaskLock[%s] is now empty.", taskLock);
|
||||
possesHolder.remove(taskLockPosse);
|
||||
}
|
||||
|
||||
if (possesHolder.isEmpty()) {
|
||||
intervalToPosses.remove(interval);
|
||||
}
|
||||
|
||||
if (intervalToPosses.isEmpty()) {
|
||||
dsRunning.remove(interval.getStart());
|
||||
locksForDatasource.remove(interval.getStart());
|
||||
}
|
||||
|
||||
if (running.get(dataSource).isEmpty()) {
|
||||
running.remove(dataSource);
|
||||
}
|
||||
|
@ -1227,39 +1203,7 @@ public class TaskLockbox
|
|||
try {
|
||||
try {
|
||||
log.info("Removing task[%s] from activeTasks", task.getId());
|
||||
try {
|
||||
// Clean upgrade segments table for entries associated with replacing task
|
||||
if (findLocksForTask(task).stream().anyMatch(lock -> lock.getType() == TaskLockType.REPLACE)) {
|
||||
final int upgradeSegmentsDeleted = metadataStorageCoordinator.deleteUpgradeSegmentsForTask(task.getId());
|
||||
log.info(
|
||||
"Deleted [%d] entries from upgradeSegments table for task[%s] with REPLACE locks.",
|
||||
upgradeSegmentsDeleted, task.getId()
|
||||
);
|
||||
}
|
||||
// Clean pending segments associated with the appending task
|
||||
if (task instanceof PendingSegmentAllocatingTask) {
|
||||
final String taskAllocatorId = ((PendingSegmentAllocatingTask) task).getTaskAllocatorId();
|
||||
if (activeAllocatorIdToTaskIds.containsKey(taskAllocatorId)) {
|
||||
final Set<String> idsInSameGroup = activeAllocatorIdToTaskIds.get(taskAllocatorId);
|
||||
idsInSameGroup.remove(task.getId());
|
||||
if (idsInSameGroup.isEmpty()) {
|
||||
final int pendingSegmentsDeleted
|
||||
= metadataStorageCoordinator.deletePendingSegmentsForTaskAllocatorId(
|
||||
task.getDataSource(),
|
||||
taskAllocatorId
|
||||
);
|
||||
log.info(
|
||||
"Deleted [%d] entries from pendingSegments table for pending segments group [%s] with APPEND locks.",
|
||||
pendingSegmentsDeleted, taskAllocatorId
|
||||
);
|
||||
}
|
||||
activeAllocatorIdToTaskIds.remove(taskAllocatorId);
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.warn(e, "Failure cleaning up upgradeSegments or pendingSegments tables.");
|
||||
}
|
||||
cleanupUpgradeAndPendingSegments(task);
|
||||
unlockAll(task);
|
||||
}
|
||||
finally {
|
||||
|
@ -1271,33 +1215,63 @@ public class TaskLockbox
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the currently-active lock posses for some task.
|
||||
*
|
||||
* @param task task for which to locate locks
|
||||
*/
|
||||
private List<TaskLockPosse> findLockPossesForTask(final Task task)
|
||||
@GuardedBy("giant")
|
||||
private void cleanupUpgradeAndPendingSegments(Task task)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
// Scan through all locks for this datasource
|
||||
final NavigableMap<DateTime, SortedMap<Interval, List<TaskLockPosse>>> dsRunning = running.get(task.getDataSource());
|
||||
if (dsRunning == null) {
|
||||
return ImmutableList.of();
|
||||
} else {
|
||||
return dsRunning.values().stream()
|
||||
.flatMap(map -> map.values().stream())
|
||||
.flatMap(Collection::stream)
|
||||
.filter(taskLockPosse -> taskLockPosse.containsTask(task))
|
||||
.collect(Collectors.toList());
|
||||
// Clean up upgrade segment entries associated with a REPLACE task
|
||||
if (findLocksForTask(task).stream().anyMatch(lock -> lock.getType() == TaskLockType.REPLACE)) {
|
||||
final int upgradeSegmentsDeleted = metadataStorageCoordinator.deleteUpgradeSegmentsForTask(task.getId());
|
||||
log.info(
|
||||
"Deleted [%d] entries from upgradeSegments table for task[%s] with REPLACE locks.",
|
||||
upgradeSegmentsDeleted, task.getId()
|
||||
);
|
||||
}
|
||||
|
||||
// Clean up pending segments associated with an APPEND task
|
||||
if (task instanceof PendingSegmentAllocatingTask) {
|
||||
final String taskAllocatorId = ((PendingSegmentAllocatingTask) task).getTaskAllocatorId();
|
||||
if (activeAllocatorIdToTaskIds.containsKey(taskAllocatorId)) {
|
||||
final Set<String> taskIdsForSameAllocator = activeAllocatorIdToTaskIds.get(taskAllocatorId);
|
||||
taskIdsForSameAllocator.remove(task.getId());
|
||||
|
||||
if (taskIdsForSameAllocator.isEmpty()) {
|
||||
final int pendingSegmentsDeleted = metadataStorageCoordinator
|
||||
.deletePendingSegmentsForTaskAllocatorId(task.getDataSource(), taskAllocatorId);
|
||||
log.info(
|
||||
"Deleted [%d] entries from pendingSegments table for taskAllocatorId[%s].",
|
||||
pendingSegmentsDeleted, taskAllocatorId
|
||||
);
|
||||
}
|
||||
activeAllocatorIdToTaskIds.remove(taskAllocatorId);
|
||||
}
|
||||
}
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
catch (Exception e) {
|
||||
log.warn(e, "Failure cleaning up upgradeSegments or pendingSegments tables.");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds all the lock posses for the given task.
|
||||
*/
|
||||
@GuardedBy("giant")
|
||||
private List<TaskLockPosse> findLockPossesForTask(final Task task)
|
||||
{
|
||||
// Scan through all locks for this datasource
|
||||
final NavigableMap<DateTime, SortedMap<Interval, List<TaskLockPosse>>> locksForDatasource
|
||||
= running.get(task.getDataSource());
|
||||
if (locksForDatasource == null) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
return locksForDatasource.values().stream()
|
||||
.flatMap(map -> map.values().stream())
|
||||
.flatMap(Collection::stream)
|
||||
.filter(taskLockPosse -> taskLockPosse.containsTask(task))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private List<TaskLockPosse> findLockPossesContainingInterval(final String dataSource, final Interval interval)
|
||||
{
|
||||
giant.lock();
|
||||
|
@ -1342,19 +1316,7 @@ public class TaskLockbox
|
|||
}
|
||||
|
||||
@VisibleForTesting
|
||||
List<TaskLockPosse> getOnlyTaskLockPosseContainingInterval(Task task, Interval interval)
|
||||
{
|
||||
giant.lock();
|
||||
try {
|
||||
return getOnlyTaskLockPosseContainingInterval(task, interval, Collections.emptySet());
|
||||
}
|
||||
finally {
|
||||
giant.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
List<TaskLockPosse> getOnlyTaskLockPosseContainingInterval(Task task, Interval interval, Set<Integer> partitionIds)
|
||||
Optional<TaskLockPosse> getOnlyTaskLockPosseContainingInterval(Task task, Interval interval)
|
||||
{
|
||||
giant.lock();
|
||||
try {
|
||||
|
@ -1364,35 +1326,20 @@ public class TaskLockbox
|
|||
.collect(Collectors.toList());
|
||||
|
||||
if (filteredPosses.isEmpty()) {
|
||||
throw new ISE("Cannot find locks for task[%s] and interval[%s]", task.getId(), interval);
|
||||
} else if (filteredPosses.size() > 1) {
|
||||
if (filteredPosses.stream()
|
||||
.anyMatch(posse -> posse.getTaskLock().getGranularity() == LockGranularity.TIME_CHUNK)) {
|
||||
throw new ISE(
|
||||
"There are multiple timeChunk lockPosses for task[%s] and interval[%s]?",
|
||||
task.getId(),
|
||||
interval
|
||||
);
|
||||
} else {
|
||||
final Map<Integer, TaskLockPosse> partitionIdsOfLocks = new HashMap<>();
|
||||
for (TaskLockPosse posse : filteredPosses) {
|
||||
final SegmentLock segmentLock = (SegmentLock) posse.getTaskLock();
|
||||
partitionIdsOfLocks.put(segmentLock.getPartitionId(), posse);
|
||||
}
|
||||
|
||||
if (partitionIds.stream().allMatch(partitionIdsOfLocks::containsKey)) {
|
||||
return partitionIds.stream().map(partitionIdsOfLocks::get).collect(Collectors.toList());
|
||||
} else {
|
||||
throw new ISE(
|
||||
"Task[%s] doesn't have locks for interval[%s] partitions[%]",
|
||||
task.getId(),
|
||||
interval,
|
||||
partitionIds.stream().filter(pid -> !partitionIdsOfLocks.containsKey(pid)).collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
}
|
||||
throw new ISE("Cannot find any lock for task[%s] and interval[%s]", task.getId(), interval);
|
||||
} else if (filteredPosses.size() == 1) {
|
||||
return Optional.of(filteredPosses.get(0));
|
||||
} else if (
|
||||
filteredPosses.stream().anyMatch(
|
||||
posse -> posse.taskLock.getGranularity() == LockGranularity.TIME_CHUNK
|
||||
)
|
||||
) {
|
||||
throw new ISE(
|
||||
"There are multiple timechunk lockPosses for task[%s] and interval[%s]",
|
||||
task.getId(), interval
|
||||
);
|
||||
} else {
|
||||
return filteredPosses;
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
finally {
|
||||
|
@ -1635,17 +1582,13 @@ public class TaskLockbox
|
|||
Preconditions.checkArgument(
|
||||
taskLock.getGroupId().equals(task.getGroupId()),
|
||||
"groupId[%s] of task[%s] is different from the existing lockPosse's groupId[%s]",
|
||||
task.getGroupId(),
|
||||
task.getId(),
|
||||
taskLock.getGroupId()
|
||||
task.getGroupId(), task.getId(), taskLock.getGroupId()
|
||||
);
|
||||
}
|
||||
Preconditions.checkArgument(
|
||||
taskLock.getNonNullPriority() == task.getPriority(),
|
||||
"priority[%s] of task[%s] is different from the existing lockPosse's priority[%s]",
|
||||
task.getPriority(),
|
||||
task.getId(),
|
||||
taskLock.getNonNullPriority()
|
||||
task.getPriority(), task.getId(), taskLock.getNonNullPriority()
|
||||
);
|
||||
return taskIds.add(task.getId());
|
||||
}
|
||||
|
@ -1724,12 +1667,6 @@ public class TaskLockbox
|
|||
return false;
|
||||
}
|
||||
|
||||
void forEachTask(Consumer<String> action)
|
||||
{
|
||||
Preconditions.checkNotNull(action, "action");
|
||||
taskIds.forEach(action);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
@ -1801,8 +1738,7 @@ public class TaskLockbox
|
|||
/**
|
||||
* Contains the task, request, lock and final result for a segment allocation.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static class SegmentAllocationHolder
|
||||
private static class SegmentAllocationHolder
|
||||
{
|
||||
final AllocationHolderList list;
|
||||
|
||||
|
|
|
@ -54,7 +54,6 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
|||
"datasource",
|
||||
Intervals.of("2020-01-01/P1D"),
|
||||
null,
|
||||
false,
|
||||
99,
|
||||
5,
|
||||
DateTimes.nowUtc()
|
||||
|
@ -65,7 +64,6 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
|||
Assert.assertEquals(taskQuery.getDataSource(), fromJson.getDataSource());
|
||||
Assert.assertEquals(taskQuery.getInterval(), fromJson.getInterval());
|
||||
Assert.assertNull(taskQuery.getVersions());
|
||||
Assert.assertEquals(taskQuery.getMarkAsUnused(), fromJson.isMarkAsUnused());
|
||||
Assert.assertEquals(taskQuery.getBatchSize(), Integer.valueOf(fromJson.getBatchSize()));
|
||||
Assert.assertEquals(taskQuery.getLimit(), fromJson.getLimit());
|
||||
Assert.assertEquals(taskQuery.getMaxUsedStatusLastUpdatedTime(), fromJson.getMaxUsedStatusLastUpdatedTime());
|
||||
|
@ -79,7 +77,6 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
|||
"datasource",
|
||||
Intervals.of("2020-01-01/P1D"),
|
||||
null,
|
||||
true,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
|
@ -90,7 +87,6 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
|||
Assert.assertEquals(taskQuery.getDataSource(), fromJson.getDataSource());
|
||||
Assert.assertEquals(taskQuery.getInterval(), fromJson.getInterval());
|
||||
Assert.assertNull(taskQuery.getVersions());
|
||||
Assert.assertEquals(taskQuery.getMarkAsUnused(), fromJson.isMarkAsUnused());
|
||||
Assert.assertEquals(100, fromJson.getBatchSize());
|
||||
Assert.assertNull(taskQuery.getLimit());
|
||||
Assert.assertNull(taskQuery.getMaxUsedStatusLastUpdatedTime());
|
||||
|
@ -105,7 +101,6 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
|||
Intervals.of("2020-01-01/P1D"),
|
||||
null,
|
||||
null,
|
||||
true,
|
||||
99,
|
||||
null,
|
||||
null
|
||||
|
@ -119,7 +114,6 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
|||
Assert.assertEquals(task.getDataSource(), taskQuery.getDataSource());
|
||||
Assert.assertEquals(task.getInterval(), taskQuery.getInterval());
|
||||
Assert.assertNull(taskQuery.getVersions());
|
||||
Assert.assertEquals(task.isMarkAsUnused(), taskQuery.getMarkAsUnused());
|
||||
Assert.assertEquals(Integer.valueOf(task.getBatchSize()), taskQuery.getBatchSize());
|
||||
Assert.assertNull(taskQuery.getLimit());
|
||||
Assert.assertNull(taskQuery.getMaxUsedStatusLastUpdatedTime());
|
||||
|
@ -134,7 +128,6 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
|||
Intervals.of("2020-01-01/P1D"),
|
||||
ImmutableList.of("v1", "v2"),
|
||||
null,
|
||||
null,
|
||||
99,
|
||||
100,
|
||||
DateTimes.nowUtc()
|
||||
|
@ -148,7 +141,6 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
|||
Assert.assertEquals(task.getDataSource(), taskQuery.getDataSource());
|
||||
Assert.assertEquals(task.getInterval(), taskQuery.getInterval());
|
||||
Assert.assertEquals(task.getVersions(), taskQuery.getVersions());
|
||||
Assert.assertNull(taskQuery.getMarkAsUnused());
|
||||
Assert.assertEquals(Integer.valueOf(task.getBatchSize()), taskQuery.getBatchSize());
|
||||
Assert.assertEquals(task.getLimit(), taskQuery.getLimit());
|
||||
Assert.assertEquals(task.getMaxUsedStatusLastUpdatedTime(), taskQuery.getMaxUsedStatusLastUpdatedTime());
|
||||
|
|
|
@ -27,6 +27,10 @@ import org.apache.druid.error.DruidExceptionMatcher;
|
|||
import org.apache.druid.indexer.TaskState;
|
||||
import org.apache.druid.indexer.report.KillTaskReport;
|
||||
import org.apache.druid.indexer.report.TaskReport;
|
||||
import org.apache.druid.indexing.common.SegmentLock;
|
||||
import org.apache.druid.indexing.common.TaskLockType;
|
||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||
import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
|
||||
import org.apache.druid.indexing.overlord.Segments;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
@ -35,6 +39,8 @@ import org.apache.druid.java.util.common.JodaUtils;
|
|||
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.easymock.Capture;
|
||||
import org.easymock.EasyMock;
|
||||
import org.hamcrest.MatcherAssert;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -116,52 +122,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
).containsExactlyInAnyOrder(segment1, segment4);
|
||||
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(1, 2, 0),
|
||||
getReportedStats()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testKillWithMarkUnused() throws Exception
|
||||
{
|
||||
final Set<DataSegment> segments = ImmutableSet.of(segment1, segment2, segment3, segment4);
|
||||
final Set<DataSegment> announced = getMetadataStorageCoordinator().commitSegments(segments, null);
|
||||
Assert.assertEquals(segments, announced);
|
||||
|
||||
Assert.assertTrue(
|
||||
getSegmentsMetadataManager().markSegmentAsUnused(
|
||||
segment2.getId()
|
||||
)
|
||||
);
|
||||
|
||||
final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Intervals.of("2019-03-01/2019-04-01"))
|
||||
.markAsUnused(true)
|
||||
.build();
|
||||
|
||||
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode());
|
||||
|
||||
final List<DataSegment> observedUnusedSegments =
|
||||
getMetadataStorageCoordinator().retrieveUnusedSegmentsForInterval(
|
||||
DATA_SOURCE,
|
||||
Intervals.of("2019/2020"),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
||||
Assert.assertEquals(ImmutableList.of(segment2), observedUnusedSegments);
|
||||
Assertions.assertThat(
|
||||
getMetadataStorageCoordinator().retrieveUsedSegmentsForInterval(
|
||||
DATA_SOURCE,
|
||||
Intervals.of("2019/2020"),
|
||||
Segments.ONLY_VISIBLE
|
||||
)
|
||||
).containsExactlyInAnyOrder(segment1, segment4);
|
||||
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(1, 2, 1),
|
||||
new KillTaskReport.Stats(1, 2),
|
||||
getReportedStats()
|
||||
);
|
||||
}
|
||||
|
@ -199,7 +160,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode());
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(4, 3, 0),
|
||||
new KillTaskReport.Stats(4, 3),
|
||||
getReportedStats()
|
||||
);
|
||||
|
||||
|
@ -247,7 +208,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode());
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(0, 1, 0),
|
||||
new KillTaskReport.Stats(0, 1),
|
||||
getReportedStats()
|
||||
);
|
||||
|
||||
|
@ -296,7 +257,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode());
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(2, 1, 0),
|
||||
new KillTaskReport.Stats(2, 1),
|
||||
getReportedStats()
|
||||
);
|
||||
|
||||
|
@ -345,7 +306,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode());
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(0, 1, 0),
|
||||
new KillTaskReport.Stats(0, 1),
|
||||
getReportedStats()
|
||||
);
|
||||
|
||||
|
@ -398,7 +359,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode());
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(0, 1, 0),
|
||||
new KillTaskReport.Stats(0, 1),
|
||||
getReportedStats()
|
||||
);
|
||||
|
||||
|
@ -419,7 +380,6 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Intervals.of("2019-03-01/2019-04-01"))
|
||||
.markAsUnused(true)
|
||||
.build();
|
||||
Assert.assertTrue(task.getInputSourceResources().isEmpty());
|
||||
}
|
||||
|
@ -430,7 +390,6 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Intervals.of("2019-03-01/2019-04-01"))
|
||||
.markAsUnused(true)
|
||||
.build();
|
||||
Assert.assertEquals(LookupLoadingSpec.Mode.NONE, task.getLookupLoadingSpec().getMode());
|
||||
}
|
||||
|
@ -472,7 +431,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(Collections.emptyList(), observedUnusedSegments);
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(4, 4, 0),
|
||||
new KillTaskReport.Stats(4, 4),
|
||||
getReportedStats()
|
||||
);
|
||||
}
|
||||
|
@ -542,7 +501,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(ImmutableList.of(), observedUnusedSegments);
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(3, 4, 0),
|
||||
new KillTaskReport.Stats(3, 4),
|
||||
getReportedStats()
|
||||
);
|
||||
}
|
||||
|
@ -628,7 +587,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(ImmutableList.of(segment3), observedUnusedSegments);
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(2, 3, 0),
|
||||
new KillTaskReport.Stats(2, 3),
|
||||
getReportedStats()
|
||||
);
|
||||
|
||||
|
@ -652,7 +611,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(ImmutableList.of(), observedUnusedSegments2);
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(1, 2, 0),
|
||||
new KillTaskReport.Stats(1, 2),
|
||||
getReportedStats()
|
||||
);
|
||||
}
|
||||
|
@ -732,7 +691,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(ImmutableList.of(segment2, segment3), observedUnusedSegments1);
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(2, 3, 0),
|
||||
new KillTaskReport.Stats(2, 3),
|
||||
getReportedStats()
|
||||
);
|
||||
|
||||
|
@ -756,7 +715,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(ImmutableList.of(), observedUnusedSegments2);
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(2, 3, 0),
|
||||
new KillTaskReport.Stats(2, 3),
|
||||
getReportedStats()
|
||||
);
|
||||
}
|
||||
|
@ -813,7 +772,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task1).get().getStatusCode());
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(2, 3, 0),
|
||||
new KillTaskReport.Stats(2, 3),
|
||||
getReportedStats()
|
||||
);
|
||||
|
||||
|
@ -838,7 +797,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task2).get().getStatusCode());
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(1, 2, 0),
|
||||
new KillTaskReport.Stats(1, 2),
|
||||
getReportedStats()
|
||||
);
|
||||
|
||||
|
@ -861,10 +820,17 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(segments, announced);
|
||||
|
||||
for (DataSegment segment : segments) {
|
||||
Assert.assertTrue(
|
||||
getSegmentsMetadataManager().markSegmentAsUnused(
|
||||
segment.getId()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Intervals.of("2018-01-01/2020-01-01"))
|
||||
.markAsUnused(true)
|
||||
.batchSize(3)
|
||||
.build();
|
||||
|
||||
|
@ -880,7 +846,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
|
||||
Assert.assertEquals(Collections.emptyList(), observedUnusedSegments);
|
||||
Assert.assertEquals(
|
||||
new KillTaskReport.Stats(4, 3, 4),
|
||||
new KillTaskReport.Stats(4, 3),
|
||||
getReportedStats()
|
||||
);
|
||||
}
|
||||
|
@ -989,45 +955,6 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidLimitWithMarkAsUnused()
|
||||
{
|
||||
MatcherAssert.assertThat(
|
||||
Assert.assertThrows(
|
||||
DruidException.class,
|
||||
() -> new KillUnusedSegmentsTaskBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Intervals.of("2018-01-01/2020-01-01"))
|
||||
.markAsUnused(true)
|
||||
.batchSize(10)
|
||||
.limit(10)
|
||||
.build()
|
||||
),
|
||||
DruidExceptionMatcher.invalidInput().expectMessageIs(
|
||||
"limit[10] cannot be provided when markAsUnused is enabled."
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidVersionsWithMarkAsUnused()
|
||||
{
|
||||
MatcherAssert.assertThat(
|
||||
Assert.assertThrows(
|
||||
DruidException.class,
|
||||
() -> new KillUnusedSegmentsTaskBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Intervals.of("2018-01-01/2020-01-01"))
|
||||
.markAsUnused(true)
|
||||
.versions(ImmutableList.of("foo"))
|
||||
.build()
|
||||
),
|
||||
DruidExceptionMatcher.invalidInput().expectMessageIs(
|
||||
"versions[[foo]] cannot be provided when markAsUnused is enabled."
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetNumTotalBatchesWithBatchSizeSmallerThanLimit()
|
||||
{
|
||||
|
@ -1046,7 +973,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
{
|
||||
final String taskId = "test_serde_task";
|
||||
|
||||
final KillTaskReport.Stats stats = new KillTaskReport.Stats(1, 2, 3);
|
||||
final KillTaskReport.Stats stats = new KillTaskReport.Stats(1, 2);
|
||||
KillTaskReport report = new KillTaskReport(taskId, stats);
|
||||
|
||||
String json = getObjectMapper().writeValueAsString(report);
|
||||
|
@ -1059,6 +986,150 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
Assert.assertEquals(stats, deserializedKillReport.getPayload());
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testIsReadyWithExclusiveLock() throws Exception
|
||||
{
|
||||
final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Intervals.of("2019-03-01/2019-04-01"))
|
||||
.build();
|
||||
|
||||
Capture<TimeChunkLockTryAcquireAction> acquireActionCapture = Capture.newInstance();
|
||||
|
||||
TaskActionClient taskActionClient = EasyMock.createMock(TaskActionClient.class);
|
||||
EasyMock.expect(taskActionClient.submit(EasyMock.capture(acquireActionCapture))).andReturn(
|
||||
new SegmentLock(
|
||||
TaskLockType.EXCLUSIVE,
|
||||
"groupId",
|
||||
"datasource",
|
||||
task.getInterval(),
|
||||
"v1",
|
||||
0,
|
||||
0
|
||||
)
|
||||
);
|
||||
EasyMock.replay(taskActionClient);
|
||||
|
||||
Assert.assertTrue(task.isReady(taskActionClient));
|
||||
|
||||
Assert.assertEquals(TaskLockType.EXCLUSIVE, acquireActionCapture.getValue().getType());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsReadyWithReplaceLock() throws Exception
|
||||
{
|
||||
final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, Boolean.TRUE))
|
||||
.interval(Intervals.of("2019-03-01/2019-04-01"))
|
||||
.build();
|
||||
|
||||
Capture<TimeChunkLockTryAcquireAction> acquireActionCapture = Capture.newInstance();
|
||||
|
||||
TaskActionClient taskActionClient = EasyMock.createMock(TaskActionClient.class);
|
||||
EasyMock.expect(taskActionClient.submit(EasyMock.capture(acquireActionCapture)))
|
||||
.andReturn(
|
||||
new SegmentLock(
|
||||
TaskLockType.REPLACE,
|
||||
"groupId",
|
||||
"datasource",
|
||||
task.getInterval(),
|
||||
"v1",
|
||||
0,
|
||||
0
|
||||
)
|
||||
);
|
||||
EasyMock.replay(taskActionClient);
|
||||
|
||||
Assert.assertTrue(task.isReady(taskActionClient));
|
||||
|
||||
Assert.assertEquals(TaskLockType.REPLACE, acquireActionCapture.getValue().getType());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsReadyWithContextAppendLock() throws Exception
|
||||
{
|
||||
final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.context(ImmutableMap.of(Tasks.TASK_LOCK_TYPE, TaskLockType.APPEND))
|
||||
.interval(Intervals.of("2019-03-01/2019-04-01"))
|
||||
.build();
|
||||
|
||||
Capture<TimeChunkLockTryAcquireAction> acquireActionCapture = Capture.newInstance();
|
||||
|
||||
TaskActionClient taskActionClient = EasyMock.createMock(TaskActionClient.class);
|
||||
EasyMock.expect(taskActionClient.submit(EasyMock.capture(acquireActionCapture)))
|
||||
.andReturn(
|
||||
new SegmentLock(
|
||||
TaskLockType.APPEND,
|
||||
"groupId",
|
||||
"datasource",
|
||||
task.getInterval(),
|
||||
"v1",
|
||||
0,
|
||||
0
|
||||
)
|
||||
);
|
||||
EasyMock.replay(taskActionClient);
|
||||
|
||||
Assert.assertTrue(task.isReady(taskActionClient));
|
||||
|
||||
Assert.assertEquals(TaskLockType.APPEND, acquireActionCapture.getValue().getType());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsReadyWithConcurrentLockHasPrecedenceOverContextLock() throws Exception
|
||||
{
|
||||
final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, Boolean.TRUE, Tasks.TASK_LOCK_TYPE, TaskLockType.APPEND))
|
||||
.interval(Intervals.of("2019-03-01/2019-04-01"))
|
||||
.build();
|
||||
|
||||
Capture<TimeChunkLockTryAcquireAction> acquireActionCapture = Capture.newInstance();
|
||||
|
||||
TaskActionClient taskActionClient = EasyMock.createMock(TaskActionClient.class);
|
||||
EasyMock.expect(taskActionClient.submit(EasyMock.capture(acquireActionCapture)))
|
||||
.andReturn(
|
||||
new SegmentLock(
|
||||
TaskLockType.REPLACE,
|
||||
"groupId",
|
||||
"datasource",
|
||||
task.getInterval(),
|
||||
"v1",
|
||||
0,
|
||||
0
|
||||
)
|
||||
);
|
||||
EasyMock.replay(taskActionClient);
|
||||
|
||||
Assert.assertTrue(task.isReady(taskActionClient));
|
||||
|
||||
Assert.assertEquals(TaskLockType.REPLACE, acquireActionCapture.getValue().getType());
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testIsReadyReturnsNullLock() throws Exception
|
||||
{
|
||||
final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, Boolean.TRUE))
|
||||
.interval(Intervals.of("2019-03-01/2019-04-01"))
|
||||
.build();
|
||||
|
||||
Capture<TimeChunkLockTryAcquireAction> acquireActionCapture = Capture.newInstance();
|
||||
|
||||
TaskActionClient taskActionClient = EasyMock.createMock(TaskActionClient.class);
|
||||
EasyMock.expect(taskActionClient.submit(EasyMock.capture(acquireActionCapture))).andReturn(null);
|
||||
EasyMock.replay(taskActionClient);
|
||||
|
||||
Assert.assertFalse(task.isReady(taskActionClient));
|
||||
|
||||
Assert.assertEquals(TaskLockType.REPLACE, acquireActionCapture.getValue().getType());
|
||||
}
|
||||
|
||||
private static class KillUnusedSegmentsTaskBuilder
|
||||
{
|
||||
private String id;
|
||||
|
@ -1066,7 +1137,6 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
private Interval interval;
|
||||
private List<String> versions;
|
||||
private Map<String, Object> context;
|
||||
private Boolean markAsUnused;
|
||||
private Integer batchSize;
|
||||
private Integer limit;
|
||||
private DateTime maxUsedStatusLastUpdatedTime;
|
||||
|
@ -1101,12 +1171,6 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
return this;
|
||||
}
|
||||
|
||||
public KillUnusedSegmentsTaskBuilder markAsUnused(Boolean markAsUnused)
|
||||
{
|
||||
this.markAsUnused = markAsUnused;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KillUnusedSegmentsTaskBuilder batchSize(Integer batchSize)
|
||||
{
|
||||
this.batchSize = batchSize;
|
||||
|
@ -1133,7 +1197,6 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
|||
interval,
|
||||
versions,
|
||||
context,
|
||||
markAsUnused,
|
||||
batchSize,
|
||||
limit,
|
||||
maxUsedStatusLastUpdatedTime
|
||||
|
|
|
@ -31,7 +31,7 @@ public class PushedSegmentsReportTest
|
|||
{
|
||||
TaskReport.ReportMap map1 = new TaskReport.ReportMap();
|
||||
TaskReport.ReportMap map2 = new TaskReport.ReportMap();
|
||||
map2.put("killTaskReport", new KillTaskReport("taskId", new KillTaskReport.Stats(1, 2, 3)));
|
||||
map2.put("killTaskReport", new KillTaskReport("taskId", new KillTaskReport.Stats(1, 2)));
|
||||
|
||||
EqualsVerifier.forClass(PushedSegmentsReport.class)
|
||||
.usingGetClass()
|
||||
|
|
|
@ -939,7 +939,6 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
|||
Intervals.of("2011-04-01/P4D"),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
|
@ -1038,7 +1037,6 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
|||
Intervals.of("2011-04-01/P4D"),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
null,
|
||||
maxSegmentsToKill,
|
||||
null
|
||||
|
|
|
@ -73,7 +73,6 @@ import org.junit.Assert;
|
|||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -83,6 +82,7 @@ import java.util.HashMap;
|
|||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
@ -91,9 +91,6 @@ public class TaskLockboxTest
|
|||
@Rule
|
||||
public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule();
|
||||
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
private ObjectMapper objectMapper;
|
||||
private TaskStorage taskStorage;
|
||||
private IndexerMetadataStorageCoordinator metadataStorageCoordinator;
|
||||
|
@ -105,9 +102,6 @@ public class TaskLockboxTest
|
|||
private final int MEDIUM_PRIORITY = 10;
|
||||
private final int LOW_PRIORITY = 5;
|
||||
|
||||
@Rule
|
||||
public final ExpectedException exception = ExpectedException.none();
|
||||
|
||||
@Before
|
||||
public void setup()
|
||||
{
|
||||
|
@ -186,14 +180,16 @@ public class TaskLockboxTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testLockAfterTaskComplete() throws InterruptedException
|
||||
public void testLockAfterTaskComplete()
|
||||
{
|
||||
Task task = NoopTask.create();
|
||||
exception.expect(ISE.class);
|
||||
exception.expectMessage("Unable to grant lock to inactive Task");
|
||||
final Task task = NoopTask.create();
|
||||
lockbox.add(task);
|
||||
lockbox.remove(task);
|
||||
acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-02"));
|
||||
ISE exception = Assert.assertThrows(
|
||||
ISE.class,
|
||||
() -> acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-02"))
|
||||
);
|
||||
Assert.assertTrue(exception.getMessage().contains("Unable to grant lock to inactive Task"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -311,12 +307,15 @@ public class TaskLockboxTest
|
|||
@Test
|
||||
public void testTryLockAfterTaskComplete()
|
||||
{
|
||||
Task task = NoopTask.create();
|
||||
exception.expect(ISE.class);
|
||||
exception.expectMessage("Unable to grant lock to inactive Task");
|
||||
final Task task = NoopTask.create();
|
||||
lockbox.add(task);
|
||||
lockbox.remove(task);
|
||||
Assert.assertFalse(tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-02")).isOk());
|
||||
|
||||
ISE exception = Assert.assertThrows(
|
||||
ISE.class,
|
||||
() -> tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-02"))
|
||||
);
|
||||
Assert.assertTrue(exception.getMessage().contains("Unable to grant lock to inactive Task"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -759,23 +758,23 @@ public class TaskLockboxTest
|
|||
).isOk()
|
||||
);
|
||||
|
||||
final List<TaskLockPosse> highLockPosses = lockbox.getOnlyTaskLockPosseContainingInterval(
|
||||
final Optional<TaskLockPosse> highLockPosse = lockbox.getOnlyTaskLockPosseContainingInterval(
|
||||
highPriorityTask,
|
||||
Intervals.of("2018-12-16T09:00:00/2018-12-16T09:30:00")
|
||||
);
|
||||
|
||||
Assert.assertEquals(1, highLockPosses.size());
|
||||
Assert.assertTrue(highLockPosses.get(0).containsTask(highPriorityTask));
|
||||
Assert.assertFalse(highLockPosses.get(0).getTaskLock().isRevoked());
|
||||
Assert.assertTrue(highLockPosse.isPresent());
|
||||
Assert.assertTrue(highLockPosse.get().containsTask(highPriorityTask));
|
||||
Assert.assertFalse(highLockPosse.get().getTaskLock().isRevoked());
|
||||
|
||||
final List<TaskLockPosse> lowLockPosses = lockbox.getOnlyTaskLockPosseContainingInterval(
|
||||
final Optional<TaskLockPosse> lowLockPosse = lockbox.getOnlyTaskLockPosseContainingInterval(
|
||||
lowPriorityTask,
|
||||
Intervals.of("2018-12-16T09:00:00/2018-12-16T10:00:00")
|
||||
);
|
||||
|
||||
Assert.assertEquals(1, lowLockPosses.size());
|
||||
Assert.assertTrue(lowLockPosses.get(0).containsTask(lowPriorityTask));
|
||||
Assert.assertTrue(lowLockPosses.get(0).getTaskLock().isRevoked());
|
||||
Assert.assertTrue(lowLockPosse.isPresent());
|
||||
Assert.assertTrue(lowLockPosse.get().containsTask(lowPriorityTask));
|
||||
Assert.assertTrue(lowLockPosse.get().getTaskLock().isRevoked());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -926,7 +926,7 @@ public class OverlordResourceTest
|
|||
auditManager
|
||||
);
|
||||
|
||||
Task task = new KillUnusedSegmentsTask("kill_all", "allow", Intervals.ETERNITY, null, null, false, 10, null, null);
|
||||
Task task = new KillUnusedSegmentsTask("kill_all", "allow", Intervals.ETERNITY, null, null, 10, null, null);
|
||||
overlordResource.taskPost(task, req);
|
||||
|
||||
Assert.assertTrue(auditEntryCapture.hasCaptured());
|
||||
|
|
|
@ -42,13 +42,20 @@ an explanation.
|
|||
./it.sh image
|
||||
```
|
||||
|
||||
Note: If you are running it on Apple Silicon processors, you would also need to uncomment all occurrences of `platform: linux/x86_64` in [dependencies.yaml](https://github.com/apache/druid/blob/master/integration-tests-ex/cases/cluster/Common/dependencies.yaml).
|
||||
|
||||
### Run an IT from the Command Line
|
||||
|
||||
```bash
|
||||
./it.sh test <category>
|
||||
```
|
||||
|
||||
Where `<category>` is one of the test categories.
|
||||
Where `<category>` is one of the test categories. You can see the list of test categories at [`src/test/java/org/apache/druid/testsEx/categories`](https://github.com/apache/druid/tree/master/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories). The corresponding test classes are also annotated with `@Category` like `@Category(HighAvailability.class)`.
|
||||
|
||||
For example, a sample command for running IT for `@Category(HighAvailability.class)` would be:
|
||||
```bash
|
||||
./it.sh test HighAvailability
|
||||
```
|
||||
|
||||
### Run an IT from the IDE
|
||||
|
||||
|
|
|
@ -297,7 +297,6 @@
|
|||
<dependency>
|
||||
<groupId>org.apache.curator</groupId>
|
||||
<artifactId>curator-client</artifactId>
|
||||
<version>5.5.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.cloud</groupId>
|
||||
|
|
|
@ -57,7 +57,7 @@ export APACHE_ARCHIVE_MIRROR_HOST=https://example.com/remote-generic-repo
|
|||
This section describes how to start integration tests against Docker containers which will be brought up automatically by following commands.
|
||||
If you want to build Docker images and run tests separately, see the next section.
|
||||
|
||||
To run all tests from a test group using Docker and Maven run the following command:
|
||||
To run all tests from a test group using Docker and Maven run the following command from `integration-tests` directory:
|
||||
|
||||
```bash
|
||||
mvn verify -P integration-tests -Dgroups=<test_group>
|
||||
|
@ -68,7 +68,7 @@ The list of test groups can be found at
|
|||
|
||||
### Run a single test
|
||||
|
||||
To run only a single test using Maven:
|
||||
To run only a single test using Maven, run the following command from `integration-tests` directory:
|
||||
|
||||
```bash
|
||||
mvn verify -P integration-tests -Dgroups=<test_group> -Dit.test=<test_name>
|
||||
|
|
|
@ -1655,7 +1655,7 @@ name: Apache Curator
|
|||
license_category: binary
|
||||
module: java-core
|
||||
license_name: Apache License version 2.0
|
||||
version: 5.5.0
|
||||
version: 5.3.0
|
||||
libraries:
|
||||
- org.apache.curator: curator-client
|
||||
- org.apache.curator: curator-framework
|
||||
|
@ -5094,7 +5094,7 @@ license_category: binary
|
|||
module: web-console
|
||||
license_name: Apache License version 2.0
|
||||
copyright: Imply Data
|
||||
version: 0.22.11
|
||||
version: 0.22.13
|
||||
|
||||
---
|
||||
|
||||
|
|
2
pom.xml
2
pom.xml
|
@ -75,7 +75,7 @@
|
|||
<java.version>8</java.version>
|
||||
<project.build.resourceEncoding>UTF-8</project.build.resourceEncoding>
|
||||
<aether.version>0.9.0.M2</aether.version>
|
||||
<apache.curator.version>5.5.0</apache.curator.version>
|
||||
<apache.curator.version>5.3.0</apache.curator.version>
|
||||
<apache.kafka.version>3.6.1</apache.kafka.version>
|
||||
<!-- when updating apache ranger, verify the usage of aws-bundle-sdk vs aws-logs-sdk
|
||||
and update as needed in extensions-core/druid-ranger-security/pm.xml -->
|
||||
|
|
|
@ -89,10 +89,57 @@ public class ComplexFieldReader implements FieldReader
|
|||
return memory.getByte(position) == ComplexFieldWriter.NULL_BYTE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isComparable()
|
||||
/**
|
||||
* Alternative interface to read the field from the byte array without creating a selector and field pointer. It is much
|
||||
* faster than wrapping the byte array in Memory for reading.
|
||||
*/
|
||||
@Nullable
|
||||
public static Object readFieldFromByteArray(
|
||||
final ComplexMetricSerde serde,
|
||||
final byte[] bytes,
|
||||
final int position
|
||||
)
|
||||
{
|
||||
return false;
|
||||
final byte nullByte = bytes[position];
|
||||
|
||||
if (nullByte == ComplexFieldWriter.NULL_BYTE) {
|
||||
return null;
|
||||
} else if (nullByte == ComplexFieldWriter.NOT_NULL_BYTE) {
|
||||
// Reads length in little-endian format
|
||||
int length;
|
||||
length = (bytes[position + 4] & 0xFF) << 24;
|
||||
length |= (bytes[position + 3] & 0xFF) << 16;
|
||||
length |= (bytes[position + 2] & 0xFF) << 8;
|
||||
length |= (bytes[position + 1] & 0xFF);
|
||||
return serde.fromBytes(bytes, position + ComplexFieldWriter.HEADER_SIZE, length);
|
||||
} else {
|
||||
throw new ISE("Unexpected null byte [%s]", nullByte);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Alternative interface to read the field from the memory without creating a selector and field pointer
|
||||
*/
|
||||
@Nullable
|
||||
public static Object readFieldFromMemory(
|
||||
final ComplexMetricSerde serde,
|
||||
final Memory memory,
|
||||
final long position
|
||||
)
|
||||
{
|
||||
final byte nullByte = memory.getByte(position);
|
||||
|
||||
if (nullByte == ComplexFieldWriter.NULL_BYTE) {
|
||||
return null;
|
||||
} else if (nullByte == ComplexFieldWriter.NOT_NULL_BYTE) {
|
||||
final int length = memory.getInt(position + Byte.BYTES);
|
||||
final byte[] bytes = new byte[length];
|
||||
memory.getByteArray(position + ComplexFieldWriter.HEADER_SIZE, bytes, 0, length);
|
||||
|
||||
return serde.fromBytes(bytes, 0, length);
|
||||
} else {
|
||||
throw new ISE("Unexpected null byte [%s]", nullByte);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -115,21 +162,8 @@ public class ComplexFieldReader implements FieldReader
|
|||
@Override
|
||||
public T getObject()
|
||||
{
|
||||
final long fieldPosition = fieldPointer.position();
|
||||
final byte nullByte = memory.getByte(fieldPosition);
|
||||
|
||||
if (nullByte == ComplexFieldWriter.NULL_BYTE) {
|
||||
return null;
|
||||
} else if (nullByte == ComplexFieldWriter.NOT_NULL_BYTE) {
|
||||
final int length = memory.getInt(fieldPosition + Byte.BYTES);
|
||||
final byte[] bytes = new byte[length];
|
||||
memory.getByteArray(fieldPosition + ComplexFieldWriter.HEADER_SIZE, bytes, 0, length);
|
||||
|
||||
//noinspection unchecked
|
||||
return (T) serde.fromBytes(bytes, 0, length);
|
||||
} else {
|
||||
throw new ISE("Unexpected null byte [%s]", nullByte);
|
||||
}
|
||||
//noinspection unchecked
|
||||
return (T) readFieldFromMemory(serde, memory, fieldPointer.position());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -56,9 +56,4 @@ public interface FieldReader
|
|||
* Whether the provided memory position points to a null value.
|
||||
*/
|
||||
boolean isNull(Memory memory, long position);
|
||||
|
||||
/**
|
||||
* Whether this field is comparable. Comparable fields can be compared as unsigned bytes.
|
||||
*/
|
||||
boolean isComparable();
|
||||
}
|
||||
|
|
|
@ -50,10 +50,4 @@ public abstract class NumericArrayFieldReader implements FieldReader
|
|||
final byte firstByte = memory.getByte(position);
|
||||
return firstByte == NumericArrayFieldWriter.NULL_ROW;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isComparable()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -76,13 +76,6 @@ public abstract class NumericFieldReader implements FieldReader
|
|||
return memory.getByte(position) == nullIndicatorByte;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean isComparable()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a column value selector for the element written at fieldPointer's position in the memory.
|
||||
* The nullilty check is handled by the nullIndicatorByte
|
||||
|
|
|
@ -123,12 +123,6 @@ public class StringFieldReader implements FieldReader
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isComparable()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Selector that reads a value from a location pointed to by {@link ReadableFieldPointer}.
|
||||
*/
|
||||
|
|
|
@ -19,11 +19,14 @@
|
|||
|
||||
package org.apache.druid.frame.key;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.primitives.Ints;
|
||||
import it.unimi.dsi.fastutil.ints.IntArrayList;
|
||||
import it.unimi.dsi.fastutil.ints.IntList;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.frame.read.FrameReaderUtils;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.serde.ComplexMetricSerde;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
|
@ -38,23 +41,94 @@ import java.util.Objects;
|
|||
*/
|
||||
public class ByteRowKeyComparator implements Comparator<byte[]>
|
||||
{
|
||||
/**
|
||||
* Key columns to compare on
|
||||
*/
|
||||
private final List<KeyColumn> keyColumns;
|
||||
|
||||
/**
|
||||
* Starting position of the first field in the row
|
||||
*/
|
||||
private final int firstFieldPosition;
|
||||
private final int[] ascDescRunLengths;
|
||||
|
||||
/**
|
||||
* Run lengths created for comparing the key columns
|
||||
*/
|
||||
private final RowKeyComparisonRunLengths rowKeyComparisonRunLengths;
|
||||
|
||||
/**
|
||||
* Pre-computed array of ComplexMetricSerde corresponding to the computed run-lengths. If the run length entry is
|
||||
* byte-comparable, the corresponding serde is null, and if it's not byte comparable, the corresponding serde isn't null
|
||||
* (since only complex columns are not byte comparable)
|
||||
*/
|
||||
private final ComplexMetricSerde[] complexMetricSerdes;
|
||||
|
||||
/**
|
||||
* Pre-computed array of the column types corresponding to the computed run-lengths. If the run length entry is
|
||||
* byte-comparable, the corresponding column type is null because we don't need the column type to compare.
|
||||
* If it's not byte comparable, the corresponding column type isn't null so that we have access to the comparator
|
||||
* for the type
|
||||
*/
|
||||
private final ColumnType[] columnTypes;
|
||||
|
||||
private ByteRowKeyComparator(
|
||||
final int firstFieldPosition,
|
||||
final int[] ascDescRunLengths
|
||||
final List<KeyColumn> keyColumns,
|
||||
final RowKeyComparisonRunLengths rowKeyComparisonRunLengths,
|
||||
final ComplexMetricSerde[] complexMetricSerdes,
|
||||
final ColumnType[] columnTypes
|
||||
)
|
||||
{
|
||||
this.firstFieldPosition = firstFieldPosition;
|
||||
this.ascDescRunLengths = ascDescRunLengths;
|
||||
this.keyColumns = keyColumns;
|
||||
this.firstFieldPosition = computeFirstFieldPosition(keyColumns.size());
|
||||
this.rowKeyComparisonRunLengths = rowKeyComparisonRunLengths;
|
||||
this.complexMetricSerdes = complexMetricSerdes;
|
||||
this.columnTypes = columnTypes;
|
||||
}
|
||||
|
||||
public static ByteRowKeyComparator create(final List<KeyColumn> keyColumns)
|
||||
public static ByteRowKeyComparator create(final List<KeyColumn> keyColumns, final RowSignature rowSignature)
|
||||
{
|
||||
final RowKeyComparisonRunLengths rowKeyComparisonRunLengths = RowKeyComparisonRunLengths.create(
|
||||
keyColumns,
|
||||
rowSignature
|
||||
);
|
||||
final RunLengthEntry[] runLengthEntries = rowKeyComparisonRunLengths.getRunLengthEntries();
|
||||
final ComplexMetricSerde[] complexMetricSerdes = new ComplexMetricSerde[runLengthEntries.length];
|
||||
final ColumnType[] columnTypes = new ColumnType[runLengthEntries.length];
|
||||
|
||||
int fieldsSeenSoFar = 0;
|
||||
|
||||
for (int i = 0; i < runLengthEntries.length; ++i) {
|
||||
if (runLengthEntries[i].isByteComparable()) {
|
||||
complexMetricSerdes[i] = null;
|
||||
columnTypes[i] = null;
|
||||
} else {
|
||||
final String columnName = keyColumns.get(fieldsSeenSoFar).columnName();
|
||||
final ColumnType columnType = rowSignature.getColumnType(columnName).orElse(null);
|
||||
if (columnType == null) {
|
||||
throw DruidException.defensive("Column type required for column [%s] for comparison", columnName);
|
||||
}
|
||||
final String complexTypeName = columnType.getComplexTypeName();
|
||||
if (complexTypeName == null) {
|
||||
throw DruidException.defensive("Expected complex type name for column [%s] for comparison", columnName);
|
||||
}
|
||||
|
||||
complexMetricSerdes[i] = Preconditions.checkNotNull(
|
||||
ComplexMetrics.getSerdeForType(complexTypeName),
|
||||
"Cannot find serde for column [%s] with type [%s]",
|
||||
columnName,
|
||||
complexTypeName
|
||||
);
|
||||
columnTypes[i] = columnType;
|
||||
}
|
||||
|
||||
fieldsSeenSoFar += runLengthEntries[i].getRunLength();
|
||||
}
|
||||
|
||||
return new ByteRowKeyComparator(
|
||||
computeFirstFieldPosition(keyColumns.size()),
|
||||
computeAscDescRunLengths(keyColumns)
|
||||
keyColumns,
|
||||
RowKeyComparisonRunLengths.create(keyColumns, rowSignature),
|
||||
complexMetricSerdes,
|
||||
columnTypes
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -68,83 +142,66 @@ public class ByteRowKeyComparator implements Comparator<byte[]>
|
|||
return Ints.checkedCast((long) fieldCount * Integer.BYTES);
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a list of sort columns, compute an array of the number of ascending fields in a run, followed by number of
|
||||
* descending fields in a run, followed by ascending, etc. For example: ASC, ASC, DESC, ASC would return [2, 1, 1]
|
||||
* and DESC, DESC, ASC would return [0, 2, 1].
|
||||
*
|
||||
* Public so {@link FrameComparisonWidgetImpl} can use it.
|
||||
*/
|
||||
public static int[] computeAscDescRunLengths(final List<KeyColumn> keyColumns)
|
||||
{
|
||||
final IntList ascDescRunLengths = new IntArrayList(4);
|
||||
|
||||
KeyOrder order = KeyOrder.ASCENDING;
|
||||
int runLength = 0;
|
||||
|
||||
for (final KeyColumn column : keyColumns) {
|
||||
if (column.order() == KeyOrder.NONE) {
|
||||
throw new IAE("Key must be sortable");
|
||||
}
|
||||
|
||||
if (column.order() != order) {
|
||||
ascDescRunLengths.add(runLength);
|
||||
runLength = 0;
|
||||
|
||||
// Invert "order".
|
||||
order = order == KeyOrder.ASCENDING ? KeyOrder.DESCENDING : KeyOrder.ASCENDING;
|
||||
}
|
||||
|
||||
runLength++;
|
||||
}
|
||||
|
||||
if (runLength > 0) {
|
||||
ascDescRunLengths.add(runLength);
|
||||
}
|
||||
|
||||
return ascDescRunLengths.toIntArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("SubtractionInCompareTo")
|
||||
public int compare(final byte[] keyArray1, final byte[] keyArray2)
|
||||
{
|
||||
// Similar logic to FrameComparaisonWidgetImpl, but implementation is different enough that we need our own.
|
||||
// Similar logic to FrameComparisonWidgetImpl, but implementation is different enough that we need our own.
|
||||
// Major difference is Frame v. Frame instead of byte[] v. byte[].
|
||||
|
||||
int comparableBytesStartPosition1 = firstFieldPosition;
|
||||
int comparableBytesStartPosition2 = firstFieldPosition;
|
||||
int currentRunStartPosition1 = firstFieldPosition;
|
||||
int currentRunStartPosition2 = firstFieldPosition;
|
||||
|
||||
boolean ascending = true;
|
||||
int field = 0;
|
||||
// Number of fields compared till now, which is equivalent to the index of the field to compare next
|
||||
int fieldsComparedTillNow = 0;
|
||||
|
||||
for (int numFields : ascDescRunLengths) {
|
||||
if (numFields > 0) {
|
||||
final int nextField = field + numFields;
|
||||
final int comparableBytesEndPosition1 = RowKeyReader.fieldEndPosition(keyArray1, nextField - 1);
|
||||
final int comparableBytesEndPosition2 = RowKeyReader.fieldEndPosition(keyArray2, nextField - 1);
|
||||
for (int i = 0; i < rowKeyComparisonRunLengths.getRunLengthEntries().length; ++i) {
|
||||
final RunLengthEntry runLengthEntry = rowKeyComparisonRunLengths.getRunLengthEntries()[i];
|
||||
|
||||
int cmp = FrameReaderUtils.compareByteArraysUnsigned(
|
||||
keyArray1,
|
||||
comparableBytesStartPosition1,
|
||||
comparableBytesEndPosition1 - comparableBytesStartPosition1,
|
||||
keyArray2,
|
||||
comparableBytesStartPosition2,
|
||||
comparableBytesEndPosition2 - comparableBytesStartPosition2
|
||||
);
|
||||
|
||||
if (cmp != 0) {
|
||||
return ascending ? cmp : -cmp;
|
||||
}
|
||||
|
||||
field = nextField;
|
||||
comparableBytesStartPosition1 = comparableBytesEndPosition1;
|
||||
comparableBytesStartPosition2 = comparableBytesEndPosition2;
|
||||
if (runLengthEntry.getRunLength() <= 0) {
|
||||
// Defensive check
|
||||
continue;
|
||||
}
|
||||
|
||||
ascending = !ascending;
|
||||
}
|
||||
// Index of the next field that will get considered. Excludes the last field of the current run length that is being
|
||||
// compared in this iteration
|
||||
final int nextField = fieldsComparedTillNow + runLengthEntry.getRunLength();
|
||||
final int currentRunEndPosition1 = RowKeyReader.fieldEndPosition(keyArray1, nextField - 1);
|
||||
final int currentRunEndPosition2 = RowKeyReader.fieldEndPosition(keyArray2, nextField - 1);
|
||||
|
||||
final int cmp;
|
||||
|
||||
if (!runLengthEntry.isByteComparable()) {
|
||||
// Only complex types are not byte comparable. Nested arrays aren't supported in MSQ
|
||||
assert runLengthEntry.getRunLength() == 1;
|
||||
cmp = FrameReaderUtils.compareComplexTypes(
|
||||
keyArray1,
|
||||
currentRunStartPosition1,
|
||||
keyArray2,
|
||||
currentRunStartPosition2,
|
||||
columnTypes[i],
|
||||
complexMetricSerdes[i]
|
||||
);
|
||||
} else {
|
||||
// The keys are byte comparable
|
||||
cmp = FrameReaderUtils.compareByteArraysUnsigned(
|
||||
keyArray1,
|
||||
currentRunStartPosition1,
|
||||
currentRunEndPosition1 - currentRunStartPosition1,
|
||||
keyArray2,
|
||||
currentRunStartPosition2,
|
||||
currentRunEndPosition2 - currentRunStartPosition2
|
||||
);
|
||||
}
|
||||
|
||||
if (cmp != 0) {
|
||||
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
|
||||
}
|
||||
|
||||
fieldsComparedTillNow = nextField;
|
||||
currentRunStartPosition1 = currentRunEndPosition1;
|
||||
currentRunStartPosition2 = currentRunEndPosition2;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
@ -159,14 +216,18 @@ public class ByteRowKeyComparator implements Comparator<byte[]>
|
|||
}
|
||||
ByteRowKeyComparator that = (ByteRowKeyComparator) o;
|
||||
return firstFieldPosition == that.firstFieldPosition
|
||||
&& Arrays.equals(ascDescRunLengths, that.ascDescRunLengths);
|
||||
&& Objects.equals(keyColumns, that.keyColumns)
|
||||
&& Objects.equals(rowKeyComparisonRunLengths, that.rowKeyComparisonRunLengths)
|
||||
&& Arrays.equals(complexMetricSerdes, that.complexMetricSerdes)
|
||||
&& Arrays.equals(columnTypes, that.columnTypes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = Objects.hash(firstFieldPosition);
|
||||
result = 31 * result + Arrays.hashCode(ascDescRunLengths);
|
||||
int result = Objects.hash(keyColumns, firstFieldPosition, rowKeyComparisonRunLengths);
|
||||
result = 31 * result + Arrays.hashCode(complexMetricSerdes);
|
||||
result = 31 * result + Arrays.hashCode(columnTypes);
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -174,8 +235,11 @@ public class ByteRowKeyComparator implements Comparator<byte[]>
|
|||
public String toString()
|
||||
{
|
||||
return "ByteRowKeyComparator{" +
|
||||
"firstFieldPosition=" + firstFieldPosition +
|
||||
", ascDescRunLengths=" + Arrays.toString(ascDescRunLengths) +
|
||||
"keyColumns=" + keyColumns +
|
||||
", firstFieldPosition=" + firstFieldPosition +
|
||||
", rowKeyComparisonRunLengths=" + rowKeyComparisonRunLengths +
|
||||
", complexMetricSerdes=" + Arrays.toString(complexMetricSerdes) +
|
||||
", columnTypes=" + Arrays.toString(columnTypes) +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -152,29 +152,29 @@ public class ClusterBy
|
|||
/**
|
||||
* Comparator that compares keys for this instance using the given signature.
|
||||
*/
|
||||
public Comparator<RowKey> keyComparator()
|
||||
public Comparator<RowKey> keyComparator(RowSignature rowSignature)
|
||||
{
|
||||
return RowKeyComparator.create(columns);
|
||||
return RowKeyComparator.create(columns, rowSignature);
|
||||
}
|
||||
|
||||
/**
|
||||
* Comparator that compares byte arrays of keys for this instance using the given signature directly.
|
||||
*/
|
||||
public Comparator<byte[]> byteKeyComparator()
|
||||
public Comparator<byte[]> byteKeyComparator(RowSignature rowSignature)
|
||||
{
|
||||
return ByteRowKeyComparator.create(columns);
|
||||
return ByteRowKeyComparator.create(columns, rowSignature);
|
||||
}
|
||||
|
||||
/**
|
||||
* Comparator that compares bucket keys for this instance. Bucket keys are retrieved by calling
|
||||
* {@link RowKeyReader#trim(RowKey, int)} with {@link #getBucketByCount()}.
|
||||
*/
|
||||
public Comparator<RowKey> bucketComparator()
|
||||
public Comparator<RowKey> bucketComparator(final RowSignature rowSignature)
|
||||
{
|
||||
if (bucketByCount == 0) {
|
||||
return Comparators.alwaysEqual();
|
||||
} else {
|
||||
return RowKeyComparator.create(columns.subList(0, bucketByCount));
|
||||
return RowKeyComparator.create(columns.subList(0, bucketByCount), rowSignature);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,8 +19,10 @@
|
|||
|
||||
package org.apache.druid.frame.key;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.primitives.Ints;
|
||||
import org.apache.datasketches.memory.Memory;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.frame.FrameType;
|
||||
import org.apache.druid.frame.field.FieldReader;
|
||||
|
@ -30,8 +32,12 @@ import org.apache.druid.frame.write.FrameWriterUtils;
|
|||
import org.apache.druid.frame.write.RowBasedFrameWriter;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.serde.ComplexMetricSerde;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
|
||||
import javax.annotation.concurrent.NotThreadSafe;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -40,16 +46,57 @@ import java.util.List;
|
|||
* Comparison logic in this class is very similar to {@link RowKeyComparator}, but is different because it works
|
||||
* on Frames instead of byte[].
|
||||
*/
|
||||
@NotThreadSafe
|
||||
public class FrameComparisonWidgetImpl implements FrameComparisonWidget
|
||||
{
|
||||
|
||||
/**
|
||||
* Frame for comparison
|
||||
*/
|
||||
private final Frame frame;
|
||||
|
||||
/**
|
||||
* Row signature of the frame
|
||||
*/
|
||||
private final RowSignature signature;
|
||||
|
||||
/**
|
||||
* Row offset region of the frame
|
||||
*/
|
||||
private final Memory rowOffsetRegion;
|
||||
|
||||
/**
|
||||
* Data region of the frame
|
||||
*/
|
||||
private final Memory dataRegion;
|
||||
|
||||
/**
|
||||
* Number of fields in the key column
|
||||
*/
|
||||
private final int keyFieldCount;
|
||||
|
||||
/**
|
||||
* Field readers for the key
|
||||
*/
|
||||
private final List<FieldReader> keyFieldReaders;
|
||||
|
||||
/**
|
||||
* Starting position of the first field in the row
|
||||
*/
|
||||
private final int firstFieldPosition;
|
||||
private final int[] ascDescRunLengths;
|
||||
|
||||
/**
|
||||
* Run lengths created for comparing the key columns
|
||||
*/
|
||||
private final RowKeyComparisonRunLengths rowKeyComparisonRunLengths;
|
||||
|
||||
/**
|
||||
* Pre-computed array of ComplexMetricSerde corresponding to the computed run-lengths. If the run length entry is
|
||||
* byte-comparable, the corresponding serde is null, and if it's not byte comparable, the corresponding serde isn't null
|
||||
* (since only complex columns are not byte comparable)
|
||||
*/
|
||||
private final ComplexMetricSerde[] complexMetricSerdes;
|
||||
private final ColumnType[] columnTypes;
|
||||
|
||||
private FrameComparisonWidgetImpl(
|
||||
final Frame frame,
|
||||
|
@ -58,7 +105,9 @@ public class FrameComparisonWidgetImpl implements FrameComparisonWidget
|
|||
final Memory dataRegion,
|
||||
final List<FieldReader> keyFieldReaders,
|
||||
final int firstFieldPosition,
|
||||
final int[] ascDescRunLengths
|
||||
final RowKeyComparisonRunLengths rowKeyComparisonRunLengths,
|
||||
final ComplexMetricSerde[] complexMetricSerdes,
|
||||
final ColumnType[] columnTypes
|
||||
)
|
||||
{
|
||||
this.frame = frame;
|
||||
|
@ -68,7 +117,9 @@ public class FrameComparisonWidgetImpl implements FrameComparisonWidget
|
|||
this.keyFieldCount = keyFieldReaders.size();
|
||||
this.keyFieldReaders = keyFieldReaders;
|
||||
this.firstFieldPosition = firstFieldPosition;
|
||||
this.ascDescRunLengths = ascDescRunLengths;
|
||||
this.rowKeyComparisonRunLengths = rowKeyComparisonRunLengths;
|
||||
this.complexMetricSerdes = complexMetricSerdes;
|
||||
this.columnTypes = columnTypes;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -95,6 +146,41 @@ public class FrameComparisonWidgetImpl implements FrameComparisonWidget
|
|||
throw new ISE("Mismatched lengths for keyColumnReaders and keyColumns");
|
||||
}
|
||||
|
||||
final RowKeyComparisonRunLengths rowKeyComparisonRunLengths = RowKeyComparisonRunLengths.create(keyColumns, signature);
|
||||
final RunLengthEntry[] runLengthEntries = rowKeyComparisonRunLengths.getRunLengthEntries();
|
||||
final ComplexMetricSerde[] complexMetricSerdes = new ComplexMetricSerde[runLengthEntries.length];
|
||||
final ColumnType[] columnTypes = new ColumnType[runLengthEntries.length];
|
||||
|
||||
int fieldsSeenSoFar = 0;
|
||||
for (int i = 0; i < runLengthEntries.length; ++i) {
|
||||
// If the run length entry isn't byte comparable, it most definitely is a complex type
|
||||
if (runLengthEntries[i].isByteComparable()) {
|
||||
complexMetricSerdes[i] = null;
|
||||
columnTypes[i] = null;
|
||||
} else {
|
||||
final String columnName = keyColumns.get(fieldsSeenSoFar).columnName();
|
||||
final ColumnType columnType = signature.getColumnType(columnName).orElse(null);
|
||||
if (columnType == null) {
|
||||
throw DruidException.defensive(
|
||||
"Cannot compare on the byte incomparable column [%s] without knowing it's type",
|
||||
columnName
|
||||
);
|
||||
}
|
||||
final String complexTypeName = columnType.getComplexTypeName();
|
||||
if (complexTypeName == null) {
|
||||
throw DruidException.defensive("Expected complex type name for column [%s] for comparison", columnName);
|
||||
}
|
||||
complexMetricSerdes[i] = Preconditions.checkNotNull(
|
||||
ComplexMetrics.getSerdeForType(complexTypeName),
|
||||
"Cannot find serde for column [%s] of type [%s]",
|
||||
columnName,
|
||||
complexTypeName
|
||||
);
|
||||
columnTypes[i] = columnType;
|
||||
}
|
||||
fieldsSeenSoFar += runLengthEntries[i].getRunLength();
|
||||
}
|
||||
|
||||
return new FrameComparisonWidgetImpl(
|
||||
FrameType.ROW_BASED.ensureType(frame),
|
||||
signature,
|
||||
|
@ -102,10 +188,15 @@ public class FrameComparisonWidgetImpl implements FrameComparisonWidget
|
|||
frame.region(RowBasedFrameWriter.ROW_DATA_REGION),
|
||||
keyColumnReaders,
|
||||
ByteRowKeyComparator.computeFirstFieldPosition(signature.size()),
|
||||
ByteRowKeyComparator.computeAscDescRunLengths(keyColumns)
|
||||
rowKeyComparisonRunLengths,
|
||||
complexMetricSerdes,
|
||||
columnTypes
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates {@link RowKey} from a row in the frame. See the layout of the {@link RowKey}
|
||||
*/
|
||||
@Override
|
||||
public RowKey readKey(int row)
|
||||
{
|
||||
|
@ -121,6 +212,7 @@ public class FrameComparisonWidgetImpl implements FrameComparisonWidget
|
|||
final long keyLength = keyEndInRow - firstFieldPosition;
|
||||
final byte[] keyBytes = new byte[Ints.checkedCast(keyFieldPointersEndInRow + keyEndInRow - firstFieldPosition)];
|
||||
|
||||
// Length of the portion of the header which isn't included in the rowKey
|
||||
final int headerSizeAdjustment = (signature.size() - keyFieldCount) * Integer.BYTES;
|
||||
for (int i = 0; i < keyFieldCount; i++) {
|
||||
final int fieldEndPosition = dataRegion.getInt(rowPosition + ((long) Integer.BYTES * i));
|
||||
|
@ -178,41 +270,56 @@ public class FrameComparisonWidgetImpl implements FrameComparisonWidget
|
|||
final long rowPosition = getRowPositionInDataRegion(row);
|
||||
|
||||
long comparableBytesStartPositionInRow = firstFieldPosition;
|
||||
int keyComparableBytesStartPosition = Integer.BYTES * keyFieldCount;
|
||||
int comparableBytesStartPositionInKey = Integer.BYTES * keyFieldCount;
|
||||
|
||||
boolean ascending = true;
|
||||
int field = 0;
|
||||
// Number of fields compared till now, which is equivalent to the index of the field to compare next
|
||||
int fieldsComparedTillNow = 0;
|
||||
|
||||
for (int numFields : ascDescRunLengths) {
|
||||
if (numFields > 0) {
|
||||
final int nextField = field + numFields;
|
||||
final long comparableBytesEndPositionInRow = getFieldEndPositionInRow(rowPosition, nextField - 1);
|
||||
final int keyComparableBytesEndPosition = RowKeyReader.fieldEndPosition(keyArray, nextField - 1);
|
||||
for (int i = 0; i < rowKeyComparisonRunLengths.getRunLengthEntries().length; ++i) {
|
||||
|
||||
final long comparableBytesLength = comparableBytesEndPositionInRow - comparableBytesStartPositionInRow;
|
||||
final int keyComparableBytesLength = keyComparableBytesEndPosition - keyComparableBytesStartPosition;
|
||||
final RunLengthEntry runLengthEntry = rowKeyComparisonRunLengths.getRunLengthEntries()[i];
|
||||
|
||||
int cmp = FrameReaderUtils.compareMemoryToByteArrayUnsigned(
|
||||
dataRegion,
|
||||
rowPosition + comparableBytesStartPositionInRow,
|
||||
comparableBytesLength,
|
||||
keyArray,
|
||||
keyComparableBytesStartPosition,
|
||||
keyComparableBytesLength
|
||||
);
|
||||
|
||||
if (cmp != 0) {
|
||||
return ascending ? cmp : -cmp;
|
||||
}
|
||||
|
||||
field += numFields;
|
||||
comparableBytesStartPositionInRow += comparableBytesLength;
|
||||
keyComparableBytesStartPosition += keyComparableBytesLength;
|
||||
if (runLengthEntry.getRunLength() <= 0) {
|
||||
// Defensive check
|
||||
continue;
|
||||
}
|
||||
|
||||
ascending = !ascending;
|
||||
}
|
||||
// Index of the next field that will get considered. Excludes the current field that we are comparing right now
|
||||
final int nextField = fieldsComparedTillNow + runLengthEntry.getRunLength();
|
||||
final int comparableBytesEndPositionInKey = RowKeyReader.fieldEndPosition(keyArray, nextField - 1);
|
||||
final int comparableBytesEndPositionInRow = getFieldEndPositionInRow(rowPosition, nextField - 1);
|
||||
|
||||
final int cmp;
|
||||
|
||||
if (!runLengthEntry.isByteComparable()) {
|
||||
// Only complex types are not byte comparable. Nested arrays aren't supported in MSQ
|
||||
assert runLengthEntry.getRunLength() == 1;
|
||||
cmp = FrameReaderUtils.compareComplexTypes(
|
||||
dataRegion,
|
||||
rowPosition + comparableBytesStartPositionInRow,
|
||||
keyArray,
|
||||
comparableBytesStartPositionInKey,
|
||||
columnTypes[i],
|
||||
complexMetricSerdes[i]
|
||||
);
|
||||
} else {
|
||||
cmp = FrameReaderUtils.compareMemoryToByteArrayUnsigned(
|
||||
dataRegion,
|
||||
rowPosition + comparableBytesStartPositionInRow,
|
||||
comparableBytesEndPositionInRow - comparableBytesStartPositionInRow,
|
||||
keyArray,
|
||||
comparableBytesStartPositionInKey,
|
||||
comparableBytesEndPositionInKey - comparableBytesStartPositionInKey
|
||||
);
|
||||
}
|
||||
|
||||
if (cmp != 0) {
|
||||
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
|
||||
}
|
||||
fieldsComparedTillNow = nextField;
|
||||
comparableBytesStartPositionInRow = comparableBytesEndPositionInRow;
|
||||
comparableBytesStartPositionInKey = comparableBytesEndPositionInKey;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
@ -230,39 +337,52 @@ public class FrameComparisonWidgetImpl implements FrameComparisonWidget
|
|||
int comparableBytesStartPositionInRow = firstFieldPosition;
|
||||
int otherComparableBytesStartPositionInRow = otherWidgetImpl.firstFieldPosition;
|
||||
|
||||
boolean ascending = true;
|
||||
int field = 0;
|
||||
// Number of fields compared till now, which is equivalent to the index of the field to compare next
|
||||
int fieldsComparedTillNow = 0;
|
||||
|
||||
for (int numFields : ascDescRunLengths) {
|
||||
if (numFields > 0) {
|
||||
final int nextField = field + numFields;
|
||||
final int comparableBytesEndPositionInRow = getFieldEndPositionInRow(rowPosition, nextField - 1);
|
||||
final int otherComparableBytesEndPositionInRow =
|
||||
otherWidgetImpl.getFieldEndPositionInRow(otherRowPosition, nextField - 1);
|
||||
|
||||
final int comparableBytesLength = comparableBytesEndPositionInRow - comparableBytesStartPositionInRow;
|
||||
final int otherComparableBytesLength =
|
||||
otherComparableBytesEndPositionInRow - otherComparableBytesStartPositionInRow;
|
||||
|
||||
int cmp = FrameReaderUtils.compareMemoryUnsigned(
|
||||
dataRegion,
|
||||
rowPosition + comparableBytesStartPositionInRow,
|
||||
comparableBytesLength,
|
||||
otherWidgetImpl.getDataRegion(),
|
||||
otherRowPosition + otherComparableBytesStartPositionInRow,
|
||||
otherComparableBytesLength
|
||||
);
|
||||
|
||||
if (cmp != 0) {
|
||||
return ascending ? cmp : -cmp;
|
||||
}
|
||||
|
||||
field += numFields;
|
||||
comparableBytesStartPositionInRow += comparableBytesLength;
|
||||
otherComparableBytesStartPositionInRow += otherComparableBytesLength;
|
||||
for (int i = 0; i < rowKeyComparisonRunLengths.getRunLengthEntries().length; ++i) {
|
||||
final RunLengthEntry runLengthEntry = rowKeyComparisonRunLengths.getRunLengthEntries()[i];
|
||||
if (runLengthEntry.getRunLength() <= 0) {
|
||||
// Defensive check
|
||||
continue;
|
||||
}
|
||||
|
||||
ascending = !ascending;
|
||||
final int nextField = fieldsComparedTillNow + runLengthEntry.getRunLength();
|
||||
final int comparableBytesEndPositionInRow = getFieldEndPositionInRow(rowPosition, nextField - 1);
|
||||
final int otherComparableBytesEndPositionInRow = otherWidgetImpl.getFieldEndPositionInRow(otherRowPosition, nextField - 1);
|
||||
|
||||
final int cmp;
|
||||
|
||||
if (!runLengthEntry.isByteComparable()) {
|
||||
// Only complex types are not byte comparable. Nested arrays aren't supported in MSQ
|
||||
assert runLengthEntry.getRunLength() == 1;
|
||||
|
||||
cmp = FrameReaderUtils.compareComplexTypes(
|
||||
dataRegion,
|
||||
rowPosition + comparableBytesStartPositionInRow,
|
||||
otherWidgetImpl.dataRegion,
|
||||
otherRowPosition + otherComparableBytesStartPositionInRow,
|
||||
columnTypes[i],
|
||||
complexMetricSerdes[i]
|
||||
);
|
||||
} else {
|
||||
cmp = FrameReaderUtils.compareMemoryUnsigned(
|
||||
dataRegion,
|
||||
rowPosition + comparableBytesStartPositionInRow,
|
||||
comparableBytesEndPositionInRow - comparableBytesStartPositionInRow,
|
||||
otherWidgetImpl.getDataRegion(),
|
||||
otherRowPosition + otherComparableBytesStartPositionInRow,
|
||||
otherComparableBytesEndPositionInRow - otherComparableBytesStartPositionInRow
|
||||
);
|
||||
}
|
||||
|
||||
if (cmp != 0) {
|
||||
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
|
||||
}
|
||||
|
||||
fieldsComparedTillNow = nextField;
|
||||
comparableBytesStartPositionInRow = comparableBytesEndPositionInRow;
|
||||
otherComparableBytesStartPositionInRow = otherComparableBytesEndPositionInRow;
|
||||
}
|
||||
|
||||
return 0;
|
||||
|
|
|
@ -69,8 +69,8 @@ public class KeyColumn
|
|||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
KeyColumn that = (KeyColumn) o;
|
||||
return order == that.order && Objects.equals(columnName, that.columnName);
|
||||
KeyColumn keyColumn = (KeyColumn) o;
|
||||
return Objects.equals(columnName, keyColumn.columnName) && order == keyColumn.order;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -27,6 +27,16 @@ import java.util.Arrays;
|
|||
|
||||
/**
|
||||
* Represents a specific sorting or hashing key. Instances of this class wrap a byte array in row-based frame format.
|
||||
*
|
||||
* Following is the layout of the RowKey with n fields
|
||||
*
|
||||
* Header section
|
||||
* byte[1..4] - End of field 1
|
||||
* byte[5..8] - End of field 2
|
||||
* ...
|
||||
* byte[4(n-1)..4n] - End of field n
|
||||
* Key section
|
||||
* byte[headerEnd+1..headerEnd+1+fieldSize1] - Data of field1
|
||||
*/
|
||||
public class RowKey
|
||||
{
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
package org.apache.druid.frame.key;
|
||||
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -36,9 +38,9 @@ public class RowKeyComparator implements Comparator<RowKey>
|
|||
this.byteRowKeyComparatorDelegate = byteRowKeyComparatorDelegate;
|
||||
}
|
||||
|
||||
public static RowKeyComparator create(final List<KeyColumn> keyColumns)
|
||||
public static RowKeyComparator create(final List<KeyColumn> keyColumns, RowSignature rowSignature)
|
||||
{
|
||||
return new RowKeyComparator(ByteRowKeyComparator.create(keyColumns));
|
||||
return new RowKeyComparator(ByteRowKeyComparator.create(keyColumns, rowSignature));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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.frame.key;
|
||||
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Denotes the ascending-descending run lengths of the fields of the keycolumns that can be compared together.
|
||||
* It analyses the key columns and their types. It coalesces the adjacent key columns if they are:
|
||||
* a. Byte comparable, i.e. the fields won't need to be deserialized before comparing. It doesn't care about the types
|
||||
* b. Have same order
|
||||
*
|
||||
* All the primitive and the primitive arrays are byte comparable. The complex types are not byte comparable, and nested arrays
|
||||
* and arrays of complex objects are not supported by MSQ right now.
|
||||
*
|
||||
* Consider a row with the key columns like:
|
||||
*
|
||||
* ColumnName ColumnType Order
|
||||
* ==========================================
|
||||
* longAsc1 LONG ASC
|
||||
* stringAsc1 STRING ASC
|
||||
* stringDesc1 STRING DESC
|
||||
* longDesc1 LONG DESC
|
||||
* complexDesc1 COMPLEX DESC
|
||||
* complexAsc1 COMPLEX ASC
|
||||
* complexAsc2 COMPLEX ASC
|
||||
* stringAsc2 STRING ASC
|
||||
*
|
||||
* The run lengths generated would be:
|
||||
*
|
||||
* RunLengthEntry Run length Order Is byte comparable Explanation
|
||||
* ====================================================================================================================
|
||||
* RunLengthEntry#1 2 ASC true Even though longAsc1 and stringAsc1 had different types,
|
||||
* both types are byte comparable and have same direction. Therefore,
|
||||
* they can be byte-compared together
|
||||
*
|
||||
* RunLengthEntry#2 2 DESC true stringDesc1 can't be clubed with the previous stringAsc1 due to
|
||||
* different ordering. It is clubbed with the following longDesc1 due
|
||||
* to the reason stated above
|
||||
* RunLengthEntry#3 1 DESC false Non byte comparable types cannot be clubbed with anything
|
||||
* RunLengthEntry#4 1 ASC false Non byte comparable types cannot be clubbed with anything
|
||||
* RunLengthEntry#5 1 ASC false Non byte comparable types cannot be clubbed with anything despite
|
||||
* the previous key column having same order and the type
|
||||
* RunLengthEntry#6 1 ASC true Cannot be clubbed with previous entry. It is own entry
|
||||
*
|
||||
*/
|
||||
public class RowKeyComparisonRunLengths
|
||||
{
|
||||
private final RunLengthEntry[] runLengthEntries;
|
||||
|
||||
private RowKeyComparisonRunLengths(final RunLengthEntry[] runLengthEntries)
|
||||
{
|
||||
this.runLengthEntries = runLengthEntries;
|
||||
}
|
||||
|
||||
public static RowKeyComparisonRunLengths create(final List<KeyColumn> keyColumns, RowSignature rowSignature)
|
||||
{
|
||||
final List<RunLengthEntryBuilder> runLengthEntryBuilders = new ArrayList<>();
|
||||
for (KeyColumn keyColumn : keyColumns) {
|
||||
if (keyColumn.order() == KeyOrder.NONE) {
|
||||
throw DruidException.defensive(
|
||||
"Cannot sort on column [%s] when the sorting order isn't provided",
|
||||
keyColumn.columnName()
|
||||
);
|
||||
}
|
||||
|
||||
ColumnType columnType = rowSignature.getColumnType(keyColumn.columnName())
|
||||
.orElseThrow(() -> DruidException.defensive("Need column types"));
|
||||
|
||||
// First key column to be processed
|
||||
if (runLengthEntryBuilders.size() == 0) {
|
||||
final boolean isByteComparable = isByteComparable(columnType);
|
||||
runLengthEntryBuilders.add(
|
||||
new RunLengthEntryBuilder(isByteComparable, keyColumn.order())
|
||||
);
|
||||
continue;
|
||||
}
|
||||
|
||||
// There is atleast one RunLengthEntry present in the array. Check if we can find a way to merge the current entry
|
||||
// with the previous one
|
||||
boolean isCurrentColumnByteComparable = isByteComparable(columnType);
|
||||
RunLengthEntryBuilder lastRunLengthEntryBuilder = runLengthEntryBuilders.get(runLengthEntryBuilders.size() - 1);
|
||||
if (lastRunLengthEntryBuilder.byteComparable
|
||||
&& isCurrentColumnByteComparable
|
||||
&& lastRunLengthEntryBuilder.order.equals(keyColumn.order())
|
||||
) {
|
||||
lastRunLengthEntryBuilder.runLength++;
|
||||
} else {
|
||||
runLengthEntryBuilders.add(
|
||||
new RunLengthEntryBuilder(
|
||||
isCurrentColumnByteComparable,
|
||||
keyColumn.order()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
RunLengthEntry[] runLengthEntries = new RunLengthEntry[runLengthEntryBuilders.size()];
|
||||
for (int i = 0; i < runLengthEntryBuilders.size(); ++i) {
|
||||
runLengthEntries[i] = runLengthEntryBuilders.get(i).build();
|
||||
}
|
||||
|
||||
return new RowKeyComparisonRunLengths(runLengthEntries);
|
||||
}
|
||||
|
||||
private static boolean isByteComparable(final ColumnType columnType)
|
||||
{
|
||||
if (columnType.is(ValueType.COMPLEX)) {
|
||||
if (columnType.getComplexTypeName() == null) {
|
||||
throw DruidException.defensive("Cannot sort unknown complex types");
|
||||
}
|
||||
// Complex types with known types are not byte comparable and must be deserialized for comparison
|
||||
return false;
|
||||
} else if (columnType.isArray() && !columnType.isPrimitiveArray()) {
|
||||
// Nested arrays aren't allowed directly in the frames - they are materialized as nested types.
|
||||
// Nested arrays aren't byte comparable, if they find a way to creep in.
|
||||
throw DruidException.defensive("Nested arrays aren't supported in row based frames");
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public RunLengthEntry[] getRunLengthEntries()
|
||||
{
|
||||
return runLengthEntries;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
RowKeyComparisonRunLengths that = (RowKeyComparisonRunLengths) o;
|
||||
return Arrays.equals(runLengthEntries, that.runLengthEntries);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Arrays.hashCode(runLengthEntries);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "RowKeyComparisonRunLengths{" +
|
||||
"runLengthEntries=" + Arrays.toString(runLengthEntries) +
|
||||
'}';
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder for {@link RunLengthEntry}. Contains mutable state, therefore it isn't suitable for equality and hashCode.
|
||||
*/
|
||||
private static class RunLengthEntryBuilder
|
||||
{
|
||||
private final boolean byteComparable;
|
||||
private final KeyOrder order;
|
||||
private int runLength;
|
||||
|
||||
public RunLengthEntryBuilder(
|
||||
final boolean byteComparable,
|
||||
final KeyOrder order
|
||||
)
|
||||
{
|
||||
this.byteComparable = byteComparable;
|
||||
this.order = order;
|
||||
this.runLength = 1;
|
||||
}
|
||||
|
||||
public RunLengthEntry build()
|
||||
{
|
||||
return new RunLengthEntry(byteComparable, order, runLength);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,83 @@
|
|||
/*
|
||||
* 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.frame.key;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Information about a continguous run of keys, that has the same sorting order
|
||||
*/
|
||||
public class RunLengthEntry
|
||||
{
|
||||
private final boolean byteComparable;
|
||||
private final KeyOrder order;
|
||||
private final int runLength;
|
||||
|
||||
RunLengthEntry(final boolean byteComparable, final KeyOrder order, final int runLength)
|
||||
{
|
||||
this.byteComparable = byteComparable;
|
||||
this.order = order;
|
||||
this.runLength = runLength;
|
||||
}
|
||||
|
||||
public boolean isByteComparable()
|
||||
{
|
||||
return byteComparable;
|
||||
}
|
||||
|
||||
public int getRunLength()
|
||||
{
|
||||
return runLength;
|
||||
}
|
||||
|
||||
public KeyOrder getOrder()
|
||||
{
|
||||
return order;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
RunLengthEntry that = (RunLengthEntry) o;
|
||||
return byteComparable == that.byteComparable && runLength == that.runLength && order == that.order;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(byteComparable, order, runLength);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "RunLengthEntry{" +
|
||||
"byteComparable=" + byteComparable +
|
||||
", order=" + order +
|
||||
", runLength=" + runLength +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -22,12 +22,15 @@ package org.apache.druid.frame.read;
|
|||
import com.google.common.primitives.Ints;
|
||||
import org.apache.datasketches.memory.Memory;
|
||||
import org.apache.druid.frame.allocation.MemoryRange;
|
||||
import org.apache.druid.frame.field.ComplexFieldReader;
|
||||
import org.apache.druid.frame.segment.row.FrameColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.serde.ComplexMetricSerde;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -216,6 +219,52 @@ public class FrameReaderUtils
|
|||
return Integer.compare(length1, length2);
|
||||
}
|
||||
|
||||
public static int compareComplexTypes(
|
||||
final byte[] array1,
|
||||
final int position1,
|
||||
final byte[] array2,
|
||||
final int position2,
|
||||
final ColumnType columnType,
|
||||
final ComplexMetricSerde complexMetricSerde
|
||||
)
|
||||
{
|
||||
return columnType.getNullableStrategy().compare(
|
||||
ComplexFieldReader.readFieldFromByteArray(complexMetricSerde, array1, position1),
|
||||
ComplexFieldReader.readFieldFromByteArray(complexMetricSerde, array2, position2)
|
||||
);
|
||||
}
|
||||
|
||||
public static int compareComplexTypes(
|
||||
final Memory memory,
|
||||
final long position1,
|
||||
final byte[] array,
|
||||
final int position2,
|
||||
final ColumnType columnType,
|
||||
final ComplexMetricSerde complexMetricSerde
|
||||
)
|
||||
{
|
||||
return columnType.getNullableStrategy().compare(
|
||||
ComplexFieldReader.readFieldFromMemory(complexMetricSerde, memory, position1),
|
||||
ComplexFieldReader.readFieldFromByteArray(complexMetricSerde, array, position2)
|
||||
);
|
||||
}
|
||||
|
||||
public static int compareComplexTypes(
|
||||
final Memory memory1,
|
||||
final long position1,
|
||||
final Memory memory2,
|
||||
final long position2,
|
||||
final ColumnType columnType,
|
||||
final ComplexMetricSerde complexMetricSerde
|
||||
)
|
||||
{
|
||||
return columnType.getNullableStrategy().compare(
|
||||
ComplexFieldReader.readFieldFromMemory(complexMetricSerde, memory1, position1),
|
||||
ComplexFieldReader.readFieldFromMemory(complexMetricSerde, memory2, position2)
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns whether a {@link ColumnSelectorFactory} may be able to provide a {@link MemoryRange}. This enables
|
||||
* efficient copying without needing to deal with each field individually.
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.frame.write;
|
|||
import org.apache.datasketches.memory.WritableMemory;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.frame.FrameType;
|
||||
import org.apache.druid.frame.field.FieldReaders;
|
||||
import org.apache.druid.frame.key.KeyColumn;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
@ -236,8 +235,8 @@ public class FrameWriterUtils
|
|||
for (final KeyColumn keyColumn : keyColumns) {
|
||||
final ColumnType columnType = signature.getColumnType(keyColumn.columnName()).orElse(null);
|
||||
|
||||
if (columnType == null || !FieldReaders.create(keyColumn.columnName(), columnType).isComparable()) {
|
||||
throw new IAE("Sort column [%s] is not comparable (type = %s)", keyColumn.columnName(), columnType);
|
||||
if (columnType == null) {
|
||||
throw new IAE("Sort column [%s] type is unknown", keyColumn.columnName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.indexer.report;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
||||
public class KillTaskReport implements TaskReport
|
||||
|
@ -85,18 +84,15 @@ public class KillTaskReport implements TaskReport
|
|||
{
|
||||
private final int numSegmentsKilled;
|
||||
private final int numBatchesProcessed;
|
||||
private final Integer numSegmentsMarkedAsUnused;
|
||||
|
||||
@JsonCreator
|
||||
public Stats(
|
||||
@JsonProperty("numSegmentsKilled") int numSegmentsKilled,
|
||||
@JsonProperty("numBatchesProcessed") int numBatchesProcessed,
|
||||
@JsonProperty("numSegmentsMarkedAsUnused") @Nullable Integer numSegmentsMarkedAsUnused
|
||||
@JsonProperty("numBatchesProcessed") int numBatchesProcessed
|
||||
)
|
||||
{
|
||||
this.numSegmentsKilled = numSegmentsKilled;
|
||||
this.numBatchesProcessed = numBatchesProcessed;
|
||||
this.numSegmentsMarkedAsUnused = numSegmentsMarkedAsUnused;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -111,13 +107,6 @@ public class KillTaskReport implements TaskReport
|
|||
return numBatchesProcessed;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
public Integer getNumSegmentsMarkedAsUnused()
|
||||
{
|
||||
return numSegmentsMarkedAsUnused;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
@ -129,14 +118,13 @@ public class KillTaskReport implements TaskReport
|
|||
}
|
||||
Stats that = (Stats) o;
|
||||
return numSegmentsKilled == that.numSegmentsKilled
|
||||
&& numBatchesProcessed == that.numBatchesProcessed
|
||||
&& Objects.equals(this.numSegmentsMarkedAsUnused, that.numSegmentsMarkedAsUnused);
|
||||
&& numBatchesProcessed == that.numBatchesProcessed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(numSegmentsKilled, numBatchesProcessed, numSegmentsMarkedAsUnused);
|
||||
return Objects.hash(numSegmentsKilled, numBatchesProcessed);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -145,7 +133,6 @@ public class KillTaskReport implements TaskReport
|
|||
return "Stats{" +
|
||||
"numSegmentsKilled=" + numSegmentsKilled +
|
||||
", numBatchesProcessed=" + numBatchesProcessed +
|
||||
", numSegmentsMarkedAsUnused=" + numSegmentsMarkedAsUnused +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -102,6 +102,20 @@ public class GroupingAggregatorFactory extends AggregatorFactory
|
|||
)
|
||||
{
|
||||
Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
||||
Preconditions.checkArgument(!CollectionUtils.isNullOrEmpty(groupings), "Must have a non-empty grouping dimensions");
|
||||
// (Long.SIZE - 1) is just a sanity check. In practice, it will be just few dimensions. This limit
|
||||
// also makes sure that values are always positive.
|
||||
Preconditions.checkArgument(
|
||||
groupings.size() < Long.SIZE,
|
||||
"Number of dimensions %s is more than supported %s",
|
||||
groupings.size(),
|
||||
Long.SIZE - 1
|
||||
);
|
||||
Preconditions.checkArgument(
|
||||
groupings.stream().distinct().count() == groupings.size(),
|
||||
"Encountered same dimension more than once in groupings"
|
||||
);
|
||||
|
||||
this.name = name;
|
||||
this.groupings = groupings;
|
||||
this.keyDimensions = keyDimensions;
|
||||
|
@ -254,15 +268,6 @@ public class GroupingAggregatorFactory extends AggregatorFactory
|
|||
*/
|
||||
private long groupingId(List<String> groupings, @Nullable Set<String> keyDimensions)
|
||||
{
|
||||
Preconditions.checkArgument(!CollectionUtils.isNullOrEmpty(groupings), "Must have a non-empty grouping dimensions");
|
||||
// (Long.SIZE - 1) is just a sanity check. In practice, it will be just few dimensions. This limit
|
||||
// also makes sure that values are always positive.
|
||||
Preconditions.checkArgument(
|
||||
groupings.size() < Long.SIZE,
|
||||
"Number of dimensions %s is more than supported %s",
|
||||
groupings.size(),
|
||||
Long.SIZE - 1
|
||||
);
|
||||
long temp = 0L;
|
||||
for (String groupingDimension : groupings) {
|
||||
temp = temp << 1;
|
||||
|
|
|
@ -179,20 +179,38 @@ public interface TypeStrategy<T> extends Comparator<Object>, Hash.Strategy<T>
|
|||
* true or false, depending on whether the semantics and implementation of the type naturally leads to groupability
|
||||
* or not. For example, it makes sense for JSON columns to be groupable, however there is little sense in grouping
|
||||
* sketches (before finalizing).
|
||||
*
|
||||
* If a type is groupable, it MUST implement the {@link #hashCode} and {@link #equals} correctly
|
||||
* <p>
|
||||
* If a type is groupable, following statements MUST hold:
|
||||
* <p>
|
||||
* a. {@link #equals(Object, Object)} must be implemented. It should return true if and only if two objects are equal
|
||||
* and can be grouped together.
|
||||
* <p>
|
||||
* b. {@link #hashCode(Object)} must be implemented, and must be consistent with equals. It should return a hashCode
|
||||
* for the given object. For two objects that are equal, it must return the same hash value. For two objects that are
|
||||
* not equal, it can return the same hash value (or not). A conscious effort must be made to minimise collisions between
|
||||
* the hash values of two non-equal objects for faster grouping.
|
||||
* <p>
|
||||
* c. {@link #compare(Object, Object)} must be consistent with equals. Apart from abiding by the definition of
|
||||
* {@link Comparator#compare}, it must not return 0 for two objects that are not equals, and converse must also hold,
|
||||
* i.e. if the value returned by compare is not zero, then the arguments must not be equal.
|
||||
*/
|
||||
default boolean groupable()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* @see #groupable()
|
||||
*/
|
||||
@Override
|
||||
default int hashCode(T o)
|
||||
{
|
||||
throw DruidException.defensive("Not implemented. Check groupable() first");
|
||||
}
|
||||
|
||||
/**
|
||||
* @see #groupable()
|
||||
*/
|
||||
@Override
|
||||
default boolean equals(T a, T b)
|
||||
{
|
||||
|
|
|
@ -20,10 +20,16 @@
|
|||
package org.apache.druid.frame.key;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.hash.Hashing;
|
||||
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||
import org.apache.druid.hll.HyperLogLogCollector;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.guava.Comparators;
|
||||
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -35,20 +41,161 @@ import java.util.stream.Collectors;
|
|||
|
||||
public class ByteRowKeyComparatorTest extends InitializedNullHandlingTest
|
||||
{
|
||||
static final RowSignature SIGNATURE =
|
||||
|
||||
static {
|
||||
ComplexMetrics.registerSerde(HyperUniquesSerde.TYPE_NAME, new HyperUniquesSerde());
|
||||
}
|
||||
|
||||
static final RowSignature NO_COMPLEX_SIGNATURE =
|
||||
RowSignature.builder()
|
||||
.add("1", ColumnType.LONG)
|
||||
.add("2", ColumnType.STRING)
|
||||
.add("3", ColumnType.LONG)
|
||||
.add("4", ColumnType.DOUBLE)
|
||||
.build();
|
||||
private static final Object[] OBJECTS1 = new Object[]{-1L, "foo", 2L, -1.2};
|
||||
private static final Object[] OBJECTS2 = new Object[]{-1L, null, 2L, 1.2d};
|
||||
private static final Object[] OBJECTS3 = new Object[]{-1L, "bar", 2L, 1.2d};
|
||||
private static final Object[] OBJECTS4 = new Object[]{-1L, "foo", 2L, 1.2d};
|
||||
private static final Object[] OBJECTS5 = new Object[]{-1L, "foo", 3L, 1.2d};
|
||||
private static final Object[] OBJECTS6 = new Object[]{-1L, "foo", 2L, 1.3d};
|
||||
private static final Object[] OBJECTS7 = new Object[]{1L, "foo", 2L, -1.2d};
|
||||
|
||||
static final RowSignature SIGNATURE =
|
||||
RowSignature.builder()
|
||||
.add("1", HyperUniquesAggregatorFactory.TYPE)
|
||||
.add("2", ColumnType.LONG)
|
||||
.add("3", ColumnType.STRING)
|
||||
.add("4", HyperUniquesAggregatorFactory.TYPE)
|
||||
.add("5", ColumnType.LONG)
|
||||
.add("6", ColumnType.DOUBLE)
|
||||
.add("7", HyperUniquesAggregatorFactory.TYPE)
|
||||
.add("8", HyperUniquesAggregatorFactory.TYPE)
|
||||
.build();
|
||||
|
||||
private static final Object[] OBJECTS1_WITHOUT_COMPLEX_COLUMN =
|
||||
new Object[]{-1L, "foo", 2L, -1.2};
|
||||
private static final Object[] OBJECTS2_WITHOUT_COMPLEX_COLUMN =
|
||||
new Object[]{-1L, null, 2L, 1.2d};
|
||||
private static final Object[] OBJECTS3_WITHOUT_COMPLEX_COLUMN =
|
||||
new Object[]{-1L, "bar", 2L, 1.2d};
|
||||
private static final Object[] OBJECTS4_WITHOUT_COMPLEX_COLUMN =
|
||||
new Object[]{-1L, "foo", 2L, 1.2d};
|
||||
private static final Object[] OBJECTS5_WITHOUT_COMPLEX_COLUMN =
|
||||
new Object[]{-1L, "foo", 3L, 1.2d};
|
||||
private static final Object[] OBJECTS6_WITHOUT_COMPLEX_COLUMN =
|
||||
new Object[]{-1L, "foo", 2L, 1.3d};
|
||||
private static final Object[] OBJECTS7_WITHOUT_COMPLEX_COLUMN =
|
||||
new Object[]{1L, "foo", 2L, -1.2d};
|
||||
private static final Object[] OBJECTS8_WITHOUT_COMPLEX_COLUMN =
|
||||
new Object[]{1L, "foo", 2L, -1.2d};
|
||||
private static final Object[] OBJECTS9_WITHOUT_COMPLEX_COLUMN =
|
||||
new Object[]{1L, "foo", 2L, -1.2d};
|
||||
|
||||
private static final Object[] OBJECTS1 =
|
||||
new Object[]{
|
||||
null,
|
||||
-1L,
|
||||
"foo",
|
||||
makeHllCollector(5),
|
||||
2L,
|
||||
-1.2,
|
||||
makeHllCollector(5),
|
||||
makeHllCollector(1)
|
||||
};
|
||||
private static final Object[] OBJECTS2 =
|
||||
new Object[]{
|
||||
null,
|
||||
-1L,
|
||||
null,
|
||||
null,
|
||||
2L,
|
||||
1.2d,
|
||||
makeHllCollector(50),
|
||||
makeHllCollector(5)
|
||||
};
|
||||
private static final Object[] OBJECTS3 =
|
||||
new Object[]{
|
||||
makeHllCollector(50),
|
||||
-1L,
|
||||
"bar",
|
||||
makeHllCollector(5),
|
||||
2L,
|
||||
1.2d,
|
||||
makeHllCollector(5),
|
||||
makeHllCollector(50)
|
||||
};
|
||||
private static final Object[] OBJECTS4 =
|
||||
new Object[]{
|
||||
makeHllCollector(50),
|
||||
-1L,
|
||||
"foo",
|
||||
makeHllCollector(100),
|
||||
2L,
|
||||
1.2d,
|
||||
makeHllCollector(1),
|
||||
makeHllCollector(5)
|
||||
};
|
||||
private static final Object[] OBJECTS5 =
|
||||
new Object[]{
|
||||
makeHllCollector(1),
|
||||
-1L,
|
||||
"foo",
|
||||
makeHllCollector(5),
|
||||
3L,
|
||||
1.2d,
|
||||
null,
|
||||
makeHllCollector(5)
|
||||
};
|
||||
private static final Object[] OBJECTS6 =
|
||||
new Object[]{
|
||||
makeHllCollector(5),
|
||||
-1L,
|
||||
"foo",
|
||||
makeHllCollector(100),
|
||||
2L,
|
||||
1.3d,
|
||||
makeHllCollector(100),
|
||||
makeHllCollector(20)
|
||||
};
|
||||
private static final Object[] OBJECTS7 =
|
||||
new Object[]{
|
||||
makeHllCollector(100),
|
||||
1L,
|
||||
"foo",
|
||||
makeHllCollector(5),
|
||||
2L,
|
||||
-1.2d,
|
||||
null,
|
||||
null
|
||||
};
|
||||
private static final Object[] OBJECTS8 =
|
||||
new Object[]{
|
||||
makeHllCollector(5),
|
||||
1L,
|
||||
"foo",
|
||||
makeHllCollector(50),
|
||||
2L,
|
||||
-1.2d,
|
||||
makeHllCollector(500),
|
||||
makeHllCollector(100)
|
||||
};
|
||||
private static final Object[] OBJECTS9 =
|
||||
new Object[]{
|
||||
makeHllCollector(5),
|
||||
1L,
|
||||
"foo",
|
||||
makeHllCollector(50),
|
||||
2L,
|
||||
-1.2d,
|
||||
makeHllCollector(500),
|
||||
makeHllCollector(10)
|
||||
};
|
||||
|
||||
static final List<Object[]> KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN = Arrays.asList(
|
||||
OBJECTS1_WITHOUT_COMPLEX_COLUMN,
|
||||
OBJECTS2_WITHOUT_COMPLEX_COLUMN,
|
||||
OBJECTS3_WITHOUT_COMPLEX_COLUMN,
|
||||
OBJECTS4_WITHOUT_COMPLEX_COLUMN,
|
||||
OBJECTS5_WITHOUT_COMPLEX_COLUMN,
|
||||
OBJECTS6_WITHOUT_COMPLEX_COLUMN,
|
||||
OBJECTS7_WITHOUT_COMPLEX_COLUMN,
|
||||
OBJECTS8_WITHOUT_COMPLEX_COLUMN,
|
||||
OBJECTS9_WITHOUT_COMPLEX_COLUMN
|
||||
);
|
||||
|
||||
static final List<Object[]> ALL_KEY_OBJECTS = Arrays.asList(
|
||||
OBJECTS1,
|
||||
|
@ -57,11 +204,13 @@ public class ByteRowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
OBJECTS4,
|
||||
OBJECTS5,
|
||||
OBJECTS6,
|
||||
OBJECTS7
|
||||
OBJECTS7,
|
||||
OBJECTS8,
|
||||
OBJECTS9
|
||||
);
|
||||
|
||||
@Test
|
||||
public void test_compare_AAAA() // AAAA = all ascending
|
||||
public void test_compare_AAAA_without_complex_column() // AAAA = all ascending, no complex column
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
|
@ -70,13 +219,13 @@ public class ByteRowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("4", KeyOrder.DESCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS),
|
||||
sortUsingByteKeyComparator(keyColumns, ALL_KEY_OBJECTS)
|
||||
sortUsingObjectComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE),
|
||||
sortUsingByteKeyComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DDDD() // DDDD = all descending
|
||||
public void test_compare_DDDD_without_complex_column() // DDDD = all descending, no complex columns
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
|
@ -85,13 +234,13 @@ public class ByteRowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS),
|
||||
sortUsingByteKeyComparator(keyColumns, ALL_KEY_OBJECTS)
|
||||
sortUsingObjectComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE),
|
||||
sortUsingByteKeyComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DAAD()
|
||||
public void test_compare_DAAD_without_complex_column()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
|
@ -100,13 +249,13 @@ public class ByteRowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS),
|
||||
sortUsingByteKeyComparator(keyColumns, ALL_KEY_OBJECTS)
|
||||
sortUsingObjectComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE),
|
||||
sortUsingByteKeyComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_ADDA()
|
||||
public void test_compare_ADDA_without_complex_column()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
|
@ -115,13 +264,13 @@ public class ByteRowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("4", KeyOrder.DESCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS),
|
||||
sortUsingByteKeyComparator(keyColumns, ALL_KEY_OBJECTS)
|
||||
sortUsingObjectComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE),
|
||||
sortUsingByteKeyComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DADA()
|
||||
public void test_compare_DADA_without_complex_column()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
|
@ -130,8 +279,103 @@ public class ByteRowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS),
|
||||
sortUsingByteKeyComparator(keyColumns, ALL_KEY_OBJECTS)
|
||||
sortUsingObjectComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE),
|
||||
sortUsingByteKeyComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DDDDDDDD() // DDDDDDDD = all descending
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
new KeyColumn("2", KeyOrder.DESCENDING),
|
||||
new KeyColumn("3", KeyOrder.DESCENDING),
|
||||
new KeyColumn("4", KeyOrder.DESCENDING),
|
||||
new KeyColumn("5", KeyOrder.DESCENDING),
|
||||
new KeyColumn("6", KeyOrder.DESCENDING),
|
||||
new KeyColumn("7", KeyOrder.DESCENDING),
|
||||
new KeyColumn("8", KeyOrder.DESCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE),
|
||||
sortUsingByteKeyComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_AAAAAAAA() // AAAAAAAA = all ascending
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING),
|
||||
new KeyColumn("5", KeyOrder.ASCENDING),
|
||||
new KeyColumn("6", KeyOrder.ASCENDING),
|
||||
new KeyColumn("7", KeyOrder.ASCENDING),
|
||||
new KeyColumn("8", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE),
|
||||
sortUsingByteKeyComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_ADDADDAA()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.DESCENDING),
|
||||
new KeyColumn("3", KeyOrder.DESCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING),
|
||||
new KeyColumn("5", KeyOrder.DESCENDING),
|
||||
new KeyColumn("6", KeyOrder.DESCENDING),
|
||||
new KeyColumn("7", KeyOrder.ASCENDING),
|
||||
new KeyColumn("8", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE),
|
||||
sortUsingByteKeyComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DAADAADD()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING),
|
||||
new KeyColumn("4", KeyOrder.DESCENDING),
|
||||
new KeyColumn("5", KeyOrder.ASCENDING),
|
||||
new KeyColumn("6", KeyOrder.ASCENDING),
|
||||
new KeyColumn("7", KeyOrder.DESCENDING),
|
||||
new KeyColumn("8", KeyOrder.DESCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE),
|
||||
sortUsingByteKeyComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DADADADA()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.DESCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING),
|
||||
new KeyColumn("5", KeyOrder.DESCENDING),
|
||||
new KeyColumn("6", KeyOrder.ASCENDING),
|
||||
new KeyColumn("7", KeyOrder.DESCENDING),
|
||||
new KeyColumn("8", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE),
|
||||
sortUsingByteKeyComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -143,16 +387,24 @@ public class ByteRowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
.verify();
|
||||
}
|
||||
|
||||
private List<RowKey> sortUsingByteKeyComparator(final List<KeyColumn> keyColumns, final List<Object[]> objectss)
|
||||
private static List<RowKey> sortUsingByteKeyComparator(
|
||||
final List<KeyColumn> keyColumns,
|
||||
final List<Object[]> objectss,
|
||||
final RowSignature rowSignature
|
||||
)
|
||||
{
|
||||
return objectss.stream()
|
||||
.map(objects -> KeyTestUtils.createKey(SIGNATURE, objects).array())
|
||||
.sorted(ByteRowKeyComparator.create(keyColumns))
|
||||
.map(objects -> KeyTestUtils.createKey(rowSignature, objects).array())
|
||||
.sorted(ByteRowKeyComparator.create(keyColumns, rowSignature))
|
||||
.map(RowKey::wrap)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private List<RowKey> sortUsingObjectComparator(final List<KeyColumn> keyColumns, final List<Object[]> objectss)
|
||||
private static List<RowKey> sortUsingObjectComparator(
|
||||
final List<KeyColumn> keyColumns,
|
||||
final List<Object[]> objectss,
|
||||
final RowSignature rowSignature
|
||||
)
|
||||
{
|
||||
final List<Object[]> sortedObjectssCopy = objectss.stream().sorted(
|
||||
(o1, o2) -> {
|
||||
|
@ -174,9 +426,20 @@ public class ByteRowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
final List<RowKey> sortedKeys = new ArrayList<>();
|
||||
|
||||
for (final Object[] objects : sortedObjectssCopy) {
|
||||
sortedKeys.add(KeyTestUtils.createKey(SIGNATURE, objects));
|
||||
sortedKeys.add(KeyTestUtils.createKey(rowSignature, objects));
|
||||
}
|
||||
|
||||
return sortedKeys;
|
||||
}
|
||||
|
||||
public static HyperLogLogCollector makeHllCollector(final int estimatedCardinality)
|
||||
{
|
||||
final HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector();
|
||||
|
||||
for (int i = 0; i < estimatedCardinality; ++i) {
|
||||
collector.add(Hashing.murmur3_128().hashBytes(StringUtils.toUtf8(String.valueOf(i))).asBytes());
|
||||
}
|
||||
|
||||
return collector;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,8 @@ package org.apache.druid.frame.key;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||
import org.apache.druid.java.util.common.guava.Comparators;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -37,21 +39,30 @@ public class ClusterByTest
|
|||
new KeyColumn("y", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final RowSignature rowSignature = RowSignature.builder()
|
||||
.add("x", ColumnType.LONG)
|
||||
.add("y", ColumnType.LONG)
|
||||
.build();
|
||||
|
||||
Assert.assertEquals(
|
||||
RowKeyComparator.create(keyColumns),
|
||||
new ClusterBy(keyColumns, 1).keyComparator()
|
||||
RowKeyComparator.create(keyColumns, rowSignature),
|
||||
new ClusterBy(keyColumns, 1).keyComparator(rowSignature)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_bucketComparator_noKey()
|
||||
{
|
||||
Assert.assertSame(Comparators.alwaysEqual(), ClusterBy.none().bucketComparator());
|
||||
Assert.assertSame(Comparators.alwaysEqual(), ClusterBy.none().bucketComparator(RowSignature.empty()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_bucketComparator_noBucketKey()
|
||||
{
|
||||
RowSignature rowSignature = RowSignature.builder()
|
||||
.add("x", ColumnType.LONG)
|
||||
.add("y", ColumnType.LONG)
|
||||
.build();
|
||||
Assert.assertSame(
|
||||
Comparators.alwaysEqual(),
|
||||
new ClusterBy(
|
||||
|
@ -60,22 +71,30 @@ public class ClusterByTest
|
|||
new KeyColumn("y", KeyOrder.ASCENDING)
|
||||
),
|
||||
0
|
||||
).bucketComparator()
|
||||
).bucketComparator(rowSignature)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_bucketComparator_withBucketKey()
|
||||
{
|
||||
RowSignature rowSignature = RowSignature.builder()
|
||||
.add("x", ColumnType.LONG)
|
||||
.add("y", ColumnType.LONG)
|
||||
.build();
|
||||
Assert.assertEquals(
|
||||
RowKeyComparator.create(ImmutableList.of(new KeyColumn("x", KeyOrder.ASCENDING))),
|
||||
RowKeyComparator.create(
|
||||
ImmutableList.of(new KeyColumn("x", KeyOrder.ASCENDING)),
|
||||
rowSignature
|
||||
|
||||
),
|
||||
new ClusterBy(
|
||||
ImmutableList.of(
|
||||
new KeyColumn("x", KeyOrder.ASCENDING),
|
||||
new KeyColumn("y", KeyOrder.ASCENDING)
|
||||
),
|
||||
1
|
||||
).bucketComparator()
|
||||
).bucketComparator(rowSignature)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -44,31 +44,215 @@ import java.util.stream.Collectors;
|
|||
|
||||
public class FrameComparisonWidgetImplTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private Frame frame;
|
||||
private Frame frameWithoutComplexColumns;
|
||||
private Frame frameWithComplexColumns;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
final StorageAdapter rowBasedAdapter = new RowBasedSegment<>(
|
||||
final StorageAdapter rowBasedAdapterWithoutComplexColumn = new RowBasedSegment<>(
|
||||
SegmentId.dummy("test"),
|
||||
Sequences.simple(RowKeyComparatorTest.ALL_KEY_OBJECTS),
|
||||
Sequences.simple(ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN),
|
||||
columnName -> {
|
||||
final int idx = RowKeyComparatorTest.SIGNATURE.getColumnNames().indexOf(columnName);
|
||||
final int idx = ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE.getColumnNames().indexOf(columnName);
|
||||
if (idx < 0) {
|
||||
return row -> null;
|
||||
} else {
|
||||
return row -> row[idx];
|
||||
}
|
||||
},
|
||||
RowKeyComparatorTest.SIGNATURE
|
||||
ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE
|
||||
).asStorageAdapter();
|
||||
|
||||
frame = Iterables.getOnlyElement(
|
||||
FrameSequenceBuilder.fromAdapter(rowBasedAdapter)
|
||||
frameWithoutComplexColumns = Iterables.getOnlyElement(
|
||||
FrameSequenceBuilder.fromAdapter(rowBasedAdapterWithoutComplexColumn)
|
||||
.frameType(FrameType.ROW_BASED)
|
||||
.frames()
|
||||
.toList()
|
||||
);
|
||||
|
||||
final StorageAdapter rowBasedAdapterWithComplexColumn = new RowBasedSegment<>(
|
||||
SegmentId.dummy("test"),
|
||||
Sequences.simple(ByteRowKeyComparatorTest.ALL_KEY_OBJECTS),
|
||||
columnName -> {
|
||||
final int idx = ByteRowKeyComparatorTest.SIGNATURE.getColumnNames().indexOf(columnName);
|
||||
if (idx < 0) {
|
||||
return row -> null;
|
||||
} else {
|
||||
return row -> row[idx];
|
||||
}
|
||||
},
|
||||
ByteRowKeyComparatorTest.SIGNATURE
|
||||
).asStorageAdapter();
|
||||
|
||||
frameWithComplexColumns = Iterables.getOnlyElement(
|
||||
FrameSequenceBuilder.fromAdapter(rowBasedAdapterWithComplexColumn)
|
||||
.frameType(FrameType.ROW_BASED)
|
||||
.frames()
|
||||
.toList()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_noComplexColumns_isPartiallyNullKey_someColumns()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final FrameComparisonWidget widget = createComparisonWidget(
|
||||
frameWithoutComplexColumns,
|
||||
keyColumns,
|
||||
ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE
|
||||
);
|
||||
|
||||
for (int i = 0; i < frameWithoutComplexColumns.numRows(); i++) {
|
||||
final boolean isAllNonNull =
|
||||
Arrays.stream(ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN.get(i))
|
||||
.limit(3)
|
||||
.allMatch(Objects::nonNull);
|
||||
|
||||
// null key part, if any, is always the second one (1)
|
||||
Assert.assertTrue(widget.hasNonNullKeyParts(i, new int[0]));
|
||||
Assert.assertTrue(widget.hasNonNullKeyParts(i, new int[]{0, 2}));
|
||||
Assert.assertEquals(isAllNonNull, widget.hasNonNullKeyParts(i, new int[]{0, 1, 2}));
|
||||
Assert.assertEquals(isAllNonNull, widget.hasNonNullKeyParts(i, new int[]{1}));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_noComplexColumns_isPartiallyNullKey_allColumns()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final FrameComparisonWidget widget = createComparisonWidget(
|
||||
frameWithoutComplexColumns,
|
||||
keyColumns,
|
||||
ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE
|
||||
);
|
||||
|
||||
for (int i = 0; i < frameWithoutComplexColumns.numRows(); i++) {
|
||||
final boolean isAllNonNull =
|
||||
Arrays.stream(ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN.get(i)).allMatch(Objects::nonNull);
|
||||
Assert.assertEquals(isAllNonNull, widget.hasNonNullKeyParts(i, new int[]{0, 1, 2, 3}));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_noComplexColumns_readKey_someColumns()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final FrameComparisonWidget widget = createComparisonWidget(
|
||||
frameWithoutComplexColumns,
|
||||
keyColumns,
|
||||
ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE
|
||||
);
|
||||
|
||||
final RowSignature signature =
|
||||
RowSignature.builder()
|
||||
.add("1", ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE.getColumnType("1").orElse(null))
|
||||
.add("2", ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE.getColumnType("2").orElse(null))
|
||||
.add("3", ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE.getColumnType("3").orElse(null))
|
||||
.build();
|
||||
|
||||
for (int i = 0; i < frameWithoutComplexColumns.numRows(); i++) {
|
||||
final Object[] expectedKeyArray = new Object[keyColumns.size()];
|
||||
System.arraycopy(
|
||||
ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN.get(i),
|
||||
0,
|
||||
expectedKeyArray,
|
||||
0,
|
||||
keyColumns.size()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
KeyTestUtils.createKey(signature, expectedKeyArray),
|
||||
widget.readKey(i)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_noComplexColumns_readKey_allColumns()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final FrameComparisonWidget widget = createComparisonWidget(
|
||||
frameWithoutComplexColumns,
|
||||
keyColumns,
|
||||
ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE
|
||||
);
|
||||
|
||||
for (int i = 0; i < frameWithoutComplexColumns.numRows(); i++) {
|
||||
Assert.assertEquals(
|
||||
KeyTestUtils.createKey(
|
||||
ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE,
|
||||
ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN.get(i)
|
||||
),
|
||||
widget.readKey(i)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_noComplexColumns_compare_frameToKey()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final FrameComparisonWidget widget = createComparisonWidget(
|
||||
frameWithoutComplexColumns,
|
||||
keyColumns,
|
||||
ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE
|
||||
);
|
||||
|
||||
// Compare self-to-self should be equal.
|
||||
for (int i = 0; i < frameWithoutComplexColumns.numRows(); i++) {
|
||||
Assert.assertEquals(
|
||||
0,
|
||||
widget.compare(
|
||||
i,
|
||||
KeyTestUtils.createKey(
|
||||
ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE,
|
||||
ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN.get(i)
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
// Check some other comparators.
|
||||
final RowKey firstKey = KeyTestUtils.createKey(
|
||||
ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE,
|
||||
ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN.get(0)
|
||||
);
|
||||
|
||||
MatcherAssert.assertThat(widget.compare(0, firstKey), Matchers.equalTo(0));
|
||||
MatcherAssert.assertThat(widget.compare(1, firstKey), Matchers.lessThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(2, firstKey), Matchers.lessThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(3, firstKey), Matchers.greaterThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(4, firstKey), Matchers.greaterThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(5, firstKey), Matchers.greaterThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(6, firstKey), Matchers.greaterThan(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -80,17 +264,24 @@ public class FrameComparisonWidgetImplTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("3", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final FrameComparisonWidget widget = createComparisonWidget(keyColumns);
|
||||
final FrameComparisonWidget widget = createComparisonWidget(
|
||||
frameWithComplexColumns,
|
||||
keyColumns,
|
||||
ByteRowKeyComparatorTest.SIGNATURE
|
||||
);
|
||||
|
||||
for (int i = 0; i < frame.numRows(); i++) {
|
||||
for (int i = 0; i < frameWithComplexColumns.numRows(); i++) {
|
||||
final boolean isAllNonNull =
|
||||
Arrays.stream(RowKeyComparatorTest.ALL_KEY_OBJECTS.get(i)).limit(3).allMatch(Objects::nonNull);
|
||||
Arrays.stream(ByteRowKeyComparatorTest.ALL_KEY_OBJECTS.get(i)).limit(3).allMatch(Objects::nonNull);
|
||||
|
||||
// null key part, if any, is always the second one (1)
|
||||
Assert.assertTrue(widget.hasNonNullKeyParts(i, new int[0]));
|
||||
Assert.assertTrue(widget.hasNonNullKeyParts(i, new int[]{0, 2}));
|
||||
// Only second is non-null throughout
|
||||
Assert.assertTrue(widget.hasNonNullKeyParts(i, new int[]{1}));
|
||||
Assert.assertTrue(widget.hasNonNullKeyParts(i, new int[]{1}));
|
||||
Assert.assertEquals(isAllNonNull, widget.hasNonNullKeyParts(i, new int[]{0, 1, 2}));
|
||||
Assert.assertEquals(isAllNonNull, widget.hasNonNullKeyParts(i, new int[]{1}));
|
||||
Assert.assertEquals(
|
||||
isAllNonNull,
|
||||
widget.hasNonNullKeyParts(i, new int[]{0}) && widget.hasNonNullKeyParts(i, new int[]{2})
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -101,15 +292,23 @@ public class FrameComparisonWidgetImplTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
new KeyColumn("4", KeyOrder.ASCENDING),
|
||||
new KeyColumn("5", KeyOrder.ASCENDING),
|
||||
new KeyColumn("6", KeyOrder.ASCENDING),
|
||||
new KeyColumn("7", KeyOrder.ASCENDING),
|
||||
new KeyColumn("8", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final FrameComparisonWidget widget = createComparisonWidget(keyColumns);
|
||||
final FrameComparisonWidget widget = createComparisonWidget(
|
||||
frameWithComplexColumns,
|
||||
keyColumns,
|
||||
ByteRowKeyComparatorTest.SIGNATURE
|
||||
);
|
||||
|
||||
for (int i = 0; i < frame.numRows(); i++) {
|
||||
for (int i = 0; i < frameWithoutComplexColumns.numRows(); i++) {
|
||||
final boolean isAllNonNull =
|
||||
Arrays.stream(RowKeyComparatorTest.ALL_KEY_OBJECTS.get(i)).allMatch(Objects::nonNull);
|
||||
Assert.assertEquals(isAllNonNull, widget.hasNonNullKeyParts(i, new int[]{0, 1, 2, 3}));
|
||||
Arrays.stream(ByteRowKeyComparatorTest.ALL_KEY_OBJECTS.get(i)).allMatch(Objects::nonNull);
|
||||
Assert.assertEquals(isAllNonNull, widget.hasNonNullKeyParts(i, new int[]{0, 1, 2, 3, 4, 5, 6, 7}));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -122,18 +321,22 @@ public class FrameComparisonWidgetImplTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("3", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final FrameComparisonWidget widget = createComparisonWidget(keyColumns);
|
||||
final FrameComparisonWidget widget = createComparisonWidget(
|
||||
frameWithComplexColumns,
|
||||
keyColumns,
|
||||
ByteRowKeyComparatorTest.SIGNATURE
|
||||
);
|
||||
|
||||
final RowSignature signature =
|
||||
RowSignature.builder()
|
||||
.add("1", RowKeyComparatorTest.SIGNATURE.getColumnType("1").orElse(null))
|
||||
.add("2", RowKeyComparatorTest.SIGNATURE.getColumnType("2").orElse(null))
|
||||
.add("3", RowKeyComparatorTest.SIGNATURE.getColumnType("3").orElse(null))
|
||||
.add("1", ByteRowKeyComparatorTest.SIGNATURE.getColumnType("1").orElse(null))
|
||||
.add("2", ByteRowKeyComparatorTest.SIGNATURE.getColumnType("2").orElse(null))
|
||||
.add("3", ByteRowKeyComparatorTest.SIGNATURE.getColumnType("3").orElse(null))
|
||||
.build();
|
||||
|
||||
for (int i = 0; i < frame.numRows(); i++) {
|
||||
for (int i = 0; i < frameWithComplexColumns.numRows(); i++) {
|
||||
final Object[] expectedKeyArray = new Object[keyColumns.size()];
|
||||
System.arraycopy(RowKeyComparatorTest.ALL_KEY_OBJECTS.get(i), 0, expectedKeyArray, 0, keyColumns.size());
|
||||
System.arraycopy(ByteRowKeyComparatorTest.ALL_KEY_OBJECTS.get(i), 0, expectedKeyArray, 0, keyColumns.size());
|
||||
Assert.assertEquals(
|
||||
KeyTestUtils.createKey(signature, expectedKeyArray),
|
||||
widget.readKey(i)
|
||||
|
@ -148,14 +351,22 @@ public class FrameComparisonWidgetImplTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
new KeyColumn("4", KeyOrder.ASCENDING),
|
||||
new KeyColumn("5", KeyOrder.ASCENDING),
|
||||
new KeyColumn("6", KeyOrder.ASCENDING),
|
||||
new KeyColumn("7", KeyOrder.ASCENDING),
|
||||
new KeyColumn("8", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final FrameComparisonWidget widget = createComparisonWidget(keyColumns);
|
||||
final FrameComparisonWidget widget = createComparisonWidget(
|
||||
frameWithComplexColumns,
|
||||
keyColumns,
|
||||
ByteRowKeyComparatorTest.SIGNATURE
|
||||
);
|
||||
|
||||
for (int i = 0; i < frame.numRows(); i++) {
|
||||
for (int i = 0; i < frameWithComplexColumns.numRows(); i++) {
|
||||
Assert.assertEquals(
|
||||
KeyTestUtils.createKey(RowKeyComparatorTest.SIGNATURE, RowKeyComparatorTest.ALL_KEY_OBJECTS.get(i)),
|
||||
KeyTestUtils.createKey(ByteRowKeyComparatorTest.SIGNATURE, ByteRowKeyComparatorTest.ALL_KEY_OBJECTS.get(i)),
|
||||
widget.readKey(i)
|
||||
);
|
||||
}
|
||||
|
@ -168,20 +379,28 @@ public class FrameComparisonWidgetImplTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
new KeyColumn("4", KeyOrder.ASCENDING),
|
||||
new KeyColumn("5", KeyOrder.ASCENDING),
|
||||
new KeyColumn("6", KeyOrder.ASCENDING),
|
||||
new KeyColumn("7", KeyOrder.ASCENDING),
|
||||
new KeyColumn("8", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final FrameComparisonWidget widget = createComparisonWidget(keyColumns);
|
||||
final FrameComparisonWidget widget = createComparisonWidget(
|
||||
frameWithComplexColumns,
|
||||
keyColumns,
|
||||
ByteRowKeyComparatorTest.SIGNATURE
|
||||
);
|
||||
|
||||
// Compare self-to-self should be equal.
|
||||
for (int i = 0; i < frame.numRows(); i++) {
|
||||
for (int i = 0; i < frameWithComplexColumns.numRows(); i++) {
|
||||
Assert.assertEquals(
|
||||
0,
|
||||
widget.compare(
|
||||
i,
|
||||
KeyTestUtils.createKey(
|
||||
RowKeyComparatorTest.SIGNATURE,
|
||||
RowKeyComparatorTest.ALL_KEY_OBJECTS.get(i)
|
||||
ByteRowKeyComparatorTest.SIGNATURE,
|
||||
ByteRowKeyComparatorTest.ALL_KEY_OBJECTS.get(i)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
@ -189,30 +408,43 @@ public class FrameComparisonWidgetImplTest extends InitializedNullHandlingTest
|
|||
|
||||
// Check some other comparators.
|
||||
final RowKey firstKey = KeyTestUtils.createKey(
|
||||
RowKeyComparatorTest.SIGNATURE,
|
||||
RowKeyComparatorTest.ALL_KEY_OBJECTS.get(0)
|
||||
ByteRowKeyComparatorTest.SIGNATURE,
|
||||
ByteRowKeyComparatorTest.ALL_KEY_OBJECTS.get(0)
|
||||
);
|
||||
|
||||
MatcherAssert.assertThat(widget.compare(0, firstKey), Matchers.equalTo(0));
|
||||
MatcherAssert.assertThat(widget.compare(1, firstKey), Matchers.lessThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(2, firstKey), Matchers.lessThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(2, firstKey), Matchers.greaterThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(3, firstKey), Matchers.greaterThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(4, firstKey), Matchers.greaterThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(5, firstKey), Matchers.greaterThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(6, firstKey), Matchers.greaterThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(7, firstKey), Matchers.greaterThan(0));
|
||||
MatcherAssert.assertThat(widget.compare(8, firstKey), Matchers.greaterThan(0));
|
||||
|
||||
final RowKey eighthKey = KeyTestUtils.createKey(
|
||||
ByteRowKeyComparatorTest.SIGNATURE,
|
||||
ByteRowKeyComparatorTest.ALL_KEY_OBJECTS.get(7)
|
||||
);
|
||||
|
||||
MatcherAssert.assertThat(widget.compare(8, eighthKey), Matchers.lessThan(0));
|
||||
}
|
||||
|
||||
private FrameComparisonWidget createComparisonWidget(final List<KeyColumn> keyColumns)
|
||||
private FrameComparisonWidget createComparisonWidget(
|
||||
final Frame frame,
|
||||
final List<KeyColumn> keyColumns,
|
||||
final RowSignature rowSignature
|
||||
)
|
||||
{
|
||||
return FrameComparisonWidgetImpl.create(
|
||||
frame,
|
||||
RowKeyComparatorTest.SIGNATURE,
|
||||
rowSignature,
|
||||
keyColumns,
|
||||
keyColumns.stream().map(
|
||||
keyColumn ->
|
||||
FieldReaders.create(
|
||||
keyColumn.columnName(),
|
||||
RowKeyComparatorTest.SIGNATURE.getColumnType(keyColumn.columnName()).get()
|
||||
rowSignature.getColumnType(keyColumn.columnName()).get()
|
||||
)
|
||||
).collect(Collectors.toList())
|
||||
);
|
||||
|
|
|
@ -22,46 +22,30 @@ package org.apache.druid.frame.key;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||
import org.apache.druid.java.util.common.guava.Comparators;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.druid.frame.key.ByteRowKeyComparatorTest.ALL_KEY_OBJECTS;
|
||||
import static org.apache.druid.frame.key.ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN;
|
||||
import static org.apache.druid.frame.key.ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE;
|
||||
import static org.apache.druid.frame.key.ByteRowKeyComparatorTest.SIGNATURE;
|
||||
|
||||
public class RowKeyComparatorTest extends InitializedNullHandlingTest
|
||||
{
|
||||
static final RowSignature SIGNATURE =
|
||||
RowSignature.builder()
|
||||
.add("1", ColumnType.LONG)
|
||||
.add("2", ColumnType.STRING)
|
||||
.add("3", ColumnType.LONG)
|
||||
.add("4", ColumnType.DOUBLE)
|
||||
.build();
|
||||
private static final Object[] OBJECTS1 = new Object[]{-1L, "foo", 2L, -1.2};
|
||||
private static final Object[] OBJECTS2 = new Object[]{-1L, null, 2L, 1.2d};
|
||||
private static final Object[] OBJECTS3 = new Object[]{-1L, "bar", 2L, 1.2d};
|
||||
private static final Object[] OBJECTS4 = new Object[]{-1L, "foo", 2L, 1.2d};
|
||||
private static final Object[] OBJECTS5 = new Object[]{-1L, "foo", 3L, 1.2d};
|
||||
private static final Object[] OBJECTS6 = new Object[]{-1L, "foo", 2L, 1.3d};
|
||||
private static final Object[] OBJECTS7 = new Object[]{1L, "foo", 2L, -1.2d};
|
||||
|
||||
static final List<Object[]> ALL_KEY_OBJECTS = Arrays.asList(
|
||||
OBJECTS1,
|
||||
OBJECTS2,
|
||||
OBJECTS3,
|
||||
OBJECTS4,
|
||||
OBJECTS5,
|
||||
OBJECTS6,
|
||||
OBJECTS7
|
||||
);
|
||||
static {
|
||||
ComplexMetrics.registerSerde(HyperUniquesSerde.TYPE_NAME, new HyperUniquesSerde());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_AAAA() // AAAA = all ascending
|
||||
public void test_compare_AAAA_without_complex_column() // AAAA = all ascending, no complex column
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
|
@ -70,13 +54,13 @@ public class RowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("4", KeyOrder.DESCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS),
|
||||
sortUsingKeyComparator(keyColumns, ALL_KEY_OBJECTS)
|
||||
sortUsingObjectComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE),
|
||||
sortUsingKeyComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DDDD() // DDDD = all descending
|
||||
public void test_compare_DDDD_without_complex_column() // DDDD = all descending, no complex columns
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
|
@ -85,13 +69,13 @@ public class RowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS),
|
||||
sortUsingKeyComparator(keyColumns, ALL_KEY_OBJECTS)
|
||||
sortUsingObjectComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE),
|
||||
sortUsingKeyComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DAAD()
|
||||
public void test_compare_DAAD_without_complex_column()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
|
@ -100,13 +84,13 @@ public class RowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS),
|
||||
sortUsingKeyComparator(keyColumns, ALL_KEY_OBJECTS)
|
||||
sortUsingObjectComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE),
|
||||
sortUsingKeyComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_ADDA()
|
||||
public void test_compare_ADDA_without_complex_column()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
|
@ -115,13 +99,13 @@ public class RowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("4", KeyOrder.DESCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS),
|
||||
sortUsingKeyComparator(keyColumns, ALL_KEY_OBJECTS)
|
||||
sortUsingObjectComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE),
|
||||
sortUsingKeyComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DADA()
|
||||
public void test_compare_DADA_without_complex_column()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
|
@ -130,8 +114,103 @@ public class RowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
new KeyColumn("4", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS),
|
||||
sortUsingKeyComparator(keyColumns, ALL_KEY_OBJECTS)
|
||||
sortUsingObjectComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE),
|
||||
sortUsingKeyComparator(keyColumns, KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN, NO_COMPLEX_SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DDDDDDDD() // DDDDDDDD = all descending
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
new KeyColumn("2", KeyOrder.DESCENDING),
|
||||
new KeyColumn("3", KeyOrder.DESCENDING),
|
||||
new KeyColumn("4", KeyOrder.DESCENDING),
|
||||
new KeyColumn("5", KeyOrder.DESCENDING),
|
||||
new KeyColumn("6", KeyOrder.DESCENDING),
|
||||
new KeyColumn("7", KeyOrder.DESCENDING),
|
||||
new KeyColumn("8", KeyOrder.DESCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE),
|
||||
sortUsingKeyComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_AAAAAAAA() // AAAAAAAA = all ascending
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING),
|
||||
new KeyColumn("5", KeyOrder.ASCENDING),
|
||||
new KeyColumn("6", KeyOrder.ASCENDING),
|
||||
new KeyColumn("7", KeyOrder.ASCENDING),
|
||||
new KeyColumn("8", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE),
|
||||
sortUsingKeyComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_ADDADDAA()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("2", KeyOrder.DESCENDING),
|
||||
new KeyColumn("3", KeyOrder.DESCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING),
|
||||
new KeyColumn("5", KeyOrder.DESCENDING),
|
||||
new KeyColumn("6", KeyOrder.DESCENDING),
|
||||
new KeyColumn("7", KeyOrder.ASCENDING),
|
||||
new KeyColumn("8", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE),
|
||||
sortUsingKeyComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DAADAADD()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.ASCENDING),
|
||||
new KeyColumn("4", KeyOrder.DESCENDING),
|
||||
new KeyColumn("5", KeyOrder.ASCENDING),
|
||||
new KeyColumn("6", KeyOrder.ASCENDING),
|
||||
new KeyColumn("7", KeyOrder.DESCENDING),
|
||||
new KeyColumn("8", KeyOrder.DESCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE),
|
||||
sortUsingKeyComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_compare_DADADADA()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("1", KeyOrder.DESCENDING),
|
||||
new KeyColumn("2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("3", KeyOrder.DESCENDING),
|
||||
new KeyColumn("4", KeyOrder.ASCENDING),
|
||||
new KeyColumn("5", KeyOrder.DESCENDING),
|
||||
new KeyColumn("6", KeyOrder.ASCENDING),
|
||||
new KeyColumn("7", KeyOrder.DESCENDING),
|
||||
new KeyColumn("8", KeyOrder.ASCENDING)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
sortUsingObjectComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE),
|
||||
sortUsingKeyComparator(keyColumns, ALL_KEY_OBJECTS, SIGNATURE)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -144,19 +223,27 @@ public class RowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
.verify();
|
||||
}
|
||||
|
||||
private List<RowKey> sortUsingKeyComparator(final List<KeyColumn> keyColumns, final List<Object[]> objectss)
|
||||
private static List<RowKey> sortUsingKeyComparator(
|
||||
final List<KeyColumn> keyColumns,
|
||||
final List<Object[]> objectss,
|
||||
final RowSignature rowSignature
|
||||
)
|
||||
{
|
||||
final List<RowKey> sortedKeys = new ArrayList<>();
|
||||
|
||||
for (final Object[] objects : objectss) {
|
||||
sortedKeys.add(KeyTestUtils.createKey(SIGNATURE, objects));
|
||||
sortedKeys.add(KeyTestUtils.createKey(rowSignature, objects));
|
||||
}
|
||||
|
||||
sortedKeys.sort(RowKeyComparator.create(keyColumns));
|
||||
sortedKeys.sort(RowKeyComparator.create(keyColumns, rowSignature));
|
||||
return sortedKeys;
|
||||
}
|
||||
|
||||
private List<RowKey> sortUsingObjectComparator(final List<KeyColumn> keyColumns, final List<Object[]> objectss)
|
||||
private static List<RowKey> sortUsingObjectComparator(
|
||||
final List<KeyColumn> keyColumns,
|
||||
final List<Object[]> objectss,
|
||||
final RowSignature rowSignature
|
||||
)
|
||||
{
|
||||
final List<Object[]> sortedObjectssCopy = objectss.stream().sorted(
|
||||
(o1, o2) -> {
|
||||
|
@ -178,7 +265,7 @@ public class RowKeyComparatorTest extends InitializedNullHandlingTest
|
|||
final List<RowKey> sortedKeys = new ArrayList<>();
|
||||
|
||||
for (final Object[] objects : sortedObjectssCopy) {
|
||||
sortedKeys.add(KeyTestUtils.createKey(SIGNATURE, objects));
|
||||
sortedKeys.add(KeyTestUtils.createKey(rowSignature, objects));
|
||||
}
|
||||
|
||||
return sortedKeys;
|
||||
|
|
|
@ -0,0 +1,809 @@
|
|||
/*
|
||||
* 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.frame.key;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
public class RowKeyComparisonRunLengthsTest
|
||||
{
|
||||
|
||||
@Test
|
||||
public void testRunLengthsWithNoKeyColumns()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = Collections.emptyList();
|
||||
final RowSignature signature = RowSignature.empty();
|
||||
final RowKeyComparisonRunLengths runLengths = RowKeyComparisonRunLengths.create(keyColumns, signature);
|
||||
Assert.assertEquals(0, runLengths.getRunLengthEntries().length);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunLengthsWithInvalidOrder()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.NONE));
|
||||
final RowSignature signature = RowSignature.builder()
|
||||
.add("a", ColumnType.LONG)
|
||||
.build();
|
||||
Assert.assertThrows(DruidException.class, () -> RowKeyComparisonRunLengths.create(keyColumns, signature));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunLengthsWithIncompleteRowSignature()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.NONE));
|
||||
final RowSignature signature = RowSignature.empty();
|
||||
Assert.assertThrows(DruidException.class, () -> RowKeyComparisonRunLengths.create(keyColumns, signature));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunLengthsWithEmptyType()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.NONE));
|
||||
final RowSignature signature1 = RowSignature.builder()
|
||||
.add("a", null)
|
||||
.build();
|
||||
Assert.assertThrows(DruidException.class, () -> RowKeyComparisonRunLengths.create(keyColumns, signature1));
|
||||
|
||||
final RowSignature signature2 = RowSignature.builder()
|
||||
.add("a", ColumnType.UNKNOWN_COMPLEX)
|
||||
.build();
|
||||
Assert.assertThrows(DruidException.class, () -> RowKeyComparisonRunLengths.create(keyColumns, signature2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunLengthsWithByteComparableTypes()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.ASCENDING));
|
||||
final List<ColumnType> byteComparableTypes = ImmutableList.of(
|
||||
ColumnType.LONG,
|
||||
ColumnType.FLOAT,
|
||||
ColumnType.DOUBLE,
|
||||
ColumnType.STRING,
|
||||
ColumnType.LONG_ARRAY,
|
||||
ColumnType.FLOAT_ARRAY,
|
||||
ColumnType.DOUBLE_ARRAY,
|
||||
ColumnType.STRING_ARRAY
|
||||
);
|
||||
|
||||
for (final ColumnType columnType : byteComparableTypes) {
|
||||
final RowSignature signature = RowSignature.builder()
|
||||
.add("a", columnType)
|
||||
.build();
|
||||
final RowKeyComparisonRunLengths runLengths = RowKeyComparisonRunLengths.create(keyColumns, signature);
|
||||
Assert.assertEquals(1, runLengths.getRunLengthEntries().length);
|
||||
Assert.assertTrue(runLengths.getRunLengthEntries()[0].isByteComparable());
|
||||
Assert.assertEquals(1, runLengths.getRunLengthEntries()[0].getRunLength());
|
||||
Assert.assertEquals(KeyOrder.ASCENDING, runLengths.getRunLengthEntries()[0].getOrder());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunLengthsWithNonByteComparableTypes()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.ASCENDING));
|
||||
// Any known complex type
|
||||
final List<ColumnType> byteComparableTypes = ImmutableList.of(ColumnType.NESTED_DATA);
|
||||
|
||||
for (final ColumnType columnType : byteComparableTypes) {
|
||||
final RowSignature signature = RowSignature.builder()
|
||||
.add("a", columnType)
|
||||
.build();
|
||||
final RowKeyComparisonRunLengths runLengths = RowKeyComparisonRunLengths.create(keyColumns, signature);
|
||||
Assert.assertEquals(1, runLengths.getRunLengthEntries().length);
|
||||
Assert.assertFalse(runLengths.getRunLengthEntries()[0].isByteComparable());
|
||||
Assert.assertEquals(1, runLengths.getRunLengthEntries()[0].getRunLength());
|
||||
Assert.assertEquals(KeyOrder.ASCENDING, runLengths.getRunLengthEntries()[0].getOrder());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunLengthsWithMultipleColumns()
|
||||
{
|
||||
final List<KeyColumn> keyColumns = ImmutableList.of(
|
||||
new KeyColumn("longAsc1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("stringAsc1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("stringDesc1", KeyOrder.DESCENDING),
|
||||
new KeyColumn("longDesc1", KeyOrder.DESCENDING),
|
||||
new KeyColumn("complexDesc1", KeyOrder.DESCENDING),
|
||||
new KeyColumn("complexAsc1", KeyOrder.ASCENDING),
|
||||
new KeyColumn("complexAsc2", KeyOrder.ASCENDING),
|
||||
new KeyColumn("stringAsc2", KeyOrder.ASCENDING)
|
||||
);
|
||||
|
||||
final RowSignature signature = RowSignature.builder()
|
||||
.add("longAsc1", ColumnType.LONG)
|
||||
.add("stringAsc1", ColumnType.STRING)
|
||||
.add("stringDesc1", ColumnType.STRING)
|
||||
.add("longDesc1", ColumnType.LONG)
|
||||
.add("complexDesc1", ColumnType.NESTED_DATA)
|
||||
.add("complexAsc1", ColumnType.NESTED_DATA)
|
||||
.add("complexAsc2", ColumnType.NESTED_DATA)
|
||||
.add("stringAsc2", ColumnType.STRING)
|
||||
.build();
|
||||
|
||||
final RunLengthEntry[] runLengthEntries =
|
||||
RowKeyComparisonRunLengths.create(keyColumns, signature).getRunLengthEntries();
|
||||
|
||||
// Input keyColumns
|
||||
// long ASC, string ASC, string DESC, long DESC, complex DESC, complex ASC, complex ASC, string ASC
|
||||
|
||||
// Output runLengthEntries would be
|
||||
// (long, string ASC) (string, long DESC) (complex DESC) (complex ASC) (complex ASC) (string ASC)
|
||||
|
||||
Assert.assertEquals(6, runLengthEntries.length);
|
||||
|
||||
Assert.assertTrue(runLengthEntries[0].isByteComparable());
|
||||
Assert.assertEquals(2, runLengthEntries[0].getRunLength());
|
||||
Assert.assertEquals(KeyOrder.ASCENDING, runLengthEntries[0].getOrder());
|
||||
|
||||
Assert.assertTrue(runLengthEntries[1].isByteComparable());
|
||||
Assert.assertEquals(2, runLengthEntries[1].getRunLength());
|
||||
Assert.assertEquals(KeyOrder.DESCENDING, runLengthEntries[1].getOrder());
|
||||
|
||||
Assert.assertFalse(runLengthEntries[2].isByteComparable());
|
||||
Assert.assertEquals(1, runLengthEntries[2].getRunLength());
|
||||
Assert.assertEquals(KeyOrder.DESCENDING, runLengthEntries[2].getOrder());
|
||||
|
||||
Assert.assertFalse(runLengthEntries[3].isByteComparable());
|
||||
Assert.assertEquals(1, runLengthEntries[3].getRunLength());
|
||||
Assert.assertEquals(KeyOrder.ASCENDING, runLengthEntries[3].getOrder());
|
||||
|
||||
Assert.assertFalse(runLengthEntries[4].isByteComparable());
|
||||
Assert.assertEquals(1, runLengthEntries[4].getRunLength());
|
||||
Assert.assertEquals(KeyOrder.ASCENDING, runLengthEntries[4].getOrder());
|
||||
|
||||
Assert.assertTrue(runLengthEntries[5].isByteComparable());
|
||||
Assert.assertEquals(1, runLengthEntries[5].getRunLength());
|
||||
Assert.assertEquals(KeyOrder.ASCENDING, runLengthEntries[5].getOrder());
|
||||
}
|
||||
|
||||
/**
|
||||
* This tests the creation of the run lengths with all the permutations of the key columns from the following space:
|
||||
* a. The KeyColumn can be either String or Complex (string is byte-comparable, nested data is not)
|
||||
* b. The KeyColumn can be either ASC or DESC
|
||||
*
|
||||
* Therefore, each key column can be one of (string ASC, string DESC, complex ASC, complex DESC). There are 64 test
|
||||
* case for all the permutations of the key columns, because there are 3 key columns, each of which can take one of
|
||||
* the 4 different configurations..
|
||||
*
|
||||
* Test cases are generated programatically. For index i from [0..64), we build the base-4 representation of the index,
|
||||
* and each digit in the representation corresponds to one of the key columns.
|
||||
*/
|
||||
@Test
|
||||
public void testRunLengthsWithAllPermutationsOfThreeLengthKeyColumns()
|
||||
{
|
||||
|
||||
ImmutableList.Builder<RunLengthEntry[]> expectedResultsBuilder = ImmutableList.builder();
|
||||
|
||||
// index = 0; KeyColumns = STRING ASC, STRING ASC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 3)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 1; KeyColumns = STRING DESC, STRING ASC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 2)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 2; KeyColumns = COMPLEX ASC, STRING ASC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 2)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 3; KeyColumns = COMPLEX DESC, STRING ASC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 2)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 4; KeyColumns = STRING ASC, STRING DESC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 5; KeyColumns = STRING DESC, STRING DESC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 2),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 6; KeyColumns = COMPLEX ASC, STRING DESC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 7; KeyColumns = COMPLEX DESC, STRING DESC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 8; KeyColumns = STRING ASC, COMPLEX ASC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 9; KeyColumns = STRING DESC, COMPLEX ASC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 10; KeyColumns = COMPLEX ASC, COMPLEX ASC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
}
|
||||
);
|
||||
|
||||
// index = 11; KeyColumns = COMPLEX DESC, COMPLEX ASC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 12; KeyColumns = STRING ASC, COMPLEX DESC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 13; KeyColumns = STRING DESC, COMPLEX DESC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 14; KeyColumns = COMPLEX ASC, COMPLEX DESC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
}
|
||||
);
|
||||
|
||||
// index = 15; KeyColumns = COMPLEX DESC, COMPLEX DESC, STRING ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 16; KeyColumns = STRING ASC, STRING ASC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 2),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
}
|
||||
);
|
||||
|
||||
// index = 17; KeyColumns = STRING DESC, STRING ASC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 18; KeyColumns = COMPLEX ASC, STRING ASC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 19; KeyColumns = COMPLEX DESC, STRING ASC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 20; KeyColumns = STRING ASC, STRING DESC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 2)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 21; KeyColumns = STRING DESC, STRING DESC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 3)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 22; KeyColumns = COMPLEX ASC, STRING DESC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 2)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 23; KeyColumns = COMPLEX DESC, STRING DESC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 2)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 24; KeyColumns = STRING ASC, COMPLEX ASC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 25; KeyColumns = STRING DESC, COMPLEX ASC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 26; KeyColumns = COMPLEX ASC, COMPLEX ASC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
}
|
||||
);
|
||||
|
||||
// index = 27; KeyColumns = COMPLEX DESC, COMPLEX ASC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 28; KeyColumns = STRING ASC, COMPLEX DESC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 29; KeyColumns = STRING DESC, COMPLEX DESC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 30; KeyColumns = COMPLEX ASC, COMPLEX DESC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
}
|
||||
);
|
||||
|
||||
// index = 31; KeyColumns = COMPLEX DESC, COMPLEX DESC, STRING DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 32; KeyColumns = STRING ASC, STRING ASC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 2),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 33; KeyColumns = STRING DESC, STRING ASC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 34; KeyColumns = COMPLEX ASC, STRING ASC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 35; KeyColumns = COMPLEX DESC, STRING ASC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 36; KeyColumns = STRING ASC, STRING DESC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 37; KeyColumns = STRING DESC, STRING DESC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 2),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 38; KeyColumns = COMPLEX ASC, STRING DESC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 39; KeyColumns = COMPLEX DESC, STRING DESC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 40; KeyColumns = STRING ASC, COMPLEX ASC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 41; KeyColumns = STRING DESC, COMPLEX ASC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 42; KeyColumns = COMPLEX ASC, COMPLEX ASC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
}
|
||||
);
|
||||
|
||||
// index = 43; KeyColumns = COMPLEX DESC, COMPLEX ASC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 44; KeyColumns = STRING ASC, COMPLEX DESC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 45; KeyColumns = STRING DESC, COMPLEX DESC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 46; KeyColumns = COMPLEX ASC, COMPLEX DESC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
}
|
||||
);
|
||||
|
||||
// index = 47; KeyColumns = COMPLEX DESC, COMPLEX DESC, COMPLEX ASC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
|
||||
// index = 48; KeyColumns = STRING ASC, STRING ASC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 2),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 49; KeyColumns = STRING DESC, STRING ASC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 50; KeyColumns = COMPLEX ASC, STRING ASC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 51; KeyColumns = COMPLEX DESC, STRING ASC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 52; KeyColumns = STRING ASC, STRING DESC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 53; KeyColumns = STRING DESC, STRING DESC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 2),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 54; KeyColumns = COMPLEX ASC, STRING DESC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 55; KeyColumns = COMPLEX DESC, STRING DESC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 56; KeyColumns = STRING ASC, COMPLEX ASC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 57; KeyColumns = STRING DESC, COMPLEX ASC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 58; KeyColumns = COMPLEX ASC, COMPLEX ASC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
}
|
||||
);
|
||||
|
||||
// index = 59; KeyColumns = COMPLEX DESC, COMPLEX ASC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 60; KeyColumns = STRING ASC, COMPLEX DESC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 61; KeyColumns = STRING DESC, COMPLEX DESC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(true, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
// index = 62; KeyColumns = COMPLEX ASC, COMPLEX DESC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.ASCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
}
|
||||
);
|
||||
|
||||
// index = 63; KeyColumns = COMPLEX DESC, COMPLEX DESC, COMPLEX DESC
|
||||
expectedResultsBuilder.add(
|
||||
new RunLengthEntry[]{
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1),
|
||||
new RunLengthEntry(false, KeyOrder.DESCENDING, 1)
|
||||
}
|
||||
);
|
||||
|
||||
List<RunLengthEntry[]> expectedResults = expectedResultsBuilder.build();
|
||||
|
||||
final List<Pair<ColumnType, KeyOrder>> columnTypeAndKeyOrder = ImmutableList.of(
|
||||
Pair.of(ColumnType.STRING, KeyOrder.ASCENDING),
|
||||
Pair.of(ColumnType.STRING, KeyOrder.DESCENDING),
|
||||
Pair.of(ColumnType.NESTED_DATA, KeyOrder.ASCENDING),
|
||||
Pair.of(ColumnType.NESTED_DATA, KeyOrder.DESCENDING)
|
||||
);
|
||||
|
||||
|
||||
for (int i = 0; i < 64; ++i) {
|
||||
Pair<List<KeyColumn>, RowSignature> keyColumnsAndRowSignature = generateKeyColumns(columnTypeAndKeyOrder, i);
|
||||
RunLengthEntry[] actualEntries = RowKeyComparisonRunLengths
|
||||
.create(keyColumnsAndRowSignature.lhs, keyColumnsAndRowSignature.rhs)
|
||||
.getRunLengthEntries();
|
||||
Assert.assertArrayEquals(StringUtils.format("Result %d incorrect", i), expectedResults.get(i), actualEntries);
|
||||
}
|
||||
}
|
||||
|
||||
private Pair<List<KeyColumn>, RowSignature> generateKeyColumns(
|
||||
final List<Pair<ColumnType, KeyOrder>> columnTypeAndKeyOrder,
|
||||
int index
|
||||
)
|
||||
{
|
||||
final List<KeyColumn> keyColumns = new ArrayList<>();
|
||||
final RowSignature.Builder builder = RowSignature.builder();
|
||||
|
||||
int firstKeyColumn = index % 4;
|
||||
keyColumns.add(new KeyColumn("a", columnTypeAndKeyOrder.get(firstKeyColumn).rhs));
|
||||
builder.add("a", columnTypeAndKeyOrder.get(firstKeyColumn).lhs);
|
||||
index /= 4;
|
||||
|
||||
int secondKeyColumn = index % 4;
|
||||
keyColumns.add(new KeyColumn("b", columnTypeAndKeyOrder.get(secondKeyColumn).rhs));
|
||||
builder.add("b", columnTypeAndKeyOrder.get(secondKeyColumn).lhs);
|
||||
index /= 4;
|
||||
|
||||
int thirdKeyColumn = index % 4; // Should be no-op, since index < 64
|
||||
keyColumns.add(new KeyColumn("c", columnTypeAndKeyOrder.get(thirdKeyColumn).rhs));
|
||||
builder.add("c", columnTypeAndKeyOrder.get(thirdKeyColumn).lhs);
|
||||
|
||||
return Pair.of(keyColumns, builder.build());
|
||||
}
|
||||
}
|
|
@ -290,7 +290,7 @@ public class SuperSorterTest
|
|||
maxBytesPerFrame
|
||||
) : new FileOutputChannelFactory(tempFolder, maxBytesPerFrame, null);
|
||||
final RowKeyReader keyReader = clusterBy.keyReader(signature);
|
||||
final Comparator<RowKey> keyComparator = clusterBy.keyComparator();
|
||||
final Comparator<RowKey> keyComparator = clusterBy.keyComparator(signature);
|
||||
final SettableFuture<ClusterByPartitions> clusterByPartitionsFuture = SettableFuture.create();
|
||||
final SuperSorterProgressTracker superSorterProgressTracker = new SuperSorterProgressTracker();
|
||||
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.druid.frame.read.FrameReader;
|
|||
import org.apache.druid.frame.segment.FrameSegment;
|
||||
import org.apache.druid.frame.segment.FrameStorageAdapter;
|
||||
import org.apache.druid.frame.testutil.FrameTestUtil;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
|
@ -88,6 +89,12 @@ import java.util.stream.Collectors;
|
|||
@RunWith(Parameterized.class)
|
||||
public class FrameWriterTest extends InitializedNullHandlingTest
|
||||
{
|
||||
|
||||
static {
|
||||
ComplexMetrics.registerSerde(HyperUniquesSerde.TYPE_NAME, new HyperUniquesSerde());
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
}
|
||||
|
||||
private static final int DEFAULT_ALLOCATOR_CAPACITY = 1_000_000;
|
||||
|
||||
@Nullable
|
||||
|
@ -272,11 +279,15 @@ public class FrameWriterTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void test_complex()
|
||||
public void test_complex_hll()
|
||||
{
|
||||
// Complex types can't be sorted, so skip the sortedness tests.
|
||||
Assume.assumeThat(sortedness, CoreMatchers.is(KeyOrder.NONE));
|
||||
testWithDataset(FrameWriterTestData.TEST_COMPLEX);
|
||||
testWithDataset(FrameWriterTestData.TEST_COMPLEX_HLL);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_complex_nested()
|
||||
{
|
||||
testWithDataset(FrameWriterTestData.TEST_COMPLEX_NESTED);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -332,26 +343,12 @@ public class FrameWriterTest extends InitializedNullHandlingTest
|
|||
// Test all possible arrangements of two different types.
|
||||
for (final FrameWriterTestData.Dataset<?> dataset1 : FrameWriterTestData.DATASETS) {
|
||||
for (final FrameWriterTestData.Dataset<?> dataset2 : FrameWriterTestData.DATASETS) {
|
||||
if (dataset1.getType().isArray() && dataset1.getType().getElementType().isNumeric()
|
||||
|| dataset2.getType().isArray() && dataset2.getType().getElementType().isNumeric()) {
|
||||
if (inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR) {
|
||||
// Skip the check if any of the dataset is a numerical array and any of the input or the output frame type
|
||||
// is COLUMNAR.
|
||||
continue;
|
||||
}
|
||||
}
|
||||
final RowSignature signature = makeSignature(Arrays.asList(dataset1, dataset2));
|
||||
final Sequence<List<Object>> rowSequence = unsortAndMakeRows(Arrays.asList(dataset1, dataset2));
|
||||
|
||||
// Sort by all columns up to the first COMPLEX one. (Can't sort by COMPLEX.)
|
||||
final List<String> sortColumns = new ArrayList<>();
|
||||
if (!dataset1.getType().is(ValueType.COMPLEX)) {
|
||||
sortColumns.add(signature.getColumnName(0));
|
||||
|
||||
if (!dataset2.getType().is(ValueType.COMPLEX)) {
|
||||
sortColumns.add(signature.getColumnName(1));
|
||||
}
|
||||
}
|
||||
sortColumns.add(signature.getColumnName(0));
|
||||
sortColumns.add(signature.getColumnName(1));
|
||||
|
||||
try {
|
||||
final Pair<Frame, Integer> writeResult = writeFrame(rowSequence, signature, sortColumns);
|
||||
|
@ -384,14 +381,9 @@ public class FrameWriterTest extends InitializedNullHandlingTest
|
|||
final Sequence<List<Object>> rowSequence = unsortAndMakeRows(FrameWriterTestData.DATASETS);
|
||||
final int totalRows = rowSequence.toList().size();
|
||||
|
||||
// Sort by all columns up to the first COMPLEX one. (Can't sort by COMPLEX.)
|
||||
final List<String> sortColumns = new ArrayList<>();
|
||||
for (int i = 0; i < signature.size(); i++) {
|
||||
if (signature.getColumnType(i).get().is(ValueType.COMPLEX)) {
|
||||
break;
|
||||
} else {
|
||||
sortColumns.add(signature.getColumnName(i));
|
||||
}
|
||||
sortColumns.add(signature.getColumnName(i));
|
||||
}
|
||||
|
||||
final ByteBuffer allocatorMemory = ByteBuffer.wrap(new byte[DEFAULT_ALLOCATOR_CAPACITY]);
|
||||
|
@ -473,7 +465,7 @@ public class FrameWriterTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
final RowSignature keySignature = KeyTestUtils.createKeySignature(keyColumns, signature);
|
||||
final Comparator<RowKey> keyComparator = RowKeyComparator.create(keyColumns);
|
||||
final Comparator<RowKey> keyComparator = RowKeyComparator.create(keyColumns, signature);
|
||||
|
||||
return Sequences.sort(
|
||||
rows,
|
||||
|
@ -514,10 +506,7 @@ public class FrameWriterTest extends InitializedNullHandlingTest
|
|||
return Collections.emptyList();
|
||||
} else {
|
||||
return sortColumnNames.stream()
|
||||
.map(
|
||||
columnName ->
|
||||
new KeyColumn(columnName, sortedness)
|
||||
)
|
||||
.map(columnName -> new KeyColumn(columnName, sortedness))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,19 +20,19 @@
|
|||
package org.apache.druid.frame.write;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.hash.Hashing;
|
||||
import it.unimi.dsi.fastutil.objects.ObjectArrays;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.frame.key.ByteRowKeyComparatorTest;
|
||||
import org.apache.druid.frame.key.KeyOrder;
|
||||
import org.apache.druid.hll.HyperLogLogCollector;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.guava.Comparators;
|
||||
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.nested.StructuredData;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -266,15 +266,34 @@ public class FrameWriterTestData
|
|||
);
|
||||
//CHECKSTYLE.ON: Regexp
|
||||
|
||||
public static final Dataset<HyperLogLogCollector> TEST_COMPLEX = new Dataset<>(
|
||||
public static final Dataset<HyperLogLogCollector> TEST_COMPLEX_HLL = new Dataset<>(
|
||||
HyperUniquesAggregatorFactory.TYPE,
|
||||
Arrays.asList(
|
||||
null,
|
||||
makeHllCollector(null),
|
||||
makeHllCollector("foo")
|
||||
ByteRowKeyComparatorTest.makeHllCollector(1),
|
||||
ByteRowKeyComparatorTest.makeHllCollector(10),
|
||||
ByteRowKeyComparatorTest.makeHllCollector(50)
|
||||
)
|
||||
);
|
||||
|
||||
// Sortedness of structured data depends on the hash value computed for the objects inside.
|
||||
public static final Dataset<StructuredData> TEST_COMPLEX_NESTED = new Dataset<>(
|
||||
ColumnType.NESTED_DATA,
|
||||
Stream.of(
|
||||
null,
|
||||
StructuredData.create("foo"),
|
||||
StructuredData.create("bar"),
|
||||
StructuredData.create(ImmutableMap.of("a", 100, "b", 200)),
|
||||
StructuredData.create(ImmutableMap.of("a", 100, "b", ImmutableList.of("x", "y"))),
|
||||
StructuredData.create(ImmutableMap.of("a", 100, "b", ImmutableMap.of("x", "y"))),
|
||||
StructuredData.wrap(100.1D),
|
||||
StructuredData.wrap(ImmutableList.of("p", "q", "r")),
|
||||
StructuredData.wrap(100),
|
||||
StructuredData.wrap(ImmutableList.of("p", "q", "r")),
|
||||
StructuredData.wrap(1000)
|
||||
).sorted(Comparators.naturalNullsFirst()).collect(Collectors.toList())
|
||||
);
|
||||
|
||||
/**
|
||||
* Wrapper around all the various TEST_* lists.
|
||||
*/
|
||||
|
@ -289,20 +308,10 @@ public class FrameWriterTestData
|
|||
.add(TEST_ARRAYS_LONG)
|
||||
.add(TEST_ARRAYS_FLOAT)
|
||||
.add(TEST_ARRAYS_DOUBLE)
|
||||
.add(TEST_COMPLEX)
|
||||
.add(TEST_COMPLEX_HLL)
|
||||
.add(TEST_COMPLEX_NESTED)
|
||||
.build();
|
||||
|
||||
private static HyperLogLogCollector makeHllCollector(@Nullable final String value)
|
||||
{
|
||||
final HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector();
|
||||
|
||||
if (value != null) {
|
||||
collector.add(Hashing.murmur3_128().hashBytes(StringUtils.toUtf8(value)).asBytes());
|
||||
}
|
||||
|
||||
return collector;
|
||||
}
|
||||
|
||||
public static class Dataset<T>
|
||||
{
|
||||
private final ColumnType type;
|
||||
|
|
|
@ -24,12 +24,9 @@ import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory;
|
|||
import org.apache.druid.frame.key.KeyColumn;
|
||||
import org.apache.druid.frame.key.KeyOrder;
|
||||
import org.apache.druid.frame.write.columnar.ColumnarFrameWriterFactory;
|
||||
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.join.filter.AllNullColumnSelectorFactory;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.hamcrest.MatcherAssert;
|
||||
|
@ -103,32 +100,6 @@ public class FrameWritersTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals(ColumnType.ofArray(ColumnType.LONG_ARRAY), e.getColumnType());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_rowBased_unsupportedSortColumnType()
|
||||
{
|
||||
// Register, but don't unregister at the end of this test, because many other tests out there expect this to exist
|
||||
// even though they don't explicitly register it.
|
||||
ComplexMetrics.registerSerde(HyperUniquesSerde.TYPE_NAME, new HyperUniquesSerde());
|
||||
|
||||
final IllegalArgumentException e = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() ->
|
||||
FrameWriters.makeFrameWriterFactory(
|
||||
FrameType.ROW_BASED,
|
||||
new ArenaMemoryAllocatorFactory(ALLOCATOR_CAPACITY),
|
||||
RowSignature.builder().add("x", HyperUniquesAggregatorFactory.TYPE).build(),
|
||||
Collections.singletonList(new KeyColumn("x", KeyOrder.ASCENDING))
|
||||
)
|
||||
);
|
||||
|
||||
MatcherAssert.assertThat(
|
||||
e,
|
||||
ThrowableMessageMatcher.hasMessage(
|
||||
CoreMatchers.containsString("Sort column [x] is not comparable (type = COMPLEX<hyperUnique>)")
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_rowBased_sortColumnsNotPrefix()
|
||||
{
|
||||
|
|
|
@ -73,7 +73,7 @@ public class TaskReportSerdeTest
|
|||
@Test
|
||||
public void testSerdeOfKillTaskReport() throws Exception
|
||||
{
|
||||
KillTaskReport originalReport = new KillTaskReport("taskId", new KillTaskReport.Stats(1, 2, 3));
|
||||
KillTaskReport originalReport = new KillTaskReport("taskId", new KillTaskReport.Stats(1, 2));
|
||||
String reportJson = jsonMapper.writeValueAsString(originalReport);
|
||||
TaskReport deserialized = jsonMapper.readValue(reportJson, TaskReport.class);
|
||||
|
||||
|
@ -81,6 +81,7 @@ public class TaskReportSerdeTest
|
|||
|
||||
KillTaskReport deserializedReport = (KillTaskReport) deserialized;
|
||||
Assert.assertEquals(originalReport, deserializedReport);
|
||||
Assert.assertEquals(originalReport.hashCode(), deserializedReport.hashCode());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -131,6 +131,14 @@ public class GroupingAggregatorFactoryTest
|
|||
));
|
||||
makeFactory(new String[Long.SIZE], null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithDuplicateGroupings()
|
||||
{
|
||||
exception.expect(IllegalArgumentException.class);
|
||||
exception.expectMessage("Encountered same dimension more than once in groupings");
|
||||
makeFactory(new String[]{"a", "a"}, null);
|
||||
}
|
||||
}
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
|
|
|
@ -43,7 +43,6 @@ public class ClientKillUnusedSegmentsTaskQuery implements ClientTaskQuery
|
|||
private final Interval interval;
|
||||
@Nullable
|
||||
private final List<String> versions;
|
||||
private final Boolean markAsUnused;
|
||||
private final Integer batchSize;
|
||||
@Nullable
|
||||
private final Integer limit;
|
||||
|
@ -56,7 +55,6 @@ public class ClientKillUnusedSegmentsTaskQuery implements ClientTaskQuery
|
|||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval,
|
||||
@JsonProperty("versions") @Nullable List<String> versions,
|
||||
@JsonProperty("markAsUnused") @Deprecated Boolean markAsUnused,
|
||||
@JsonProperty("batchSize") Integer batchSize,
|
||||
@JsonProperty("limit") @Nullable Integer limit,
|
||||
@JsonProperty("maxUsedStatusLastUpdatedTime") @Nullable DateTime maxUsedStatusLastUpdatedTime
|
||||
|
@ -72,7 +70,6 @@ public class ClientKillUnusedSegmentsTaskQuery implements ClientTaskQuery
|
|||
this.dataSource = dataSource;
|
||||
this.interval = interval;
|
||||
this.versions = versions;
|
||||
this.markAsUnused = markAsUnused;
|
||||
this.batchSize = batchSize;
|
||||
this.limit = limit;
|
||||
this.maxUsedStatusLastUpdatedTime = maxUsedStatusLastUpdatedTime;
|
||||
|
@ -112,20 +109,6 @@ public class ClientKillUnusedSegmentsTaskQuery implements ClientTaskQuery
|
|||
return versions;
|
||||
}
|
||||
|
||||
/**
|
||||
* This field has been deprecated as "kill" tasks should not be responsible for
|
||||
* marking segments as unused. Instead, users should call the Coordinator API
|
||||
* {@code /{dataSourceName}/markUnused} to explicitly mark segments as unused.
|
||||
* Segments may also be marked unused by the Coordinator if they become overshadowed
|
||||
* or have a {@code DropRule} applied to them.
|
||||
*/
|
||||
@Deprecated
|
||||
@JsonProperty
|
||||
public Boolean getMarkAsUnused()
|
||||
{
|
||||
return markAsUnused;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Integer getBatchSize()
|
||||
{
|
||||
|
@ -161,7 +144,6 @@ public class ClientKillUnusedSegmentsTaskQuery implements ClientTaskQuery
|
|||
&& Objects.equals(dataSource, that.dataSource)
|
||||
&& Objects.equals(interval, that.interval)
|
||||
&& Objects.equals(versions, that.versions)
|
||||
&& Objects.equals(markAsUnused, that.markAsUnused)
|
||||
&& Objects.equals(batchSize, that.batchSize)
|
||||
&& Objects.equals(limit, that.limit)
|
||||
&& Objects.equals(maxUsedStatusLastUpdatedTime, that.maxUsedStatusLastUpdatedTime);
|
||||
|
@ -170,6 +152,6 @@ public class ClientKillUnusedSegmentsTaskQuery implements ClientTaskQuery
|
|||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(id, dataSource, interval, versions, markAsUnused, batchSize, limit, maxUsedStatusLastUpdatedTime);
|
||||
return Objects.hash(id, dataSource, interval, versions, batchSize, limit, maxUsedStatusLastUpdatedTime);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -109,7 +109,6 @@ public interface OverlordClient
|
|||
dataSource,
|
||||
interval,
|
||||
versions,
|
||||
false,
|
||||
null,
|
||||
maxSegmentsToKill,
|
||||
maxUsedStatusLastUpdatedTime
|
||||
|
|
|
@ -33,7 +33,6 @@ public class ClientKillUnusedSegmentsTaskQueryTest
|
|||
private static final String DATA_SOURCE = "data_source";
|
||||
public static final DateTime START = DateTimes.nowUtc();
|
||||
private static final Interval INTERVAL = new Interval(START, START.plus(1));
|
||||
private static final Boolean MARK_UNUSED = true;
|
||||
private static final Integer BATCH_SIZE = 999;
|
||||
private static final Integer LIMIT = 1000;
|
||||
|
||||
|
@ -47,7 +46,6 @@ public class ClientKillUnusedSegmentsTaskQueryTest
|
|||
DATA_SOURCE,
|
||||
INTERVAL,
|
||||
null,
|
||||
true,
|
||||
BATCH_SIZE,
|
||||
LIMIT,
|
||||
null
|
||||
|
@ -78,12 +76,6 @@ public class ClientKillUnusedSegmentsTaskQueryTest
|
|||
Assert.assertEquals(INTERVAL, clientKillUnusedSegmentsQuery.getInterval());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetMarkUnused()
|
||||
{
|
||||
Assert.assertEquals(MARK_UNUSED, clientKillUnusedSegmentsQuery.getMarkAsUnused());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetBatchSize()
|
||||
{
|
||||
|
|
|
@ -436,7 +436,6 @@ public class OverlordClientImplTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
||||
|
|
|
@ -90,7 +90,19 @@ public class GroupingSqlAggregator implements SqlAggregator
|
|||
}
|
||||
}
|
||||
}
|
||||
AggregatorFactory factory = new GroupingAggregatorFactory(name, arguments);
|
||||
AggregatorFactory factory;
|
||||
try {
|
||||
factory = new GroupingAggregatorFactory(name, arguments);
|
||||
}
|
||||
catch (Exception e) {
|
||||
plannerContext.setPlanningError(
|
||||
"Initialisation of Grouping Aggregator Factory in case of [%s] threw [%s]",
|
||||
aggregateCall,
|
||||
e.getMessage()
|
||||
);
|
||||
return null;
|
||||
}
|
||||
|
||||
return Aggregation.create(factory);
|
||||
}
|
||||
|
||||
|
|
|
@ -66,6 +66,7 @@ import org.apache.druid.sql.calcite.rule.ProjectAggregatePruneUnusedCallRule;
|
|||
import org.apache.druid.sql.calcite.rule.ReverseLookupRule;
|
||||
import org.apache.druid.sql.calcite.rule.RewriteFirstValueLastValueRule;
|
||||
import org.apache.druid.sql.calcite.rule.SortCollapseRule;
|
||||
import org.apache.druid.sql.calcite.rule.logical.DruidAggregateRemoveRedundancyRule;
|
||||
import org.apache.druid.sql.calcite.rule.logical.DruidLogicalRules;
|
||||
import org.apache.druid.sql.calcite.run.EngineFeature;
|
||||
|
||||
|
@ -496,6 +497,7 @@ public class CalciteRulesManager
|
|||
rules.add(FilterJoinExcludePushToChildRule.FILTER_ON_JOIN_EXCLUDE_PUSH_TO_CHILD);
|
||||
rules.add(SortCollapseRule.instance());
|
||||
rules.add(ProjectAggregatePruneUnusedCallRule.instance());
|
||||
rules.add(DruidAggregateRemoveRedundancyRule.instance());
|
||||
|
||||
return rules.build();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,164 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.rule.logical;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.calcite.plan.RelOptRule;
|
||||
import org.apache.calcite.plan.RelOptRuleCall;
|
||||
import org.apache.calcite.plan.RelOptUtil;
|
||||
import org.apache.calcite.rel.RelNode;
|
||||
import org.apache.calcite.rel.core.Aggregate;
|
||||
import org.apache.calcite.rel.core.Aggregate.Group;
|
||||
import org.apache.calcite.rel.core.AggregateCall;
|
||||
import org.apache.calcite.rel.core.Project;
|
||||
import org.apache.calcite.rel.rules.TransformationRule;
|
||||
import org.apache.calcite.rex.RexInputRef;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.tools.RelBuilder;
|
||||
import org.apache.calcite.util.ImmutableBitSet;
|
||||
import org.apache.calcite.util.Util;
|
||||
import org.apache.calcite.util.mapping.Mappings;
|
||||
import org.checkerframework.checker.nullness.qual.Nullable;
|
||||
import org.immutables.value.Value;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Planner rule that recognizes a {@link Aggregate}
|
||||
* on top of a {@link Project} and if possible
|
||||
* aggregate through the project or removes the project.
|
||||
* <p>
|
||||
* This is updated version of {@link org.apache.calcite.rel.rules.AggregateProjectMergeRule}
|
||||
* to be able to handle expressions.
|
||||
*/
|
||||
@Value.Enclosing
|
||||
public class DruidAggregateRemoveRedundancyRule
|
||||
extends RelOptRule
|
||||
implements TransformationRule
|
||||
{
|
||||
|
||||
/**
|
||||
* Creates a DruidAggregateRemoveRedundancyRule.
|
||||
*/
|
||||
private static final DruidAggregateRemoveRedundancyRule INSTANCE = new DruidAggregateRemoveRedundancyRule();
|
||||
|
||||
private DruidAggregateRemoveRedundancyRule()
|
||||
{
|
||||
super(operand(Aggregate.class, operand(Project.class, any())));
|
||||
}
|
||||
|
||||
public static DruidAggregateRemoveRedundancyRule instance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onMatch(RelOptRuleCall call)
|
||||
{
|
||||
final Aggregate aggregate = call.rel(0);
|
||||
final Project project = call.rel(1);
|
||||
RelNode x = apply(call, aggregate, project);
|
||||
if (x != null) {
|
||||
call.transformTo(x);
|
||||
call.getPlanner().prune(aggregate);
|
||||
}
|
||||
}
|
||||
|
||||
public static @Nullable RelNode apply(RelOptRuleCall call, Aggregate aggregate, Project project)
|
||||
{
|
||||
final Set<Integer> interestingFields = RelOptUtil.getAllFields(aggregate);
|
||||
if (interestingFields.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
final Map<Integer, Integer> map = new HashMap<>();
|
||||
final Map<RexNode, Integer> assignedRefForExpr = new HashMap<>();
|
||||
List<RexNode> newRexNodes = new ArrayList<>();
|
||||
for (int source : interestingFields) {
|
||||
final RexNode rex = project.getProjects().get(source);
|
||||
if (!assignedRefForExpr.containsKey(rex)) {
|
||||
RexNode newNode = new RexInputRef(source, rex.getType());
|
||||
assignedRefForExpr.put(rex, newRexNodes.size());
|
||||
newRexNodes.add(newNode);
|
||||
}
|
||||
map.put(source, assignedRefForExpr.get(rex));
|
||||
}
|
||||
|
||||
if (newRexNodes.size() == project.getProjects().size()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final ImmutableBitSet newGroupSet = aggregate.getGroupSet().permute(map);
|
||||
ImmutableList<ImmutableBitSet> newGroupingSets = null;
|
||||
if (aggregate.getGroupType() != Group.SIMPLE) {
|
||||
newGroupingSets =
|
||||
ImmutableBitSet.ORDERING.immutableSortedCopy(
|
||||
Sets.newTreeSet(ImmutableBitSet.permute(aggregate.getGroupSets(), map)));
|
||||
}
|
||||
|
||||
final ImmutableList.Builder<AggregateCall> aggCalls = ImmutableList.builder();
|
||||
final int sourceCount = aggregate.getInput().getRowType().getFieldCount();
|
||||
final int targetCount = newRexNodes.size();
|
||||
final Mappings.TargetMapping targetMapping = Mappings.target(map, sourceCount, targetCount);
|
||||
for (AggregateCall aggregateCall : aggregate.getAggCallList()) {
|
||||
aggCalls.add(aggregateCall.transform(targetMapping));
|
||||
}
|
||||
|
||||
final RelBuilder relBuilder = call.builder();
|
||||
relBuilder.push(project);
|
||||
relBuilder.project(newRexNodes);
|
||||
|
||||
final Aggregate newAggregate =
|
||||
aggregate.copy(aggregate.getTraitSet(), relBuilder.build(),
|
||||
newGroupSet, newGroupingSets, aggCalls.build()
|
||||
);
|
||||
relBuilder.push(newAggregate);
|
||||
|
||||
final List<Integer> newKeys =
|
||||
Util.transform(
|
||||
aggregate.getGroupSet().asList(),
|
||||
key -> Objects.requireNonNull(
|
||||
map.get(key),
|
||||
() -> "no value found for key " + key + " in " + map
|
||||
)
|
||||
);
|
||||
|
||||
// Add a project if the group set is not in the same order or
|
||||
// contains duplicates.
|
||||
if (!newKeys.equals(newGroupSet.asList())) {
|
||||
final List<Integer> posList = new ArrayList<>();
|
||||
for (int newKey : newKeys) {
|
||||
posList.add(newGroupSet.indexOf(newKey));
|
||||
}
|
||||
for (int i = newAggregate.getGroupCount();
|
||||
i < newAggregate.getRowType().getFieldCount(); i++) {
|
||||
posList.add(i);
|
||||
}
|
||||
relBuilder.project(relBuilder.fields(posList));
|
||||
}
|
||||
|
||||
return relBuilder.build();
|
||||
}
|
||||
}
|
|
@ -1041,16 +1041,12 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
|
||||
final List<ValueType> types = new ArrayList<>();
|
||||
|
||||
final boolean isMSQ = isMSQRowType(queryResults.signature);
|
||||
|
||||
if (!isMSQ) {
|
||||
for (int i = 0; i < queryResults.signature.getColumnNames().size(); i++) {
|
||||
Optional<ColumnType> columnType = queryResults.signature.getColumnType(i);
|
||||
if (columnType.isPresent()) {
|
||||
types.add(columnType.get().getType());
|
||||
} else {
|
||||
types.add(null);
|
||||
}
|
||||
for (int i = 0; i < queryResults.signature.getColumnNames().size(); i++) {
|
||||
Optional<ColumnType> columnType = queryResults.signature.getColumnType(i);
|
||||
if (columnType.isPresent()) {
|
||||
types.add(columnType.get().getType());
|
||||
} else {
|
||||
types.add(null);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1067,19 +1063,13 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
matchMode.validate(
|
||||
row,
|
||||
i,
|
||||
isMSQ ? null : types.get(i),
|
||||
types.get(i),
|
||||
expectedCell,
|
||||
resultCell);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isMSQRowType(RowSignature signature)
|
||||
{
|
||||
List<String> colNames = signature.getColumnNames();
|
||||
return colNames.size() == 1 && "TASK".equals(colNames.get(0));
|
||||
}
|
||||
|
||||
public void assertResultsEquals(String sql, List<Object[]> expectedResults, List<Object[]> results)
|
||||
{
|
||||
int minSize = Math.min(results.size(), expectedResults.size());
|
||||
|
@ -1226,6 +1216,11 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
assumeFalse(testBuilder().config.isRunningMSQ(), "test case is not MSQ compatible");
|
||||
}
|
||||
|
||||
protected boolean isRunningMSQ()
|
||||
{
|
||||
return testBuilder().config.isRunningMSQ();
|
||||
}
|
||||
|
||||
protected static boolean isRewriteJoinToFilter(final Map<String, Object> queryContext)
|
||||
{
|
||||
return (boolean) queryContext.getOrDefault(
|
||||
|
|
|
@ -91,13 +91,13 @@ import java.util.stream.Collectors;
|
|||
@SqlTestFrameworkConfig.ComponentSupplier(NestedComponentSupplier.class)
|
||||
public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
||||
{
|
||||
private static final String DATA_SOURCE = "nested";
|
||||
private static final String DATA_SOURCE_MIXED = "nested_mix";
|
||||
private static final String DATA_SOURCE_MIXED_2 = "nested_mix_2";
|
||||
private static final String DATA_SOURCE_ARRAYS = "arrays";
|
||||
private static final String DATA_SOURCE_ALL = "all_auto";
|
||||
public static final String DATA_SOURCE = "nested";
|
||||
public static final String DATA_SOURCE_MIXED = "nested_mix";
|
||||
public static final String DATA_SOURCE_MIXED_2 = "nested_mix_2";
|
||||
public static final String DATA_SOURCE_ARRAYS = "arrays";
|
||||
public static final String DATA_SOURCE_ALL = "all_auto";
|
||||
|
||||
private static final List<ImmutableMap<String, Object>> RAW_ROWS = ImmutableList.of(
|
||||
public static final List<ImmutableMap<String, Object>> RAW_ROWS = ImmutableList.of(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("t", "2000-01-01")
|
||||
.put("string", "aaa")
|
||||
|
@ -149,7 +149,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
.build()
|
||||
);
|
||||
|
||||
private static final InputRowSchema ALL_JSON_COLUMNS = new InputRowSchema(
|
||||
public static final InputRowSchema ALL_JSON_COLUMNS = new InputRowSchema(
|
||||
new TimestampSpec("t", "iso", null),
|
||||
DimensionsSpec.builder().setDimensions(
|
||||
ImmutableList.<DimensionSchema>builder()
|
||||
|
@ -163,7 +163,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
null
|
||||
);
|
||||
|
||||
private static final InputRowSchema JSON_AND_SCALAR_MIX = new InputRowSchema(
|
||||
public static final InputRowSchema JSON_AND_SCALAR_MIX = new InputRowSchema(
|
||||
new TimestampSpec("t", "iso", null),
|
||||
DimensionsSpec.builder().setDimensions(
|
||||
ImmutableList.<DimensionSchema>builder()
|
||||
|
@ -176,10 +176,10 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
).build(),
|
||||
null
|
||||
);
|
||||
private static final List<InputRow> ROWS =
|
||||
public static final List<InputRow> ROWS =
|
||||
RAW_ROWS.stream().map(raw -> TestDataBuilder.createRow(raw, ALL_JSON_COLUMNS)).collect(Collectors.toList());
|
||||
|
||||
private static final List<InputRow> ROWS_MIX =
|
||||
public static final List<InputRow> ROWS_MIX =
|
||||
RAW_ROWS.stream().map(raw -> TestDataBuilder.createRow(raw, JSON_AND_SCALAR_MIX)).collect(Collectors.toList());
|
||||
|
||||
public static class NestedComponentSupplier extends StandardComponentSupplier
|
||||
|
@ -1085,6 +1085,10 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testGroupByRootSingleTypeStringMixed2SparseJsonValueNonExistentPath()
|
||||
{
|
||||
// Fails while planning for MSQ because MSQ expects a defined type for the virtual column while planning (to figure
|
||||
// out the scanSignature) whereas the NestedFieldVirtualColumn cannot determine the type for the non-existant path,
|
||||
// due to which it returns null
|
||||
msqIncompatible();
|
||||
testQuery(
|
||||
"SELECT "
|
||||
+ "JSON_VALUE(string_sparse, '$[1]'), "
|
||||
|
@ -2697,6 +2701,8 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testJsonAndArrayAgg()
|
||||
{
|
||||
// MSQ cannot handle non-primitive arrays
|
||||
msqIncompatible();
|
||||
cannotVectorize();
|
||||
testQuery(
|
||||
"SELECT "
|
||||
|
@ -5422,6 +5428,49 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
public void testScanStringNotNullCast()
|
||||
{
|
||||
skipVectorize();
|
||||
final List<Object[]> expectedResults;
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
expectedResults = ImmutableList.of(
|
||||
new Object[]{10L},
|
||||
new Object[]{10L}
|
||||
);
|
||||
} else {
|
||||
if (isRunningMSQ()) {
|
||||
expectedResults = ImmutableList.of(
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{10L},
|
||||
new Object[]{10L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L}
|
||||
);
|
||||
} else {
|
||||
expectedResults = ImmutableList.of(
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{10L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{10L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L}
|
||||
);
|
||||
}
|
||||
}
|
||||
testQuery(
|
||||
"SELECT "
|
||||
+ "CAST(string_sparse as BIGINT)"
|
||||
|
@ -5439,27 +5488,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
NullHandling.sqlCompatible() ?
|
||||
ImmutableList.of(
|
||||
new Object[]{10L},
|
||||
new Object[]{10L}
|
||||
) :
|
||||
ImmutableList.of(
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{10L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{10L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L}
|
||||
),
|
||||
expectedResults,
|
||||
RowSignature.builder()
|
||||
.add("EXPR$0", ColumnType.LONG)
|
||||
.build()
|
||||
|
@ -5914,6 +5943,8 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testScanAllTypesAuto()
|
||||
{
|
||||
// Variant types are not supported by MSQ.
|
||||
msqIncompatible();
|
||||
skipVectorize();
|
||||
testQuery(
|
||||
"SELECT * FROM druid.all_auto",
|
||||
|
@ -6852,6 +6883,8 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testJsonQueryArrayNullArray()
|
||||
{
|
||||
// Array complex JSON isn't supported
|
||||
msqIncompatible();
|
||||
cannotVectorize();
|
||||
testBuilder()
|
||||
.sql("SELECT JSON_QUERY_ARRAY(arrayObject, '$.') FROM druid.arrays where arrayObject is null limit 1")
|
||||
|
@ -7071,6 +7104,10 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testJsonValueNestedEmptyArray()
|
||||
{
|
||||
// The data set has empty arrays, however MSQ returns nulls. The root cause of the issue is the incorrect
|
||||
// capabilities returned by NestedFieldVirtualColumn when planning which causes MSQ to treat the nested path
|
||||
// as STRING, even though it is an array.
|
||||
msqIncompatible();
|
||||
// test for regression
|
||||
skipVectorize();
|
||||
testQuery(
|
||||
|
|
|
@ -8788,8 +8788,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
)
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING),
|
||||
new DefaultDimensionSpec("v0", "d1", ColumnType.LONG)
|
||||
new DefaultDimensionSpec("v0", "d0", ColumnType.LONG),
|
||||
new DefaultDimensionSpec("dim1", "d1", ColumnType.STRING)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(
|
||||
|
@ -8832,9 +8832,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
new FilteredAggregatorFactory(
|
||||
new CountAggregatorFactory("_a1"),
|
||||
and(
|
||||
notNull("d0"),
|
||||
notNull("d1"),
|
||||
equality("a1", 0L, ColumnType.LONG),
|
||||
expressionFilter("\"d1\"")
|
||||
expressionFilter("\"d0\"")
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -12938,8 +12938,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
.setVirtualColumns(expressionVirtualColumn("v0", "1", ColumnType.LONG))
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "d0", ColumnType.LONG),
|
||||
new DefaultDimensionSpec("v0", "d1", ColumnType.LONG)
|
||||
new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)
|
||||
)
|
||||
)
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
|
@ -15680,10 +15679,63 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
.build()
|
||||
)
|
||||
).expectedResults(
|
||||
ResultMatchMode.RELAX_NULLS,
|
||||
ImmutableList.of(
|
||||
new Object[]{null, null, null}
|
||||
)
|
||||
);
|
||||
NullHandling.sqlCompatible() ? ImmutableList.of(
|
||||
new Object[]{null, null, null}
|
||||
) : ImmutableList.of(
|
||||
new Object[]{false, false, ""}
|
||||
)
|
||||
).run();
|
||||
}
|
||||
|
||||
@SqlTestFrameworkConfig.NumMergeBuffers(4)
|
||||
@Test
|
||||
public void testGroupingSetsWithAggrgateCase()
|
||||
{
|
||||
cannotVectorize();
|
||||
msqIncompatible();
|
||||
final Map<String, Object> queryContext = ImmutableMap.of(
|
||||
PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT, false,
|
||||
PlannerConfig.CTX_KEY_USE_GROUPING_SET_FOR_EXACT_DISTINCT, true
|
||||
);
|
||||
testBuilder()
|
||||
.sql(
|
||||
"SELECT\n"
|
||||
+ " TIME_FLOOR(\"__time\", 'PT1H') ,\n"
|
||||
+ " COUNT(DISTINCT \"page\") ,\n"
|
||||
+ " COUNT(DISTINCT CASE WHEN \"channel\" = '#it.wikipedia' THEN \"user\" END), \n"
|
||||
+ " COUNT(DISTINCT \"user\") FILTER (WHERE \"channel\" = '#it.wikipedia'), "
|
||||
+ " COUNT(DISTINCT \"user\") \n"
|
||||
+ "FROM \"wikipedia\"\n"
|
||||
+ "GROUP BY 1"
|
||||
)
|
||||
.queryContext(queryContext)
|
||||
.expectedResults(
|
||||
ImmutableList.of(
|
||||
new Object[]{1442016000000L, 264L, 5L, 5L, 149L},
|
||||
new Object[]{1442019600000L, 1090L, 14L, 14L, 506L},
|
||||
new Object[]{1442023200000L, 1045L, 10L, 10L, 459L},
|
||||
new Object[]{1442026800000L, 766L, 10L, 10L, 427L},
|
||||
new Object[]{1442030400000L, 781L, 6L, 6L, 427L},
|
||||
new Object[]{1442034000000L, 1223L, 10L, 10L, 448L},
|
||||
new Object[]{1442037600000L, 2092L, 13L, 13L, 498L},
|
||||
new Object[]{1442041200000L, 2181L, 21L, 21L, 574L},
|
||||
new Object[]{1442044800000L, 1552L, 36L, 36L, 707L},
|
||||
new Object[]{1442048400000L, 1624L, 44L, 44L, 770L},
|
||||
new Object[]{1442052000000L, 1710L, 37L, 37L, 785L},
|
||||
new Object[]{1442055600000L, 1532L, 40L, 40L, 799L},
|
||||
new Object[]{1442059200000L, 1633L, 45L, 45L, 855L},
|
||||
new Object[]{1442062800000L, 1958L, 44L, 44L, 905L},
|
||||
new Object[]{1442066400000L, 1779L, 48L, 48L, 886L},
|
||||
new Object[]{1442070000000L, 1868L, 37L, 37L, 949L},
|
||||
new Object[]{1442073600000L, 1846L, 50L, 50L, 969L},
|
||||
new Object[]{1442077200000L, 2168L, 38L, 38L, 941L},
|
||||
new Object[]{1442080800000L, 2043L, 40L, 40L, 925L},
|
||||
new Object[]{1442084400000L, 1924L, 32L, 32L, 930L},
|
||||
new Object[]{1442088000000L, 1736L, 31L, 31L, 882L},
|
||||
new Object[]{1442091600000L, 1672L, 40L, 40L, 861L},
|
||||
new Object[]{1442095200000L, 1504L, 28L, 28L, 716L},
|
||||
new Object[]{1442098800000L, 1407L, 20L, 20L, 631L}
|
||||
)
|
||||
).run();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -148,6 +148,26 @@ public class QueryTestRunner
|
|||
this.capture = capture;
|
||||
}
|
||||
|
||||
public QueryResults(
|
||||
final Map<String, Object> queryContext,
|
||||
final String vectorizeOption,
|
||||
final RowSignature rowSignature,
|
||||
final List<Object[]> results,
|
||||
final List<Query<?>> recordedQueries,
|
||||
final PlannerCaptureHook capture
|
||||
)
|
||||
{
|
||||
this.queryContext = queryContext;
|
||||
this.vectorizeOption = vectorizeOption;
|
||||
this.sqlSignature = null;
|
||||
this.signature = rowSignature;
|
||||
this.results = results;
|
||||
this.recordedQueries = recordedQueries;
|
||||
this.resourceActions = null;
|
||||
this.exception = null;
|
||||
this.capture = capture;
|
||||
}
|
||||
|
||||
public QueryResults(
|
||||
final Map<String, Object> queryContext,
|
||||
final String vectorizeOption,
|
||||
|
@ -165,9 +185,9 @@ public class QueryTestRunner
|
|||
this.sqlSignature = null;
|
||||
}
|
||||
|
||||
public QueryResults withResults(List<Object[]> newResults)
|
||||
public QueryResults withSignatureAndResults(final RowSignature rowSignature, final List<Object[]> newResults)
|
||||
{
|
||||
return new QueryResults(queryContext, vectorizeOption, sqlSignature, newResults, recordedQueries, capture);
|
||||
return new QueryResults(queryContext, vectorizeOption, rowSignature, newResults, recordedQueries, capture);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -14,7 +14,7 @@
|
|||
"@blueprintjs/datetime2": "^0.9.35",
|
||||
"@blueprintjs/icons": "^4.16.0",
|
||||
"@blueprintjs/popover2": "^1.14.9",
|
||||
"@druid-toolkit/query": "^0.22.11",
|
||||
"@druid-toolkit/query": "^0.22.13",
|
||||
"@druid-toolkit/visuals-core": "^0.3.3",
|
||||
"@druid-toolkit/visuals-react": "^0.3.3",
|
||||
"ace-builds": "~1.4.14",
|
||||
|
@ -1004,9 +1004,9 @@
|
|||
}
|
||||
},
|
||||
"node_modules/@druid-toolkit/query": {
|
||||
"version": "0.22.11",
|
||||
"resolved": "https://registry.npmjs.org/@druid-toolkit/query/-/query-0.22.11.tgz",
|
||||
"integrity": "sha512-VVEn/tsEr9fb+8eKc+nu3/YH7l+LZ1vd0D32UDo66GLS3cI+EKOCM7VYC8lTvB1tAS+98w/EzfbdlRPlkSeOoQ==",
|
||||
"version": "0.22.13",
|
||||
"resolved": "https://registry.npmjs.org/@druid-toolkit/query/-/query-0.22.13.tgz",
|
||||
"integrity": "sha512-p0Cmmbk55vLaYs2WWcUr09qDRU2IrkXOxGgUG+wS6Uuq/ALBqSmUDlbMSxB3vJjMvegiwgJ8+n7VfVpO0t/bJg==",
|
||||
"dependencies": {
|
||||
"tslib": "^2.5.2"
|
||||
}
|
||||
|
@ -19146,9 +19146,9 @@
|
|||
"dev": true
|
||||
},
|
||||
"@druid-toolkit/query": {
|
||||
"version": "0.22.11",
|
||||
"resolved": "https://registry.npmjs.org/@druid-toolkit/query/-/query-0.22.11.tgz",
|
||||
"integrity": "sha512-VVEn/tsEr9fb+8eKc+nu3/YH7l+LZ1vd0D32UDo66GLS3cI+EKOCM7VYC8lTvB1tAS+98w/EzfbdlRPlkSeOoQ==",
|
||||
"version": "0.22.13",
|
||||
"resolved": "https://registry.npmjs.org/@druid-toolkit/query/-/query-0.22.13.tgz",
|
||||
"integrity": "sha512-p0Cmmbk55vLaYs2WWcUr09qDRU2IrkXOxGgUG+wS6Uuq/ALBqSmUDlbMSxB3vJjMvegiwgJ8+n7VfVpO0t/bJg==",
|
||||
"requires": {
|
||||
"tslib": "^2.5.2"
|
||||
}
|
||||
|
|
|
@ -68,7 +68,7 @@
|
|||
"@blueprintjs/datetime2": "^0.9.35",
|
||||
"@blueprintjs/icons": "^4.16.0",
|
||||
"@blueprintjs/popover2": "^1.14.9",
|
||||
"@druid-toolkit/query": "^0.22.11",
|
||||
"@druid-toolkit/query": "^0.22.13",
|
||||
"@druid-toolkit/visuals-core": "^0.3.3",
|
||||
"@druid-toolkit/visuals-react": "^0.3.3",
|
||||
"ace-builds": "~1.4.14",
|
||||
|
|
|
@ -104,7 +104,7 @@ export const RecordTablePane = React.memo(function RecordTablePane(props: Record
|
|||
const finalPage =
|
||||
hasMoreResults && Math.floor(queryResult.rows.length / pagination.pageSize) === pagination.page; // on the last page
|
||||
|
||||
const numericColumnBraces = getNumericColumnBraces(queryResult, pagination);
|
||||
const numericColumnBraces = getNumericColumnBraces(queryResult, undefined, pagination);
|
||||
return (
|
||||
<div className={classNames('record-table-pane', { 'more-results': hasMoreResults })}>
|
||||
{finalPage ? (
|
||||
|
|
|
@ -38,8 +38,8 @@ export interface AsyncActionDialogProps {
|
|||
className?: string;
|
||||
icon?: IconName;
|
||||
intent?: Intent;
|
||||
successText: string;
|
||||
failText: string;
|
||||
successText: ReactNode;
|
||||
failText: ReactNode;
|
||||
warningChecks?: ReactNode[];
|
||||
children?: ReactNode;
|
||||
}
|
||||
|
|
|
@ -66,7 +66,12 @@ export const KillDatasourceDialog = function KillDatasourceDialog(
|
|||
return resp.data;
|
||||
}}
|
||||
confirmButtonText="Permanently delete unused segments"
|
||||
successText="Kill task was issued. Unused segments in datasource will be deleted"
|
||||
successText={
|
||||
<>
|
||||
Kill task was issued. Unused segments in datasource <Tag minimal>{datasource}</Tag> will
|
||||
be deleted
|
||||
</>
|
||||
}
|
||||
failText="Failed submit kill task"
|
||||
intent={Intent.DANGER}
|
||||
onClose={onClose}
|
||||
|
|
|
@ -440,7 +440,10 @@ export class Execution {
|
|||
value.queryContext = queryContext;
|
||||
const parsedQuery = parseSqlQuery(sqlQuery);
|
||||
if (value.result && (parsedQuery || queryContext)) {
|
||||
value.result = value.result.attachQuery({ context: queryContext }, parsedQuery);
|
||||
value.result = value.result.attachQuery(
|
||||
{ ...this.nativeQuery, context: queryContext },
|
||||
parsedQuery,
|
||||
);
|
||||
}
|
||||
|
||||
return new Execution(value);
|
||||
|
@ -463,7 +466,10 @@ export class Execution {
|
|||
public changeResult(result: QueryResult): Execution {
|
||||
return new Execution({
|
||||
...this.valueOf(),
|
||||
result: result.attachQuery({}, this.sqlQuery ? parseSqlQuery(this.sqlQuery) : undefined),
|
||||
result: result.attachQuery(
|
||||
this.nativeQuery,
|
||||
this.sqlQuery ? parseSqlQuery(this.sqlQuery) : undefined,
|
||||
),
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
|
||||
import type {
|
||||
QueryParameter,
|
||||
QueryPayload,
|
||||
SqlClusteredByClause,
|
||||
SqlExpression,
|
||||
SqlPartitionedByClause,
|
||||
|
@ -446,7 +447,7 @@ export class WorkbenchQuery {
|
|||
|
||||
public getApiQuery(makeQueryId: () => string = uuidv4): {
|
||||
engine: DruidEngine;
|
||||
query: Record<string, any>;
|
||||
query: QueryPayload;
|
||||
prefixLines: number;
|
||||
cancelQueryId?: string;
|
||||
} {
|
||||
|
@ -478,7 +479,7 @@ export class WorkbenchQuery {
|
|||
};
|
||||
}
|
||||
|
||||
let apiQuery: Record<string, any> = {};
|
||||
let apiQuery: QueryPayload;
|
||||
if (this.isJsonLike()) {
|
||||
try {
|
||||
apiQuery = Hjson.parse(queryString);
|
||||
|
|
|
@ -338,6 +338,22 @@ export function pluralIfNeeded(n: NumberLike, singular: string, plural?: string)
|
|||
|
||||
// ----------------------------
|
||||
|
||||
export function partition<T>(xs: T[], predicate: (x: T, i: number) => boolean): [T[], T[]] {
|
||||
const match: T[] = [];
|
||||
const nonMatch: T[] = [];
|
||||
|
||||
for (let i = 0; i < xs.length; i++) {
|
||||
const x = xs[i];
|
||||
if (predicate(x, i)) {
|
||||
match.push(x);
|
||||
} else {
|
||||
nonMatch.push(x);
|
||||
}
|
||||
}
|
||||
|
||||
return [match, nonMatch];
|
||||
}
|
||||
|
||||
export function filterMap<T, Q>(xs: readonly T[], f: (x: T, i: number) => Q | undefined): Q[] {
|
||||
return xs.map(f).filter((x: Q | undefined) => typeof x !== 'undefined') as Q[];
|
||||
}
|
||||
|
|
|
@ -32,9 +32,16 @@ export function changePage(pagination: Pagination, page: number): Pagination {
|
|||
return deepSet(pagination, 'page', page);
|
||||
}
|
||||
|
||||
export interface ColumnHint {
|
||||
displayName?: string;
|
||||
group?: string;
|
||||
formatter?: (x: any) => string;
|
||||
}
|
||||
|
||||
export function getNumericColumnBraces(
|
||||
queryResult: QueryResult,
|
||||
pagination?: Pagination,
|
||||
columnHints: Map<string, ColumnHint> | undefined,
|
||||
pagination: Pagination | undefined,
|
||||
): Record<number, string[]> {
|
||||
let rows = queryResult.rows;
|
||||
|
||||
|
@ -47,8 +54,9 @@ export function getNumericColumnBraces(
|
|||
if (rows.length) {
|
||||
queryResult.header.forEach((column, i) => {
|
||||
if (!oneOf(column.nativeType, 'LONG', 'FLOAT', 'DOUBLE')) return;
|
||||
const formatter = columnHints?.get(column.name)?.formatter || formatNumber;
|
||||
const brace = filterMap(rows, row =>
|
||||
oneOf(typeof row[i], 'number', 'bigint') ? formatNumber(row[i]) : undefined,
|
||||
oneOf(typeof row[i], 'number', 'bigint') ? formatter(row[i]) : undefined,
|
||||
);
|
||||
if (rows.length === brace.length) {
|
||||
numericColumnBraces[i] = brace;
|
||||
|
|
|
@ -16,7 +16,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import { FormGroup, InputGroup, Intent, MenuItem, Switch } from '@blueprintjs/core';
|
||||
import { FormGroup, InputGroup, Intent, MenuItem, Switch, Tag } from '@blueprintjs/core';
|
||||
import { IconNames } from '@blueprintjs/icons';
|
||||
import { SqlQuery, T } from '@druid-toolkit/query';
|
||||
import classNames from 'classnames';
|
||||
|
@ -651,8 +651,18 @@ GROUP BY 1, 2`;
|
|||
return resp.data;
|
||||
}}
|
||||
confirmButtonText="Mark as unused all segments"
|
||||
successText="All segments in datasource have been marked as unused"
|
||||
failText="Failed to mark as unused all segments in datasource"
|
||||
successText={
|
||||
<>
|
||||
All segments in datasource <Tag minimal>{datasourceToMarkAsUnusedAllSegmentsIn}</Tag>{' '}
|
||||
have been marked as unused
|
||||
</>
|
||||
}
|
||||
failText={
|
||||
<>
|
||||
Failed to mark as unused all segments in datasource{' '}
|
||||
<Tag minimal>{datasourceToMarkAsUnusedAllSegmentsIn}</Tag>
|
||||
</>
|
||||
}
|
||||
intent={Intent.DANGER}
|
||||
onClose={() => {
|
||||
this.setState({ datasourceToMarkAsUnusedAllSegmentsIn: undefined });
|
||||
|
@ -684,8 +694,19 @@ GROUP BY 1, 2`;
|
|||
return resp.data;
|
||||
}}
|
||||
confirmButtonText="Mark as used all segments"
|
||||
successText="All non-overshadowed segments in datasource have been marked as used"
|
||||
failText="Failed to mark as used all non-overshadowed segments in datasource"
|
||||
successText={
|
||||
<>
|
||||
All non-overshadowed segments in datasource{' '}
|
||||
<Tag minimal>{datasourceToMarkAllNonOvershadowedSegmentsAsUsedIn}</Tag> have been marked
|
||||
as used
|
||||
</>
|
||||
}
|
||||
failText={
|
||||
<>
|
||||
Failed to mark as used all non-overshadowed segments in datasource{' '}
|
||||
<Tag minimal>{datasourceToMarkAllNonOvershadowedSegmentsAsUsedIn}</Tag>
|
||||
</>
|
||||
}
|
||||
intent={Intent.PRIMARY}
|
||||
onClose={() => {
|
||||
this.setState({ datasourceToMarkAllNonOvershadowedSegmentsAsUsedIn: undefined });
|
||||
|
|
|
@ -31,7 +31,7 @@ import ReactTable from 'react-table';
|
|||
import { BracedText, Deferred, TableCell } from '../../../../../components';
|
||||
import { possibleDruidFormatForValues, TIME_COLUMN } from '../../../../../druid-models';
|
||||
import { SMALL_TABLE_PAGE_SIZE, SMALL_TABLE_PAGE_SIZE_OPTIONS } from '../../../../../react-table';
|
||||
import type { Pagination, QueryAction } from '../../../../../utils';
|
||||
import type { ColumnHint, Pagination, QueryAction } from '../../../../../utils';
|
||||
import {
|
||||
columnToIcon,
|
||||
columnToWidth,
|
||||
|
@ -60,30 +60,34 @@ function isComparable(x: unknown): boolean {
|
|||
return x !== null && x !== '';
|
||||
}
|
||||
|
||||
function columnNester(columns: TableColumn[], groupHints: string[] | undefined): TableColumn[] {
|
||||
if (!groupHints) return columns;
|
||||
function columnNester(
|
||||
tableColumns: TableColumn[],
|
||||
resultColumns: readonly Column[],
|
||||
columnHints: Map<string, ColumnHint> | undefined,
|
||||
): TableColumn[] {
|
||||
if (!columnHints) return tableColumns;
|
||||
|
||||
const ret: TableColumn[] = [];
|
||||
let currentGroupHint: string | null = null;
|
||||
let currentGroupName: string | null = null;
|
||||
let currentColumnGroup: TableColumn | null = null;
|
||||
for (let i = 0; i < columns.length; i++) {
|
||||
const column = columns[i];
|
||||
const groupHint = groupHints[i];
|
||||
if (groupHint) {
|
||||
if (currentGroupHint === groupHint) {
|
||||
currentColumnGroup!.columns!.push(column);
|
||||
for (let i = 0; i < tableColumns.length; i++) {
|
||||
const tableColumn = tableColumns[i];
|
||||
const group = columnHints.get(resultColumns[i].name)?.group;
|
||||
if (group) {
|
||||
if (currentGroupName === group) {
|
||||
currentColumnGroup!.columns!.push(tableColumn);
|
||||
} else {
|
||||
currentGroupHint = groupHint;
|
||||
currentGroupName = group;
|
||||
ret.push(
|
||||
(currentColumnGroup = {
|
||||
Header: <div className="group-cell">{currentGroupHint}</div>,
|
||||
columns: [column],
|
||||
Header: <div className="group-cell">{currentGroupName}</div>,
|
||||
columns: [tableColumn],
|
||||
}),
|
||||
);
|
||||
}
|
||||
} else {
|
||||
ret.push(column);
|
||||
currentGroupHint = null;
|
||||
ret.push(tableColumn);
|
||||
currentGroupName = null;
|
||||
currentColumnGroup = null;
|
||||
}
|
||||
}
|
||||
|
@ -94,12 +98,12 @@ function columnNester(columns: TableColumn[], groupHints: string[] | undefined):
|
|||
export interface GenericOutputTableProps {
|
||||
queryResult: QueryResult;
|
||||
onQueryAction(action: QueryAction): void;
|
||||
onOrderByChange?(columnIndex: number, desc: boolean): void;
|
||||
onOrderByChange?(columnName: string, desc: boolean): void;
|
||||
onExport?(): void;
|
||||
runeMode: boolean;
|
||||
showTypeIcons: boolean;
|
||||
initPageSize?: number;
|
||||
groupHints?: string[];
|
||||
columnHints?: Map<string, ColumnHint>;
|
||||
}
|
||||
|
||||
export const GenericOutputTable = React.memo(function GenericOutputTable(
|
||||
|
@ -113,7 +117,7 @@ export const GenericOutputTable = React.memo(function GenericOutputTable(
|
|||
runeMode,
|
||||
showTypeIcons,
|
||||
initPageSize,
|
||||
groupHints,
|
||||
columnHints,
|
||||
} = props;
|
||||
const parsedQuery = queryResult.sqlQuery;
|
||||
const [pagination, setPagination] = useState<Pagination>({
|
||||
|
@ -159,7 +163,7 @@ export const GenericOutputTable = React.memo(function GenericOutputTable(
|
|||
icon={reverseOrderByDirection === 'ASC' ? IconNames.SORT_ASC : IconNames.SORT_DESC}
|
||||
text={`Order ${reverseOrderByDirection === 'ASC' ? 'ascending' : 'descending'}`}
|
||||
onClick={() => {
|
||||
onOrderByChange(headerIndex, reverseOrderByDirection !== 'ASC');
|
||||
onOrderByChange(header, reverseOrderByDirection !== 'ASC');
|
||||
}}
|
||||
/>,
|
||||
);
|
||||
|
@ -170,7 +174,7 @@ export const GenericOutputTable = React.memo(function GenericOutputTable(
|
|||
icon={IconNames.SORT_DESC}
|
||||
text="Order descending"
|
||||
onClick={() => {
|
||||
onOrderByChange(headerIndex, true);
|
||||
onOrderByChange(header, true);
|
||||
}}
|
||||
/>,
|
||||
<MenuItem
|
||||
|
@ -178,7 +182,7 @@ export const GenericOutputTable = React.memo(function GenericOutputTable(
|
|||
icon={IconNames.SORT_ASC}
|
||||
text="Order ascending"
|
||||
onClick={() => {
|
||||
onOrderByChange(headerIndex, false);
|
||||
onOrderByChange(header, false);
|
||||
}}
|
||||
/>,
|
||||
);
|
||||
|
@ -426,7 +430,7 @@ export const GenericOutputTable = React.memo(function GenericOutputTable(
|
|||
const finalPage =
|
||||
hasMoreResults && Math.floor(queryResult.rows.length / pagination.pageSize) === pagination.page; // on the last page
|
||||
|
||||
const numericColumnBraces = getNumericColumnBraces(queryResult, pagination);
|
||||
const numericColumnBraces = getNumericColumnBraces(queryResult, columnHints, pagination);
|
||||
return (
|
||||
<div className={classNames('generic-output-table', { 'more-results': hasMoreResults })}>
|
||||
{finalPage ? (
|
||||
|
@ -479,7 +483,7 @@ export const GenericOutputTable = React.memo(function GenericOutputTable(
|
|||
<div className="clickable-cell">
|
||||
<div className="output-name">
|
||||
{icon && <Icon className="type-icon" icon={icon} size={12} />}
|
||||
{h}
|
||||
{columnHints?.get(h)?.displayName ?? h}
|
||||
{hasFilterOnHeader(h, i) && <Icon icon={IconNames.FILTER} size={14} />}
|
||||
</div>
|
||||
</div>
|
||||
|
@ -490,6 +494,7 @@ export const GenericOutputTable = React.memo(function GenericOutputTable(
|
|||
accessor: String(i),
|
||||
Cell(row) {
|
||||
const value = row.value;
|
||||
const formatter = columnHints?.get(h)?.formatter || formatNumber;
|
||||
return (
|
||||
<div>
|
||||
<Popover2
|
||||
|
@ -498,7 +503,7 @@ export const GenericOutputTable = React.memo(function GenericOutputTable(
|
|||
{numericColumnBraces[i] ? (
|
||||
<BracedText
|
||||
className="table-padding"
|
||||
text={formatNumber(value)}
|
||||
text={formatter(value)}
|
||||
braces={numericColumnBraces[i]}
|
||||
padFractionalPart
|
||||
/>
|
||||
|
@ -516,7 +521,8 @@ export const GenericOutputTable = React.memo(function GenericOutputTable(
|
|||
: undefined,
|
||||
};
|
||||
}),
|
||||
groupHints,
|
||||
queryResult.header,
|
||||
columnHints,
|
||||
)}
|
||||
/>
|
||||
)}
|
||||
|
|
|
@ -16,11 +16,13 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import type { SqlOrderByExpression } from '@druid-toolkit/query';
|
||||
import { Button } from '@blueprintjs/core';
|
||||
import type { SqlOrderByExpression, SqlTable } from '@druid-toolkit/query';
|
||||
import {
|
||||
C,
|
||||
F,
|
||||
SqlCase,
|
||||
SqlColumn,
|
||||
SqlExpression,
|
||||
SqlFunction,
|
||||
SqlLiteral,
|
||||
|
@ -35,15 +37,23 @@ import ReactDOM from 'react-dom';
|
|||
|
||||
import { Loader } from '../../../components';
|
||||
import { useQueryManager } from '../../../hooks';
|
||||
import type { ColumnHint } from '../../../utils';
|
||||
import { formatInteger, formatPercent } from '../../../utils';
|
||||
import { getInitQuery } from '../utils';
|
||||
|
||||
import { GenericOutputTable } from './components';
|
||||
import { shiftTimeInWhere } from './utils/utils';
|
||||
import { getWhereForCompares, shiftTimeInExpression } from './utils/utils';
|
||||
|
||||
import './table-react-module.scss';
|
||||
|
||||
type MultipleValueMode = 'null' | 'empty' | 'latest' | 'latestNonNull' | 'count';
|
||||
|
||||
type CompareType = 'value' | 'delta' | 'absDelta' | 'percent' | 'absPercent';
|
||||
|
||||
// As of this writing ordering the outer query on something other than __time sometimes throws an error, set this to false / remove it
|
||||
// when ordering on non __time is more robust
|
||||
const NEEDS_GROUPING_TO_ORDER = true;
|
||||
|
||||
const KNOWN_AGGREGATIONS = [
|
||||
'COUNT',
|
||||
'SUM',
|
||||
|
@ -73,13 +83,37 @@ const KNOWN_AGGREGATIONS = [
|
|||
'ANY_VALUE',
|
||||
];
|
||||
|
||||
const TOP_VALUES_NAME = 'top_values';
|
||||
const TOP_VALUES_K = 5000;
|
||||
|
||||
function coalesce0(ex: SqlExpression) {
|
||||
return F('COALESCE', ex, SqlLiteral.ZERO);
|
||||
}
|
||||
|
||||
function safeDivide0(a: SqlExpression, b: SqlExpression) {
|
||||
return coalesce0(F('SAFE_DIVIDE', a, b));
|
||||
}
|
||||
|
||||
function anyValue(ex: SqlExpression) {
|
||||
return F('ANY_VALUE', ex);
|
||||
}
|
||||
|
||||
function addTableScope(expression: SqlExpression, newTableScope: string): SqlExpression {
|
||||
return expression.walk(ex => {
|
||||
if (ex instanceof SqlColumn && !ex.getTableName()) {
|
||||
return ex.changeTableName(newTableScope);
|
||||
}
|
||||
return ex;
|
||||
}) as SqlExpression;
|
||||
}
|
||||
|
||||
function toGroupByExpression(
|
||||
splitColumn: ExpressionMeta,
|
||||
timeBucket: string,
|
||||
compareShiftDuration?: string,
|
||||
) {
|
||||
const { expression, sqlType, name } = splitColumn;
|
||||
return expression
|
||||
return addTableScope(expression, 't')
|
||||
.applyIf(sqlType === 'TIMESTAMP' && compareShiftDuration, e =>
|
||||
F.timeShift(e, compareShiftDuration!, 1),
|
||||
)
|
||||
|
@ -131,9 +165,21 @@ function toShowColumnExpression(
|
|||
return ex.as(showColumn.name);
|
||||
}
|
||||
|
||||
function getJoinCondition(
|
||||
splitColumns: ExpressionMeta[],
|
||||
table1: SqlTable,
|
||||
table2: SqlTable,
|
||||
): SqlExpression {
|
||||
return SqlExpression.and(
|
||||
...splitColumns.map(splitColumn =>
|
||||
table1.column(splitColumn.name).isNotDistinctFrom(table2.column(splitColumn.name)),
|
||||
),
|
||||
);
|
||||
}
|
||||
|
||||
interface QueryAndHints {
|
||||
query: SqlQuery;
|
||||
groupHints: string[];
|
||||
columnHints: Map<string, ColumnHint>;
|
||||
}
|
||||
|
||||
export default typedVisualModule({
|
||||
|
@ -200,13 +246,14 @@ export default typedVisualModule({
|
|||
|
||||
compares: {
|
||||
type: 'options',
|
||||
options: ['PT1M', 'PT5M', 'PT1H', 'P1D', 'P1M'],
|
||||
options: ['PT1M', 'PT5M', 'PT1H', 'PT6H', 'P1D', 'P1M'],
|
||||
control: {
|
||||
label: 'Compares',
|
||||
optionLabels: {
|
||||
PT1M: '1 minute',
|
||||
PT5M: '5 minutes',
|
||||
PT1H: '1 hour',
|
||||
PT6H: '6 hours',
|
||||
P1D: '1 day',
|
||||
P1M: '1 month',
|
||||
},
|
||||
|
@ -214,12 +261,31 @@ export default typedVisualModule({
|
|||
},
|
||||
},
|
||||
|
||||
showDelta: {
|
||||
type: 'boolean',
|
||||
compareTypes: {
|
||||
type: 'options',
|
||||
options: ['value', 'delta', 'absDelta', 'percent', 'absPercent'],
|
||||
default: ['value', 'delta'],
|
||||
control: {
|
||||
visible: ({ params }) => Boolean((params.compares || []).length),
|
||||
label: 'Compare types',
|
||||
visible: ({ params }) => Boolean((params.compares || []).length) && !params.pivotColumn,
|
||||
optionLabels: {
|
||||
value: 'Value',
|
||||
delta: 'Delta',
|
||||
absDelta: 'Abs. delta',
|
||||
percent: 'Percent',
|
||||
absPercent: 'Abs. percent',
|
||||
},
|
||||
},
|
||||
},
|
||||
restrictTop: {
|
||||
type: 'boolean',
|
||||
default: true,
|
||||
control: {
|
||||
label: `Restrict to top ${formatInteger(TOP_VALUES_K)} when ordering on delta`,
|
||||
visible: ({ params }) => Boolean((params.compares || []).length) && !params.pivotColumn,
|
||||
},
|
||||
},
|
||||
|
||||
maxRows: {
|
||||
type: 'number',
|
||||
default: 200,
|
||||
|
@ -287,7 +353,7 @@ function TableModule(props: TableModuleProps) {
|
|||
},
|
||||
});
|
||||
|
||||
const queryAndHints = useMemo(() => {
|
||||
const queryAndHints = useMemo((): QueryAndHints | undefined => {
|
||||
const splitColumns: ExpressionMeta[] = parameterValues.splitColumns;
|
||||
const timeBucket: string = parameterValues.timeBucket || 'PT1H';
|
||||
const showColumns: ExpressionMeta[] = parameterValues.showColumns;
|
||||
|
@ -295,26 +361,71 @@ function TableModule(props: TableModuleProps) {
|
|||
const pivotColumn: ExpressionMeta = parameterValues.pivotColumn;
|
||||
const metrics: ExpressionMeta[] = parameterValues.metrics;
|
||||
const compares: string[] = parameterValues.compares || [];
|
||||
const showDelta: boolean = parameterValues.showDelta;
|
||||
const compareTypes: CompareType[] = parameterValues.compareTypes;
|
||||
const restrictTop: boolean = parameterValues.restrictTop;
|
||||
const maxRows: number = parameterValues.maxRows;
|
||||
|
||||
const pivotValues = pivotColumn ? pivotValueState.data : undefined;
|
||||
if (pivotColumn && !pivotValues) return;
|
||||
|
||||
const hasCompare = Boolean(compares.length);
|
||||
const effectiveOrderBy =
|
||||
orderBy || C(metrics[0]?.name || splitColumns[0]?.name).toOrderByExpression('DESC');
|
||||
|
||||
const hasCompare = !pivotColumn && Boolean(compares.length) && Boolean(compareTypes.length);
|
||||
|
||||
const orderByColumnName = (effectiveOrderBy.expression as SqlColumn).getName();
|
||||
let orderByCompareMeasure: string | undefined;
|
||||
let orderByCompareDuration: string | undefined;
|
||||
let orderByCompareType: CompareType | undefined;
|
||||
if (hasCompare) {
|
||||
const m = orderByColumnName.match(
|
||||
/^(.+):cmp:([^:]+):(value|delta|absDelta|percent|absPercent)$/,
|
||||
);
|
||||
if (m) {
|
||||
orderByCompareMeasure = m[1];
|
||||
orderByCompareDuration = m[2];
|
||||
orderByCompareType = m[3] as CompareType;
|
||||
}
|
||||
}
|
||||
|
||||
const metricExpression = metrics.find(m => m.name === orderByCompareMeasure)?.expression;
|
||||
const topValuesQuery =
|
||||
restrictTop && metricExpression && orderByCompareType !== 'value' && splitColumns.length
|
||||
? getInitQuery(table, getWhereForCompares(where, compares))
|
||||
.applyForEach(splitColumns, (q, splitColumn) =>
|
||||
q.addSelect(toGroupByExpression(splitColumn, timeBucket), {
|
||||
addToGroupBy: 'end',
|
||||
}),
|
||||
)
|
||||
.changeOrderByExpression(metricExpression.toOrderByExpression('DESC'))
|
||||
.changeLimitValue(TOP_VALUES_K)
|
||||
: undefined;
|
||||
|
||||
const columnHints = new Map<string, ColumnHint>();
|
||||
const mainQuery = getInitQuery(table, where)
|
||||
.applyIf(topValuesQuery, q =>
|
||||
q.addInnerJoin(
|
||||
T(TOP_VALUES_NAME),
|
||||
getJoinCondition(splitColumns, T('t'), T(TOP_VALUES_NAME)),
|
||||
),
|
||||
)
|
||||
.applyForEach(splitColumns, (q, splitColumn) =>
|
||||
q.addSelect(toGroupByExpression(splitColumn, timeBucket), {
|
||||
addToGroupBy: 'end',
|
||||
}),
|
||||
)
|
||||
.applyForEach(showColumns, (q, showColumn) =>
|
||||
q.addSelect(toShowColumnExpression(showColumn, multipleValueMode)),
|
||||
.applyIf(!orderByCompareDuration, q =>
|
||||
q.applyForEach(showColumns, (q, showColumn) =>
|
||||
q.addSelect(toShowColumnExpression(showColumn, multipleValueMode)),
|
||||
),
|
||||
)
|
||||
.applyForEach(pivotValues || [''], (q, pivotValue, i) =>
|
||||
q.applyForEach(metrics, (q, metric) =>
|
||||
q.addSelect(
|
||||
q.applyForEach(metrics, (q, metric) => {
|
||||
const alias = `${metric.name}${pivotColumn && i > 0 ? `:${pivotValue}` : ''}`;
|
||||
if (pivotColumn) {
|
||||
columnHints.set(alias, { displayName: metric.name, group: pivotValue });
|
||||
}
|
||||
return q.addSelect(
|
||||
metric.expression
|
||||
.as(metric.name)
|
||||
.applyIf(pivotColumn, q =>
|
||||
|
@ -323,115 +434,204 @@ function TableModule(props: TableModuleProps) {
|
|||
pivotColumn.expression.equal(pivotValue),
|
||||
KNOWN_AGGREGATIONS,
|
||||
)
|
||||
.as(`${metric.name}${i > 0 ? ` [${pivotValue}]` : ''}`),
|
||||
.as(alias),
|
||||
),
|
||||
),
|
||||
),
|
||||
);
|
||||
}),
|
||||
)
|
||||
.applyIf(metrics.length > 0 || splitColumns.length > 0, q =>
|
||||
q.changeOrderByExpression(
|
||||
orderBy || C(metrics[0]?.name || splitColumns[0]?.name).toOrderByExpression('DESC'),
|
||||
),
|
||||
)
|
||||
.changeLimitValue(maxRows);
|
||||
.applyIf(!orderByCompareDuration, q =>
|
||||
q
|
||||
.applyIf(metrics.length > 0 || splitColumns.length > 0, q =>
|
||||
q.changeOrderByExpression(effectiveOrderBy),
|
||||
)
|
||||
.changeLimitValue(maxRows),
|
||||
);
|
||||
|
||||
if (!hasCompare) {
|
||||
return {
|
||||
query: mainQuery,
|
||||
groupHints: pivotColumn
|
||||
? splitColumns
|
||||
.map(() => '')
|
||||
.concat(
|
||||
showColumns.map(() => ''),
|
||||
(pivotValues || []).flatMap(v => metrics.map(() => v)),
|
||||
)
|
||||
: [],
|
||||
columnHints,
|
||||
};
|
||||
}
|
||||
|
||||
const main = T('main');
|
||||
return {
|
||||
query: SqlQuery.from(main)
|
||||
.changeWithParts(
|
||||
[SqlWithPart.simple('main', mainQuery)].concat(
|
||||
compares.map((comparePeriod, i) =>
|
||||
SqlWithPart.simple(
|
||||
`compare${i}`,
|
||||
getInitQuery(table, shiftTimeInWhere(where, comparePeriod))
|
||||
.applyForEach(splitColumns, (q, splitColumn) =>
|
||||
q.addSelect(toGroupByExpression(splitColumn, timeBucket, comparePeriod), {
|
||||
addToGroupBy: 'end',
|
||||
}),
|
||||
)
|
||||
.applyForEach(metrics, (q, metric) =>
|
||||
q.addSelect(metric.expression.as(metric.name)),
|
||||
const leader = T(orderByCompareDuration ? `compare_${orderByCompareDuration}` : 'main');
|
||||
const query = SqlQuery.from(leader)
|
||||
.changeWithParts(
|
||||
(
|
||||
(topValuesQuery
|
||||
? [SqlWithPart.simple(TOP_VALUES_NAME, topValuesQuery)]
|
||||
: []) as SqlWithPart[]
|
||||
).concat(
|
||||
SqlWithPart.simple('main', mainQuery),
|
||||
compares.map(compare =>
|
||||
SqlWithPart.simple(
|
||||
`compare_${compare}`,
|
||||
getInitQuery(table, shiftTimeInExpression(where, compare))
|
||||
.applyIf(topValuesQuery, q =>
|
||||
q.addInnerJoin(
|
||||
T(TOP_VALUES_NAME),
|
||||
getJoinCondition(splitColumns, T('t'), T(TOP_VALUES_NAME)),
|
||||
),
|
||||
),
|
||||
)
|
||||
.applyForEach(splitColumns, (q, splitColumn) =>
|
||||
q.addSelect(toGroupByExpression(splitColumn, timeBucket, compare), {
|
||||
addToGroupBy: 'end',
|
||||
}),
|
||||
)
|
||||
.applyIf(orderByCompareDuration === compare, q =>
|
||||
q.applyForEach(showColumns, (q, showColumn) =>
|
||||
q.addSelect(toShowColumnExpression(showColumn, multipleValueMode)),
|
||||
),
|
||||
)
|
||||
.applyForEach(metrics, (q, metric) =>
|
||||
q.addSelect(metric.expression.as(metric.name)),
|
||||
)
|
||||
.applyIf(compare === orderByCompareDuration && orderByCompareType === 'value', q =>
|
||||
q
|
||||
.changeOrderByExpression(
|
||||
effectiveOrderBy.changeExpression(C(orderByCompareMeasure!)),
|
||||
)
|
||||
.changeLimitValue(maxRows),
|
||||
),
|
||||
),
|
||||
),
|
||||
)
|
||||
.changeSelectExpressions(
|
||||
splitColumns
|
||||
.map(splitColumn => main.column(splitColumn.name).as(splitColumn.name))
|
||||
.concat(
|
||||
showColumns.map(showColumn => main.column(showColumn.name).as(showColumn.name)),
|
||||
metrics.map(metric => main.column(metric.name).as(metric.name)),
|
||||
compares.flatMap((_, i) =>
|
||||
metrics.flatMap(metric => {
|
||||
const c = T(`compare${i}`).column(metric.name);
|
||||
|
||||
const ret = [SqlFunction.simple('COALESCE', [c, 0]).as(`#prev: ${metric.name}`)];
|
||||
|
||||
if (showDelta) {
|
||||
ret.push(
|
||||
F.stringFormat(
|
||||
'%.1f%%',
|
||||
SqlFunction.simple('SAFE_DIVIDE', [
|
||||
SqlExpression.parse(`(${main.column(metric.name)} - ${c}) * 100.0`),
|
||||
c,
|
||||
]),
|
||||
).as(`%chg: ${metric.name}`),
|
||||
);
|
||||
}
|
||||
|
||||
return ret;
|
||||
}),
|
||||
),
|
||||
),
|
||||
)
|
||||
.changeSelectExpressions(
|
||||
splitColumns
|
||||
.map(splitColumn => main.column(splitColumn.name).as(splitColumn.name))
|
||||
.concat(
|
||||
showColumns.map(showColumn =>
|
||||
leader
|
||||
.column(showColumn.name)
|
||||
.applyIf(NEEDS_GROUPING_TO_ORDER, anyValue)
|
||||
.as(showColumn.name),
|
||||
),
|
||||
)
|
||||
.applyForEach(compares, (q, _comparePeriod, i) =>
|
||||
metrics.map(metric =>
|
||||
main
|
||||
.column(metric.name)
|
||||
.applyIf(NEEDS_GROUPING_TO_ORDER, anyValue)
|
||||
.applyIf(orderByCompareDuration, coalesce0)
|
||||
.as(metric.name),
|
||||
),
|
||||
compares.flatMap(compare =>
|
||||
metrics.flatMap(metric => {
|
||||
const c = T(`compare_${compare}`)
|
||||
.column(metric.name)
|
||||
.applyIf(NEEDS_GROUPING_TO_ORDER, anyValue)
|
||||
.applyIf(compare !== orderByCompareDuration, coalesce0);
|
||||
|
||||
const mainMetric = main
|
||||
.column(metric.name)
|
||||
.applyIf(NEEDS_GROUPING_TO_ORDER, anyValue)
|
||||
.applyIf(orderByCompareDuration, coalesce0);
|
||||
|
||||
const diff = mainMetric.subtract(c);
|
||||
|
||||
const ret: SqlExpression[] = [];
|
||||
|
||||
if (compareTypes.includes('value')) {
|
||||
const valueName = `${metric.name}:cmp:${compare}:value`;
|
||||
columnHints.set(valueName, {
|
||||
group: `Comparison to ${compare}`,
|
||||
displayName: `${metric.name} (value)`,
|
||||
});
|
||||
ret.push(c.as(valueName));
|
||||
}
|
||||
|
||||
if (compareTypes.includes('delta')) {
|
||||
const deltaName = `${metric.name}:cmp:${compare}:delta`;
|
||||
columnHints.set(deltaName, {
|
||||
group: `Comparison to ${compare}`,
|
||||
displayName: `${metric.name} (delta)`,
|
||||
});
|
||||
ret.push(diff.as(deltaName));
|
||||
}
|
||||
|
||||
if (compareTypes.includes('absDelta')) {
|
||||
const deltaName = `${metric.name}:cmp:${compare}:absDelta`;
|
||||
columnHints.set(deltaName, {
|
||||
group: `Comparison to ${compare}`,
|
||||
displayName: `${metric.name} (Abs. delta)`,
|
||||
});
|
||||
ret.push(F('ABS', diff).as(deltaName));
|
||||
}
|
||||
|
||||
if (compareTypes.includes('percent')) {
|
||||
const percentName = `${metric.name}:cmp:${compare}:percent`;
|
||||
columnHints.set(percentName, {
|
||||
group: `Comparison to ${compare}`,
|
||||
displayName: `${metric.name} (%)`,
|
||||
formatter: formatPercent,
|
||||
});
|
||||
ret.push(
|
||||
safeDivide0(diff.multiply(SqlLiteral.ONE_POINT_ZERO), c).as(percentName),
|
||||
);
|
||||
}
|
||||
|
||||
if (compareTypes.includes('absPercent')) {
|
||||
const percentName = `${metric.name}:cmp:${compare}:absPercent`;
|
||||
columnHints.set(percentName, {
|
||||
group: `Comparison to ${compare}`,
|
||||
displayName: `${metric.name} (abs. %)`,
|
||||
formatter: formatPercent,
|
||||
});
|
||||
ret.push(
|
||||
F('ABS', safeDivide0(diff.multiply(SqlLiteral.ONE_POINT_ZERO), c)).as(
|
||||
percentName,
|
||||
),
|
||||
);
|
||||
}
|
||||
|
||||
return ret;
|
||||
}),
|
||||
),
|
||||
),
|
||||
)
|
||||
.applyIf(orderByCompareDuration, q =>
|
||||
q.addLeftJoin(
|
||||
main,
|
||||
getJoinCondition(splitColumns, main, T(`compare_${orderByCompareDuration}`)),
|
||||
),
|
||||
)
|
||||
.applyForEach(
|
||||
compares.filter(c => c !== orderByCompareDuration),
|
||||
(q, compare) =>
|
||||
q.addLeftJoin(
|
||||
T(`compare${i}`),
|
||||
SqlExpression.and(
|
||||
...splitColumns.map(splitColumn =>
|
||||
main
|
||||
.column(splitColumn.name)
|
||||
.isNotDistinctFrom(T(`compare${i}`).column(splitColumn.name)),
|
||||
),
|
||||
),
|
||||
T(`compare_${compare}`),
|
||||
getJoinCondition(splitColumns, main, T(`compare_${compare}`)),
|
||||
),
|
||||
),
|
||||
groupHints: splitColumns
|
||||
.map(() => 'Current')
|
||||
.concat(
|
||||
showColumns.map(() => 'Current'),
|
||||
metrics.map(() => 'Current'),
|
||||
compares.flatMap(comparePeriod =>
|
||||
metrics
|
||||
.flatMap(() => (showDelta ? ['', ''] : ['']))
|
||||
.map(() => `Comparison to ${comparePeriod}`),
|
||||
),
|
||||
),
|
||||
)
|
||||
.applyIf(NEEDS_GROUPING_TO_ORDER, q =>
|
||||
q.changeGroupByExpressions(splitColumns.map((_, i) => SqlLiteral.index(i))),
|
||||
)
|
||||
.addOrderBy(effectiveOrderBy)
|
||||
.changeLimitValue(maxRows);
|
||||
|
||||
for (const splitColumn of splitColumns) {
|
||||
columnHints.set(splitColumn.name, { group: 'Current' });
|
||||
}
|
||||
for (const showColumn of showColumns) {
|
||||
columnHints.set(showColumn.name, { group: 'Current' });
|
||||
}
|
||||
for (const metric of metrics) {
|
||||
columnHints.set(metric.name, { group: 'Current' });
|
||||
}
|
||||
|
||||
return {
|
||||
query,
|
||||
columnHints,
|
||||
};
|
||||
}, [table, where, parameterValues, orderBy, pivotValueState.data]);
|
||||
|
||||
const [resultState] = useQueryManager({
|
||||
query: queryAndHints,
|
||||
processQuery: async (queryAndHints: QueryAndHints) => {
|
||||
const { query, groupHints } = queryAndHints;
|
||||
const { query, columnHints } = queryAndHints;
|
||||
return {
|
||||
result: await sqlQuery(query),
|
||||
groupHints,
|
||||
columnHints,
|
||||
};
|
||||
},
|
||||
});
|
||||
|
@ -440,19 +640,24 @@ function TableModule(props: TableModuleProps) {
|
|||
return (
|
||||
<div className="table-module">
|
||||
{resultState.error ? (
|
||||
resultState.getErrorMessage()
|
||||
<div>
|
||||
<div>{resultState.getErrorMessage()}</div>
|
||||
{resultState.getErrorMessage()?.includes('not found in any table') && orderBy && (
|
||||
<Button text="Clear order by" onClick={() => setOrderBy(undefined)} />
|
||||
)}
|
||||
</div>
|
||||
) : resultData ? (
|
||||
<GenericOutputTable
|
||||
runeMode={false}
|
||||
queryResult={resultData.result}
|
||||
groupHints={resultData.groupHints}
|
||||
columnHints={resultData.columnHints}
|
||||
showTypeIcons={false}
|
||||
onOrderByChange={(headerIndex, desc) => {
|
||||
const idx = SqlLiteral.index(headerIndex);
|
||||
if (orderBy && String(orderBy.expression) === String(idx)) {
|
||||
onOrderByChange={(columnName, desc) => {
|
||||
const column = C(columnName);
|
||||
if (orderBy && orderBy.expression.equals(column)) {
|
||||
setOrderBy(orderBy.reverseDirection());
|
||||
} else {
|
||||
setOrderBy(idx.toOrderByExpression(desc ? 'DESC' : 'ASC'));
|
||||
setOrderBy(column.toOrderByExpression(desc ? 'DESC' : 'ASC'));
|
||||
}
|
||||
}}
|
||||
onQueryAction={action => {
|
||||
|
|
|
@ -18,21 +18,64 @@
|
|||
|
||||
import { SqlExpression } from '@druid-toolkit/query';
|
||||
|
||||
import { shiftTimeInWhere } from './utils';
|
||||
import { getWhereForCompares, shiftTimeInExpression } from './utils';
|
||||
|
||||
describe('shiftTimeInWhere', () => {
|
||||
it('works with TIME_IN_INTERVAL', () => {
|
||||
describe('getWhereForCompares', () => {
|
||||
it('works', () => {
|
||||
expect(
|
||||
shiftTimeInWhere(
|
||||
getWhereForCompares(
|
||||
SqlExpression.parse(
|
||||
`TIME_IN_INTERVAL("__time", '2016-06-27/2016-06-28') AND "country" = 'United States'`,
|
||||
),
|
||||
['PT1H', 'P1D'],
|
||||
).toString(),
|
||||
).toEqual(
|
||||
`(TIME_IN_INTERVAL("__time", '2016-06-27/2016-06-28') OR (TIME_SHIFT(TIMESTAMP '2016-06-27', 'PT1H', -1) <= "__time" AND "__time" < TIME_SHIFT(TIMESTAMP '2016-06-28', 'PT1H', -1)) OR (TIME_SHIFT(TIMESTAMP '2016-06-27', 'P1D', -1) <= "__time" AND "__time" < TIME_SHIFT(TIMESTAMP '2016-06-28', 'P1D', -1))) AND "country" = 'United States'`,
|
||||
);
|
||||
});
|
||||
});
|
||||
|
||||
describe('shiftTimeInExpression', () => {
|
||||
it('works with TIME_IN_INTERVAL (date)', () => {
|
||||
expect(
|
||||
shiftTimeInExpression(
|
||||
SqlExpression.parse(`TIME_IN_INTERVAL("__time", '2016-06-27/2016-06-28')`),
|
||||
'P1D',
|
||||
).toString(),
|
||||
).toEqual(`TIME_IN_INTERVAL(TIME_SHIFT("__time", 'P1D', 1), '2016-06-27/2016-06-28')`);
|
||||
).toEqual(
|
||||
`TIME_SHIFT(TIMESTAMP '2016-06-27', 'P1D', -1) <= "__time" AND "__time" < TIME_SHIFT(TIMESTAMP '2016-06-28', 'P1D', -1)`,
|
||||
);
|
||||
});
|
||||
|
||||
it('works with TIME_IN_INTERVAL (date and time)', () => {
|
||||
expect(
|
||||
shiftTimeInExpression(
|
||||
SqlExpression.parse(
|
||||
`TIME_IN_INTERVAL("__time", '2016-06-27T12:34:56/2016-06-28T12:34:56')`,
|
||||
),
|
||||
'P1D',
|
||||
).toString(),
|
||||
).toEqual(
|
||||
`TIME_SHIFT(TIMESTAMP '2016-06-27 12:34:56', 'P1D', -1) <= "__time" AND "__time" < TIME_SHIFT(TIMESTAMP '2016-06-28 12:34:56', 'P1D', -1)`,
|
||||
);
|
||||
});
|
||||
|
||||
it('works with TIME_IN_INTERVAL (date and time, zulu)', () => {
|
||||
expect(
|
||||
shiftTimeInExpression(
|
||||
SqlExpression.parse(
|
||||
`TIME_IN_INTERVAL("__time", '2016-06-27T12:34:56Z/2016-06-28T12:34:56Z')`,
|
||||
),
|
||||
'P1D',
|
||||
).toString(),
|
||||
).toEqual(
|
||||
`TIME_SHIFT(TIME_PARSE('2016-06-27 12:34:56', NULL, 'Etc/UTC'), 'P1D', -1) <= "__time" AND "__time" < TIME_SHIFT(TIME_PARSE('2016-06-28 12:34:56', NULL, 'Etc/UTC'), 'P1D', -1)`,
|
||||
);
|
||||
});
|
||||
|
||||
it('works with relative time', () => {
|
||||
expect(
|
||||
shiftTimeInWhere(
|
||||
shiftTimeInExpression(
|
||||
SqlExpression.parse(
|
||||
`(TIME_SHIFT(MAX_DATA_TIME(), 'PT1H', -1) <= "__time" AND "__time" < MAX_DATA_TIME())`,
|
||||
),
|
||||
|
@ -45,7 +88,7 @@ describe('shiftTimeInWhere', () => {
|
|||
|
||||
it('works with relative time (specific timestamps)', () => {
|
||||
expect(
|
||||
shiftTimeInWhere(
|
||||
shiftTimeInExpression(
|
||||
SqlExpression.parse(
|
||||
`TIMESTAMP '2016-06-27 20:31:02.498' <= "__time" AND "__time" < TIMESTAMP '2016-06-27 21:31:02.498'`,
|
||||
),
|
||||
|
|
|
@ -16,28 +16,77 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import type { SqlExpression } from '@druid-toolkit/query';
|
||||
import { F, SqlFunction, SqlLiteral } from '@druid-toolkit/query';
|
||||
import { F, SqlExpression, SqlFunction, SqlLiteral } from '@druid-toolkit/query';
|
||||
|
||||
export function shiftTimeInWhere(where: SqlExpression, period: string): SqlExpression {
|
||||
return where.walk(ex => {
|
||||
import { partition } from '../../../../utils';
|
||||
|
||||
const IS_DATE_LIKE = /^[+-]?\d\d\d\d[^']+$/;
|
||||
|
||||
function isoStringToTimestampLiteral(iso: string): SqlExpression {
|
||||
const zulu = iso.endsWith('Z');
|
||||
const cleanIso = iso.replace('T', ' ').replace('Z', '');
|
||||
let sql: string;
|
||||
if (zulu) {
|
||||
sql = `TIME_PARSE('${cleanIso}', NULL, 'Etc/UTC')`;
|
||||
} else {
|
||||
sql = `TIMESTAMP '${cleanIso}'`;
|
||||
}
|
||||
return SqlExpression.parse(sql);
|
||||
}
|
||||
|
||||
export function getWhereForCompares(where: SqlExpression, compares: string[]): SqlExpression {
|
||||
const whereParts = where.decomposeViaAnd({ flatten: true });
|
||||
const [timeExpressions, timelessExpressions] = partition(whereParts, expressionUsesTime);
|
||||
return SqlExpression.and(
|
||||
SqlExpression.or(
|
||||
SqlExpression.and(...timeExpressions),
|
||||
...compares.map(compare =>
|
||||
SqlExpression.and(
|
||||
...timeExpressions.map(timeExpression => shiftTimeInExpression(timeExpression, compare)),
|
||||
),
|
||||
),
|
||||
),
|
||||
...timelessExpressions,
|
||||
);
|
||||
}
|
||||
|
||||
function expressionUsesTime(expression: SqlExpression): boolean {
|
||||
return shiftTimeInExpression(expression, 'P1D') !== expression;
|
||||
}
|
||||
|
||||
export function shiftTimeInExpression(expression: SqlExpression, compare: string): SqlExpression {
|
||||
return expression.walk(ex => {
|
||||
if (ex instanceof SqlLiteral) {
|
||||
// Works with: __time < TIMESTAMP '2022-01-02 03:04:05'
|
||||
if (ex.isDate()) {
|
||||
return F('TIME_SHIFT', ex, period, -1);
|
||||
return F.timeShift(ex, compare, -1);
|
||||
}
|
||||
} else if (ex instanceof SqlFunction) {
|
||||
const effectiveFunctionName = ex.getEffectiveFunctionName();
|
||||
|
||||
// Works with: TIME_IN_INTERVAL(__time, '<interval>')
|
||||
if (effectiveFunctionName === 'TIME_IN_INTERVAL') {
|
||||
return ex.changeArgs(ex.args!.change(0, F('TIME_SHIFT', ex.getArg(0), period, 1)));
|
||||
// Ideally we could rewrite it to TIME_IN_INTERVAL(TIME_SHIFT(__time, period, 1), '<interval>') but that would be slow in the current Druid
|
||||
// return ex.changeArgs(ex.args!.change(0, F('TIME_SHIFT', ex.getArg(0), period, 1)));a
|
||||
|
||||
const interval = ex.getArgAsString(1);
|
||||
if (!interval) return ex;
|
||||
|
||||
const [start, end] = interval.split('/');
|
||||
if (!IS_DATE_LIKE.test(start) || !IS_DATE_LIKE.test(end)) return ex;
|
||||
|
||||
const t = ex.getArg(0);
|
||||
if (!t) return ex;
|
||||
|
||||
return F.timeShift(isoStringToTimestampLiteral(start), compare, -1)
|
||||
.lessThanOrEqual(t)
|
||||
.and(t.lessThan(F.timeShift(isoStringToTimestampLiteral(end), compare, -1)));
|
||||
}
|
||||
|
||||
// Works with: TIME_SHIFT(...) <= __time
|
||||
// and: __time < MAX_DATA_TIME()
|
||||
if (effectiveFunctionName === 'TIME_SHIFT' || effectiveFunctionName === 'MAX_DATA_TIME') {
|
||||
return F('TIME_SHIFT', ex, period, -1);
|
||||
return F.timeShift(ex, compare, -1);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ export function toggle<T>(xs: readonly T[], x: T, eq?: (a: T, b: T) => boolean):
|
|||
}
|
||||
|
||||
export function getInitQuery(table: SqlExpression, where: SqlExpression): SqlQuery {
|
||||
return SqlQuery.from(table).applyIf(String(where) !== 'TRUE', q =>
|
||||
return SqlQuery.from(table.as('t')).applyIf(String(where) !== 'TRUE', q =>
|
||||
q.changeWhereExpression(where),
|
||||
);
|
||||
}
|
||||
|
|
|
@ -16,7 +16,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import { Button, Icon, Intent } from '@blueprintjs/core';
|
||||
import { Button, Icon, Intent, Tag } from '@blueprintjs/core';
|
||||
import { IconNames } from '@blueprintjs/icons';
|
||||
import React from 'react';
|
||||
import type { Filter } from 'react-table';
|
||||
|
@ -295,8 +295,16 @@ export class LookupsView extends React.PureComponent<LookupsViewProps, LookupsVi
|
|||
);
|
||||
}}
|
||||
confirmButtonText="Delete lookup"
|
||||
successText="Lookup was deleted"
|
||||
failText="Could not delete lookup"
|
||||
successText={
|
||||
<>
|
||||
Lookup <Tag minimal>{deleteLookupName}</Tag> was deleted
|
||||
</>
|
||||
}
|
||||
failText={
|
||||
<>
|
||||
Could not delete lookup <Tag minimal>{deleteLookupName}</Tag>
|
||||
</>
|
||||
}
|
||||
intent={Intent.DANGER}
|
||||
onClose={() => {
|
||||
this.setState({ deleteLookupTier: undefined, deleteLookupName: undefined });
|
||||
|
|
|
@ -16,7 +16,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import { Button, ButtonGroup, Intent, Label, MenuItem } from '@blueprintjs/core';
|
||||
import { Button, ButtonGroup, Intent, Label, MenuItem, Tag } from '@blueprintjs/core';
|
||||
import { IconNames } from '@blueprintjs/icons';
|
||||
import { sum } from 'd3-array';
|
||||
import React from 'react';
|
||||
|
@ -699,8 +699,16 @@ ORDER BY
|
|||
return resp.data;
|
||||
}}
|
||||
confirmButtonText="Disable worker"
|
||||
successText="Worker has been disabled"
|
||||
failText="Could not disable worker"
|
||||
successText={
|
||||
<>
|
||||
Worker <Tag minimal>{middleManagerDisableWorkerHost}</Tag> has been disabled
|
||||
</>
|
||||
}
|
||||
failText={
|
||||
<>
|
||||
Could not disable worker <Tag minimal>{middleManagerDisableWorkerHost}</Tag>
|
||||
</>
|
||||
}
|
||||
intent={Intent.DANGER}
|
||||
onClose={() => {
|
||||
this.setState({ middleManagerDisableWorkerHost: undefined });
|
||||
|
|
|
@ -103,7 +103,7 @@ export const PreviewTable = React.memo(function PreviewTable(props: PreviewTable
|
|||
);
|
||||
}
|
||||
|
||||
const numericColumnBraces = getNumericColumnBraces(queryResult);
|
||||
const numericColumnBraces = getNumericColumnBraces(queryResult, undefined, undefined);
|
||||
return (
|
||||
<div className="preview-table">
|
||||
<ReactTable
|
||||
|
|
|
@ -515,7 +515,7 @@ export const SchemaStep = function SchemaStep(props: SchemaStepProps) {
|
|||
throw new DruidError(e);
|
||||
}
|
||||
|
||||
return result.attachQuery({}, SqlQuery.maybeParse(previewQueryString));
|
||||
return result.attachQuery({} as any, SqlQuery.maybeParse(previewQueryString));
|
||||
}
|
||||
},
|
||||
backgroundStatusCheck: executionBackgroundResultStatusCheck,
|
||||
|
|
|
@ -190,7 +190,7 @@ export class FlexibleQueryInput extends React.PureComponent<
|
|||
const found = dedupe(findAllSqlQueriesInText(queryString), ({ startRowColumn }) =>
|
||||
String(startRowColumn.row),
|
||||
);
|
||||
if (found.length <= 1) return []; // Do not highlight a single query or no queries
|
||||
if (!found.length) return [];
|
||||
|
||||
// Do not report the first query if it is basically the main query minus whitespace
|
||||
const firstQuery = found[0].sql;
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue