Merge remote-tracking branch 'apache/master' into quidem-runner-extension-submit

This commit is contained in:
Zoltan Haindrich 2024-05-14 07:36:48 +00:00
commit 9578953678
102 changed files with 4546 additions and 1265 deletions

1
.gitignore vendored
View File

@ -1,4 +1,5 @@
dist
longCompress
target
*.iml
*.ipr

View File

@ -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);
}
}

View File

@ -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.

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -1,7 +1,7 @@
---
id: tutorial-jdbc
title: Use the JDBC driver to query Druid
sidebar_label: JDBC connector
sidebar_label: JDBC connector tutorial
---
<!--

View File

@ -1,7 +1,7 @@
---
id: tutorial-sketches-theta
title: Approximations with Theta sketches
sidebar_label: Theta sketches
sidebar_label: Theta sketches tutorial
---
<!--

View File

@ -128,7 +128,6 @@
<dependency>
<groupId>org.apache.curator</groupId>
<artifactId>curator-client</artifactId>
<version>5.5.0</version>
<scope>provided</scope>
</dependency>
<dependency>

View File

@ -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());

View File

@ -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

View File

@ -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);
}
}

View File

@ -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

View File

@ -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();
}
}

View File

@ -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);

View File

@ -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);
}

View File

@ -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);

View File

@ -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) -> {

View File

@ -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) -> {

View File

@ -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;

View File

@ -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()
);
});
}
}

View File

@ -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();
}
};
}
}

View File

@ -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"))
);

View File

@ -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
{

View File

@ -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());
}
}
}

View File

@ -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());
}
}

View File

@ -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 +
'}';
}
}

View File

@ -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;
}
}

View File

@ -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;

View File

@ -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());

View File

@ -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

View File

@ -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()

View File

@ -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

View File

@ -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

View File

@ -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());

View File

@ -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

View File

@ -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>

View File

@ -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>

View File

@ -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
---

View File

@ -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 -->

View File

@ -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

View File

@ -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();
}

View File

@ -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;
}
}

View File

@ -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

View File

@ -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}.
*/

View File

@ -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) +
'}';
}
}

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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

View File

@ -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
{

View File

@ -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

View File

@ -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);
}
}
}

View File

@ -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 +
'}';
}
}

View File

@ -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.

View File

@ -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());
}
}
}

View File

@ -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 +
'}';
}
}

View File

@ -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;

View File

@ -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)
{

View File

@ -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;
}
}

View File

@ -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)
);
}

View File

@ -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())
);

View File

@ -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;

View File

@ -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());
}
}

View File

@ -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();

View File

@ -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());
}
}

View File

@ -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;

View File

@ -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()
{

View File

@ -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

View File

@ -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)

View File

@ -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);
}
}

View File

@ -109,7 +109,6 @@ public interface OverlordClient
dataSource,
interval,
versions,
false,
null,
maxSegmentsToKill,
maxUsedStatusLastUpdatedTime

View File

@ -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()
{

View File

@ -436,7 +436,6 @@ public class OverlordClientImplTest
null,
null,
null,
null,
null
);

View File

@ -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);
}

View File

@ -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();
}

View File

@ -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();
}
}

View File

@ -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(

View File

@ -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(

View File

@ -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();
}
}

View File

@ -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);
}
}

View File

@ -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"
}

View File

@ -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",

View File

@ -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 ? (

View File

@ -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;
}

View File

@ -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}

View File

@ -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,
),
});
}

View File

@ -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);

View File

@ -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[];
}

View File

@ -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;

View File

@ -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 });

View File

@ -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,
)}
/>
)}

View File

@ -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 => {

View File

@ -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'`,
),

View File

@ -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);
}
}

View File

@ -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),
);
}

View File

@ -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 });

View File

@ -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 });

View File

@ -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

View File

@ -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,

View File

@ -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