diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java index f212380eedad..43c5dc82a90e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java @@ -37,6 +37,8 @@ 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.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 @@ enum KeyGenerator */ 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 String generateKey(int rowNumber, int keyLength) */ 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 int getChannelNumber(int rowNumber, int numRows, int numChannels) 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 sortKey = ImmutableList.of(new KeyColumn(KEY, KeyOrder.ASCENDING)); private List> channelFrames; @@ -200,6 +217,14 @@ public int getChannelNumber(int rowNumber, int numRows, int numChannels) @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 void setupTrial() ChannelDistribution.valueOf(StringUtils.toUpperCase(channelDistributionString)); // Create channelRows which holds rows for each channel. - final List>> channelRows = new ArrayList<>(); + final List>> 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 void setupTrial() // Populate "channelRows". for (int rowNumber = 0; rowNumber < numRows; rowNumber++) { - final String keyString = keyGenerator.generateKey(rowNumber, keyLength); - final NonnullPair row = new NonnullPair<>(keyString, valueString); + final Comparable keyObject = keyGenerator.generateKey(rowNumber, keyLength, columnType); + final NonnullPair row = new NonnullPair<>(keyObject, valueString); channelRows.get(channelDistribution.getChannelNumber(rowNumber, numRows, numChannels)).add(row); } // Sort each "channelRows". - for (List> rows : channelRows) { + for (List> rows : channelRows) { rows.sort(Comparator.comparing(row -> row.lhs)); } // Populate each "channelFrames". for (int channelNumber = 0; channelNumber < numChannels; channelNumber++) { - final List> rows = channelRows.get(channelNumber); - final RowBasedSegment> segment = + final List> rows = channelRows.get(channelNumber); + final RowBasedSegment> segment = new RowBasedSegment<>( SegmentId.dummy("__dummy"), Sequences.simple(rows), @@ -350,4 +376,14 @@ public void mergeChannels(Blackhole blackhole) 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); + } } diff --git a/codestyle/druid-forbidden-apis.txt b/codestyle/druid-forbidden-apis.txt index b35bdede7749..c53007f76e37 100644 --- a/codestyle/druid-forbidden-apis.txt +++ b/codestyle/druid-forbidden-apis.txt @@ -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. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java index 6d3939497cfe..f51cdccd9ede 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImpl.java @@ -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 @@ private ClusterByStatisticsCollectorImpl( 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 static ClusterByStatisticsCollector create( ) { 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, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollectorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollectorFactory.java index 432fd76e8587..d69c24bec82f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollectorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/DistinctKeyCollectorFactory.java @@ -23,6 +23,7 @@ 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 @@ private DistinctKeyCollectorFactory(Comparator comparator) 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 diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectors.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectors.java index 17490d6f1481..a559e57d5472 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectors.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/KeyCollectors.java @@ -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 @@ private 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); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java index 4f6bd6dbcd13..b254255c855c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorFactory.java @@ -27,6 +27,7 @@ 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 @@ private QuantilesSketchKeyCollectorFactory(final Comparator comparator) 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 diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java new file mode 100644 index 000000000000..efddab577e77 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQComplexGroupByTest.java @@ -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 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 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\"}]'\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 context) + { + final Map 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\"}]'\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 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\"}]'\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(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImplTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImplTest.java index 77502dddd38c..b2fbf500ce4e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImplTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/ClusterByStatisticsCollectorImplTest.java @@ -108,7 +108,7 @@ public void test_clusterByX_unique() .iterator(); final NavigableMap> sortedKeyWeights = - computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE)); doTest( clusterBy, @@ -157,7 +157,7 @@ public void test_clusterByX_everyKeyAppearsTwice() } final NavigableMap> sortedKeyWeights = - computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE)); doTest( clusterBy, @@ -208,7 +208,7 @@ public void test_clusterByX_everyKeyAppearsTwice_withAggregation() } final NavigableMap> sortedKeyWeights = - computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE)); doTest( clusterBy, @@ -267,7 +267,7 @@ public void test_clusterByXYbucketByX_threeX_uniqueY() } final NavigableMap> sortedKeyWeights = - computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE)); doTest( clusterBy, @@ -331,7 +331,7 @@ public void test_clusterByXYbucketByX_maxX_uniqueY() } final NavigableMap> sortedKeyWeights = - computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE)); doTest( clusterBy, @@ -402,7 +402,7 @@ public void test_clusterByXYbucketByX_maxX_lowCardinalityY_withAggregation() } final NavigableMap> sortedKeyWeights = - computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator()); + computeSortedKeyWeightsFromUnweightedKeys(keys, clusterBy.keyComparator(SIGNATURE)); doTest( clusterBy, @@ -551,7 +551,7 @@ private void doTest( final BiConsumer testFn ) { - final Comparator comparator = clusterBy.keyComparator(); + final Comparator comparator = clusterBy.keyComparator(SIGNATURE); // Load into single collector, sorted order. final ClusterByStatisticsCollectorImpl sortedCollector = makeCollector(clusterBy, aggregate); @@ -649,7 +649,7 @@ private static void verifyPartitions( testName, partitions, sortedKeyWeights.firstKey(), - clusterBy.keyComparator() + clusterBy.keyComparator(SIGNATURE) ); verifyPartitionWeights(testName, clusterBy, partitions, sortedKeyWeights, aggregate, expectedPartitionSize); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorTest.java index c5efeaa039b2..cbdc58ae4acf 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DelegateOrMinKeyCollectorTest.java @@ -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 comparator = clusterBy.keyComparator(); + private final Comparator comparator = clusterBy.keyComparator(signature); static { NullHandling.initializeForTests(); @@ -53,7 +53,7 @@ public void testEmpty() final DelegateOrMinKeyCollector collector = new DelegateOrMinKeyCollectorFactory<>( comparator, - QuantilesSketchKeyCollectorFactory.create(clusterBy) + QuantilesSketchKeyCollectorFactory.create(clusterBy, signature) ).newKeyCollector(); Assert.assertTrue(collector.getDelegate().isPresent()); @@ -69,8 +69,8 @@ public void testDelegateAndMinKeyNotNullThrowsException() { 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 void testAdd() final DelegateOrMinKeyCollector collector = new DelegateOrMinKeyCollectorFactory<>( comparator, - QuantilesSketchKeyCollectorFactory.create(clusterBy) + QuantilesSketchKeyCollectorFactory.create(clusterBy, signature) ).newKeyCollector(); RowKey key = createKey(1L); @@ -100,7 +100,7 @@ public void testDownSampleSingleKey() final DelegateOrMinKeyCollector collector = new DelegateOrMinKeyCollectorFactory<>( comparator, - QuantilesSketchKeyCollectorFactory.create(clusterBy) + QuantilesSketchKeyCollectorFactory.create(clusterBy, signature) ).newKeyCollector(); RowKey key = createKey(1L); @@ -128,7 +128,7 @@ public void testDownSampleTwoKeys() final DelegateOrMinKeyCollector collector = new DelegateOrMinKeyCollectorFactory<>( comparator, - QuantilesSketchKeyCollectorFactory.create(clusterBy) + QuantilesSketchKeyCollectorFactory.create(clusterBy, signature) ).newKeyCollector(); RowKey key = createKey(1L); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DistinctKeyCollectorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DistinctKeyCollectorTest.java index 9ac750958c35..697005934e5f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DistinctKeyCollectorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/DistinctKeyCollectorTest.java @@ -28,6 +28,8 @@ 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 @@ public class DistinctKeyCollectorTest { private final ClusterBy clusterBy = new ClusterBy(ImmutableList.of(new KeyColumn("x", KeyOrder.ASCENDING)), 0); - private final Comparator comparator = clusterBy.keyComparator(); + private final RowSignature signature = RowSignature.builder().add("x", ColumnType.LONG).build(); + private final Comparator 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 void test_sequentialKeys_unweighted() ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); KeyCollectorTestUtils.doTest( - DistinctKeyCollectorFactory.create(clusterBy), + DistinctKeyCollectorFactory.create(clusterBy, signature), keyWeights, comparator, (testName, collector) -> { @@ -95,7 +98,7 @@ public void test_uniformRandomKeys_unweighted() ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); KeyCollectorTestUtils.doTest( - DistinctKeyCollectorFactory.create(clusterBy), + DistinctKeyCollectorFactory.create(clusterBy, signature), keyWeights, comparator, (testName, collector) -> { @@ -113,7 +116,7 @@ public void test_uniformRandomKeys_unweighted() @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 void test_uniformRandomKeys_unweighted_downSampledToOneKey() ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator).firstKey(); KeyCollectorTestUtils.doTest( - DistinctKeyCollectorFactory.create(clusterBy), + DistinctKeyCollectorFactory.create(clusterBy, signature), keyWeights, comparator, (testName, collector) -> { @@ -161,7 +164,7 @@ public void test_nonUniformRandomKeys_unweighted() ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); KeyCollectorTestUtils.doTest( - DistinctKeyCollectorFactory.create(clusterBy), + DistinctKeyCollectorFactory.create(clusterBy, signature), keyWeights, comparator, (testName, collector) -> { @@ -184,7 +187,7 @@ public void test_uniformRandomKeys_barbellWeighted() ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); KeyCollectorTestUtils.doTest( - DistinctKeyCollectorFactory.create(clusterBy), + DistinctKeyCollectorFactory.create(clusterBy, signature), keyWeights, comparator, (testName, collector) -> { @@ -211,7 +214,7 @@ public void test_uniformRandomKeys_inverseBarbellWeighted() ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); KeyCollectorTestUtils.doTest( - DistinctKeyCollectorFactory.create(clusterBy), + DistinctKeyCollectorFactory.create(clusterBy, signature), keyWeights, comparator, (testName, collector) -> { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorTest.java index 64288e1b2fc0..ebc677ad1144 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollectorTest.java @@ -43,7 +43,8 @@ public class QuantilesSketchKeyCollectorTest { private final ClusterBy clusterBy = new ClusterBy(ImmutableList.of(new KeyColumn("x", KeyOrder.ASCENDING)), 0); - private final Comparator comparator = clusterBy.keyComparator(); + private final RowSignature signature = RowSignature.builder().add("x", ColumnType.LONG).build(); + private final Comparator 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 void test_sequentialKeys_unweighted() ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); KeyCollectorTestUtils.doTest( - QuantilesSketchKeyCollectorFactory.create(clusterBy), + QuantilesSketchKeyCollectorFactory.create(clusterBy, signature), keyWeights, comparator, (testName, collector) -> { @@ -96,7 +97,7 @@ public void test_uniformRandomKeys_unweighted() ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); KeyCollectorTestUtils.doTest( - QuantilesSketchKeyCollectorFactory.create(clusterBy), + QuantilesSketchKeyCollectorFactory.create(clusterBy, signature), keyWeights, comparator, (testName, collector) -> { @@ -114,7 +115,7 @@ public void test_uniformRandomKeys_unweighted_downSampledToSmallestSize() ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator).firstKey(); KeyCollectorTestUtils.doTest( - QuantilesSketchKeyCollectorFactory.create(clusterBy), + QuantilesSketchKeyCollectorFactory.create(clusterBy, signature), keyWeights, comparator, (testName, collector) -> { @@ -147,7 +148,7 @@ public void test_uniformRandomKeys_barbellWeighted() ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); KeyCollectorTestUtils.doTest( - QuantilesSketchKeyCollectorFactory.create(clusterBy), + QuantilesSketchKeyCollectorFactory.create(clusterBy, signature), keyWeights, comparator, (testName, collector) -> { @@ -169,8 +170,8 @@ public void test_uniformRandomKeys_barbellWeighted() 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 void test_uniformRandomKeys_inverseBarbellWeighted() ClusterByStatisticsCollectorImplTest.computeSortedKeyWeightsFromWeightedKeys(keyWeights, comparator); KeyCollectorTestUtils.doTest( - QuantilesSketchKeyCollectorFactory.create(clusterBy), + QuantilesSketchKeyCollectorFactory.create(clusterBy, signature), keyWeights, comparator, (testName, collector) -> { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index fc7cfe5d9bea..6774848771a6 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -76,6 +76,7 @@ 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; @@ -290,6 +291,101 @@ private static Supplier> getSupplierForSegment(Function< .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; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteNestedDataQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteNestedDataQueryMSQTest.java new file mode 100644 index 000000000000..d988ec890d3b --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteNestedDataQueryMSQTest.java @@ -0,0 +1,98 @@ +/* + * 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.FileUtils; +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.run.SqlEngine; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +/** + * Runs {@link CalciteNestedDataQueryTest} but with MSQ engine + */ +public class CalciteNestedDataQueryMSQTest extends CalciteNestedDataQueryTest +{ + + @Override + public void configureGuice(DruidInjectorBuilder builder) + { + super.configureGuice(builder); + builder.addModules( + CalciteMSQTestsHelper.fetchModules(FileUtils::createTempDir, 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() + ); + }); + } + +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/ExtractResultsFactory.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/ExtractResultsFactory.java index 7b694e1384be..5d3c1d996ac8 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/ExtractResultsFactory.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/ExtractResultsFactory.java @@ -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 QueryTestRunner.QueryRunStep make(QueryTestBuilder builder, QueryTestRunn return new QueryTestRunner.BaseExecuteQuery(builder) { final List extractedResults = new ArrayList<>(); + final RowSignature resultsSignature = null; final MSQTestOverlordServiceClient overlordClient = overlordClientSupplier.get(); @@ -99,7 +102,10 @@ public void run() 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 List results() { return extractedResults; } + + private RowSignature convertColumnAndTypeToRowSignature(final List columnAndTypes) + { + final RowSignature.Builder builder = RowSignature.builder(); + for (MSQResultsReport.ColumnAndType columnAndType : columnAndTypes) { + builder.add(columnAndType.getName(), columnAndType.getType()); + } + return builder.build(); + } }; } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index a3b6fa3d458c..b32b8fd7777c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -1199,7 +1199,7 @@ public void verifyResults() // assert rollup Assert.assertEquals(expectedRollUp, queryableIndex.getMetadata().isRollup()); - // asset query granularity + // assert query granularity Assert.assertEquals(expectedQueryGranularity, queryableIndex.getMetadata().getQueryGranularity()); // assert aggregator factories @@ -1231,7 +1231,7 @@ public void verifyResults() .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")) ); diff --git a/processing/src/main/java/org/apache/druid/frame/field/ComplexFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/ComplexFieldReader.java index 045835cbb818..c0a0c872faec 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/ComplexFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/ComplexFieldReader.java @@ -89,10 +89,57 @@ public boolean isNull(Memory memory, long position) 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 + ) + { + 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 + ) { - return false; + 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 @@ private Selector(Memory memory, ReadableFieldPointer fieldPointer, ComplexMetric @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 diff --git a/processing/src/main/java/org/apache/druid/frame/field/FieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/FieldReader.java index b0dfdfa4995a..bc9d631361f3 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/FieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/FieldReader.java @@ -56,9 +56,4 @@ DimensionSelector makeDimensionSelector( * 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(); } diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldReader.java index 7eab229e6be6..8d6f3958b1e4 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldReader.java @@ -50,10 +50,4 @@ public boolean isNull(Memory memory, long position) final byte firstByte = memory.getByte(position); return firstByte == NumericArrayFieldWriter.NULL_ROW; } - - @Override - public boolean isComparable() - { - return true; - } } diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java index cf83c405d092..1e11cfa65f37 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java @@ -76,13 +76,6 @@ public boolean isNull(Memory memory, long position) 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 diff --git a/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java index c0085b46f743..2513a2d24444 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java @@ -123,12 +123,6 @@ public boolean isNull(Memory memory, long position) } } - @Override - public boolean isComparable() - { - return true; - } - /** * Selector that reads a value from a location pointed to by {@link ReadableFieldPointer}. */ diff --git a/processing/src/main/java/org/apache/druid/frame/key/ByteRowKeyComparator.java b/processing/src/main/java/org/apache/druid/frame/key/ByteRowKeyComparator.java index e7b7e2871d03..35695c080132 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/ByteRowKeyComparator.java +++ b/processing/src/main/java/org/apache/druid/frame/key/ByteRowKeyComparator.java @@ -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 @@ */ public class ByteRowKeyComparator implements Comparator { + /** + * Key columns to compare on + */ + private final List 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 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 keyColumns) + public static ByteRowKeyComparator create(final List 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 static int computeFirstFieldPosition(final int fieldCount) 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 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]; + + if (runLengthEntry.getRunLength() <= 0) { + // Defensive check + continue; + } - int cmp = FrameReaderUtils.compareByteArraysUnsigned( + // 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, - comparableBytesStartPosition1, - comparableBytesEndPosition1 - comparableBytesStartPosition1, + currentRunStartPosition1, keyArray2, - comparableBytesStartPosition2, - comparableBytesEndPosition2 - comparableBytesStartPosition2 + 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 ascending ? cmp : -cmp; - } - - field = nextField; - comparableBytesStartPosition1 = comparableBytesEndPosition1; - comparableBytesStartPosition2 = comparableBytesEndPosition2; + if (cmp != 0) { + return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp; } - ascending = !ascending; + fieldsComparedTillNow = nextField; + currentRunStartPosition1 = currentRunEndPosition1; + currentRunStartPosition2 = currentRunEndPosition2; } - return 0; } @@ -159,14 +216,18 @@ public boolean equals(Object o) } 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 int hashCode() 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) + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java b/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java index 3be8cf22cb40..305b00dc6df3 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java +++ b/processing/src/main/java/org/apache/druid/frame/key/ClusterBy.java @@ -152,29 +152,29 @@ public RowKeyReader keyReader(final ColumnInspector inspector) /** * Comparator that compares keys for this instance using the given signature. */ - public Comparator keyComparator() + public Comparator 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 byteKeyComparator() + public Comparator 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 bucketComparator() + public Comparator 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); } } diff --git a/processing/src/main/java/org/apache/druid/frame/key/FrameComparisonWidgetImpl.java b/processing/src/main/java/org/apache/druid/frame/key/FrameComparisonWidgetImpl.java index d7008fcab450..a243e53dedcd 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/FrameComparisonWidgetImpl.java +++ b/processing/src/main/java/org/apache/druid/frame/key/FrameComparisonWidgetImpl.java @@ -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.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 @@ * 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 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 @@ private FrameComparisonWidgetImpl( final Memory dataRegion, final List keyFieldReaders, final int firstFieldPosition, - final int[] ascDescRunLengths + final RowKeyComparisonRunLengths rowKeyComparisonRunLengths, + final ComplexMetricSerde[] complexMetricSerdes, + final ColumnType[] columnTypes ) { this.frame = frame; @@ -68,7 +117,9 @@ private FrameComparisonWidgetImpl( 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 static FrameComparisonWidgetImpl create( 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 static FrameComparisonWidgetImpl create( 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 RowKey readKey(int row) 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 int compare(int row, RowKey key) final long rowPosition = getRowPositionInDataRegion(row); long comparableBytesStartPositionInRow = firstFieldPosition; - int keyComparableBytesStartPosition = Integer.BYTES * keyFieldCount; + int comparableBytesStartPositionInKey = Integer.BYTES * keyFieldCount; + + // Number of fields compared till now, which is equivalent to the index of the field to compare next + int fieldsComparedTillNow = 0; - boolean ascending = true; - int field = 0; + for (int i = 0; i < rowKeyComparisonRunLengths.getRunLengthEntries().length; ++i) { - 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); + final RunLengthEntry runLengthEntry = rowKeyComparisonRunLengths.getRunLengthEntries()[i]; + + if (runLengthEntry.getRunLength() <= 0) { + // Defensive check + continue; + } - final long comparableBytesLength = comparableBytesEndPositionInRow - comparableBytesStartPositionInRow; - final int keyComparableBytesLength = keyComparableBytesEndPosition - keyComparableBytesStartPosition; + // 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); - int cmp = FrameReaderUtils.compareMemoryToByteArrayUnsigned( + 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, - comparableBytesLength, keyArray, - keyComparableBytesStartPosition, - keyComparableBytesLength + comparableBytesStartPositionInKey, + columnTypes[i], + complexMetricSerdes[i] + ); + } else { + cmp = FrameReaderUtils.compareMemoryToByteArrayUnsigned( + dataRegion, + rowPosition + comparableBytesStartPositionInRow, + comparableBytesEndPositionInRow - comparableBytesStartPositionInRow, + keyArray, + comparableBytesStartPositionInKey, + comparableBytesEndPositionInKey - comparableBytesStartPositionInKey ); - - if (cmp != 0) { - return ascending ? cmp : -cmp; - } - - field += numFields; - comparableBytesStartPositionInRow += comparableBytesLength; - keyComparableBytesStartPosition += keyComparableBytesLength; } - ascending = !ascending; + if (cmp != 0) { + return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp; + } + fieldsComparedTillNow = nextField; + comparableBytesStartPositionInRow = comparableBytesEndPositionInRow; + comparableBytesStartPositionInKey = comparableBytesEndPositionInKey; } - return 0; } @@ -230,39 +337,52 @@ public int compare(final int row, final FrameComparisonWidget otherWidget, final 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); + for (int i = 0; i < rowKeyComparisonRunLengths.getRunLengthEntries().length; ++i) { + final RunLengthEntry runLengthEntry = rowKeyComparisonRunLengths.getRunLengthEntries()[i]; + if (runLengthEntry.getRunLength() <= 0) { + // Defensive check + continue; + } + + final int nextField = fieldsComparedTillNow + runLengthEntry.getRunLength(); + 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; + final int cmp; - int cmp = FrameReaderUtils.compareMemoryUnsigned( + 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, - comparableBytesLength, + otherWidgetImpl.dataRegion, + otherRowPosition + otherComparableBytesStartPositionInRow, + columnTypes[i], + complexMetricSerdes[i] + ); + } else { + cmp = FrameReaderUtils.compareMemoryUnsigned( + dataRegion, + rowPosition + comparableBytesStartPositionInRow, + comparableBytesEndPositionInRow - comparableBytesStartPositionInRow, otherWidgetImpl.getDataRegion(), otherRowPosition + otherComparableBytesStartPositionInRow, - otherComparableBytesLength + otherComparableBytesEndPositionInRow - otherComparableBytesStartPositionInRow ); + } - if (cmp != 0) { - return ascending ? cmp : -cmp; - } - - field += numFields; - comparableBytesStartPositionInRow += comparableBytesLength; - otherComparableBytesStartPositionInRow += otherComparableBytesLength; + if (cmp != 0) { + return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp; } - ascending = !ascending; + fieldsComparedTillNow = nextField; + comparableBytesStartPositionInRow = comparableBytesEndPositionInRow; + otherComparableBytesStartPositionInRow = otherComparableBytesEndPositionInRow; } return 0; diff --git a/processing/src/main/java/org/apache/druid/frame/key/KeyColumn.java b/processing/src/main/java/org/apache/druid/frame/key/KeyColumn.java index 781161647733..cf73c0ecddb0 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/KeyColumn.java +++ b/processing/src/main/java/org/apache/druid/frame/key/KeyColumn.java @@ -69,8 +69,8 @@ public boolean equals(Object o) 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 diff --git a/processing/src/main/java/org/apache/druid/frame/key/RowKey.java b/processing/src/main/java/org/apache/druid/frame/key/RowKey.java index 1c63afd201e5..985fc42f6312 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/RowKey.java +++ b/processing/src/main/java/org/apache/druid/frame/key/RowKey.java @@ -27,6 +27,16 @@ /** * 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 { diff --git a/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparator.java b/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparator.java index 3e7e6faed57e..a78b3e9aa559 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparator.java +++ b/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparator.java @@ -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 @@ private RowKeyComparator(final ByteRowKeyComparator byteRowKeyComparatorDelegate this.byteRowKeyComparatorDelegate = byteRowKeyComparatorDelegate; } - public static RowKeyComparator create(final List keyColumns) + public static RowKeyComparator create(final List keyColumns, RowSignature rowSignature) { - return new RowKeyComparator(ByteRowKeyComparator.create(keyColumns)); + return new RowKeyComparator(ByteRowKeyComparator.create(keyColumns, rowSignature)); } @Override diff --git a/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparisonRunLengths.java b/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparisonRunLengths.java new file mode 100644 index 000000000000..cc05dea993e5 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparisonRunLengths.java @@ -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 keyColumns, RowSignature rowSignature) + { + final List 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); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/key/RunLengthEntry.java b/processing/src/main/java/org/apache/druid/frame/key/RunLengthEntry.java new file mode 100644 index 000000000000..c99f80de0613 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/key/RunLengthEntry.java @@ -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 + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/read/FrameReaderUtils.java b/processing/src/main/java/org/apache/druid/frame/read/FrameReaderUtils.java index 30a2fb3b1c9c..15ea29bcdd60 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/FrameReaderUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/read/FrameReaderUtils.java @@ -22,12 +22,15 @@ 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 static int compareByteArraysUnsigned( 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. diff --git a/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java b/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java index 0bb78b2109b7..0b497b545ac6 100644 --- a/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java @@ -22,7 +22,6 @@ 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 static void verifySortColumns( 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()); } } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/TypeStrategy.java b/processing/src/main/java/org/apache/druid/segment/column/TypeStrategy.java index 3d2493bc80c5..81aa03778362 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/TypeStrategy.java +++ b/processing/src/main/java/org/apache/druid/segment/column/TypeStrategy.java @@ -179,20 +179,38 @@ default T fromBytes(byte[] value) * 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 + *

+ * If a type is groupable, following statements MUST hold: + *

+ * 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. + *

+ * 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. + *

+ * 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) { diff --git a/processing/src/test/java/org/apache/druid/frame/key/ByteRowKeyComparatorTest.java b/processing/src/test/java/org/apache/druid/frame/key/ByteRowKeyComparatorTest.java index a78ed0c00a70..ef764f6123ba 100644 --- a/processing/src/test/java/org/apache/druid/frame/key/ByteRowKeyComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/frame/key/ByteRowKeyComparatorTest.java @@ -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 @@ 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 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 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 keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.DESCENDING), @@ -70,13 +219,13 @@ public void test_compare_AAAA() // AAAA = all ascending 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 keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.ASCENDING), @@ -85,13 +234,13 @@ public void test_compare_DDDD() // DDDD = all descending 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 keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.ASCENDING), @@ -100,13 +249,13 @@ public void test_compare_DAAD() 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 keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.DESCENDING), @@ -115,13 +264,13 @@ public void test_compare_ADDA() 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 keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.DESCENDING), @@ -130,8 +279,103 @@ public void test_compare_DADA() 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 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 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 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 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 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 void test_equals() .verify(); } - private List sortUsingByteKeyComparator(final List keyColumns, final List objectss) + private static List sortUsingByteKeyComparator( + final List keyColumns, + final List 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 sortUsingObjectComparator(final List keyColumns, final List objectss) + private static List sortUsingObjectComparator( + final List keyColumns, + final List objectss, + final RowSignature rowSignature + ) { final List sortedObjectssCopy = objectss.stream().sorted( (o1, o2) -> { @@ -174,9 +426,20 @@ private List sortUsingObjectComparator(final List keyColumns, final List 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; + } } diff --git a/processing/src/test/java/org/apache/druid/frame/key/ClusterByTest.java b/processing/src/test/java/org/apache/druid/frame/key/ClusterByTest.java index bfa16a221d65..a0399bf39761 100644 --- a/processing/src/test/java/org/apache/druid/frame/key/ClusterByTest.java +++ b/processing/src/test/java/org/apache/druid/frame/key/ClusterByTest.java @@ -22,6 +22,8 @@ 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 void test_keyComparator() 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 void test_bucketComparator_noBucketKey() 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) ); } diff --git a/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java b/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java index 64556c6775b3..c53ca04a9096 100644 --- a/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java +++ b/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java @@ -44,27 +44,49 @@ 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() @@ -72,7 +94,7 @@ public void setUp() } @Test - public void test_isPartiallyNullKey_someColumns() + public void test_noComplexColumns_isPartiallyNullKey_someColumns() { final List keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.ASCENDING), @@ -80,11 +102,17 @@ public void test_isPartiallyNullKey_someColumns() new KeyColumn("3", KeyOrder.ASCENDING) ); - final FrameComparisonWidget widget = createComparisonWidget(keyColumns); + final FrameComparisonWidget widget = createComparisonWidget( + frameWithoutComplexColumns, + keyColumns, + ByteRowKeyComparatorTest.NO_COMPLEX_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)).limit(3).allMatch(Objects::nonNull); + 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])); @@ -95,7 +123,7 @@ public void test_isPartiallyNullKey_someColumns() } @Test - public void test_isPartiallyNullKey_allColumns() + public void test_noComplexColumns_isPartiallyNullKey_allColumns() { final List keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.ASCENDING), @@ -104,17 +132,21 @@ public void test_isPartiallyNullKey_allColumns() new KeyColumn("4", KeyOrder.ASCENDING) ); - final FrameComparisonWidget widget = createComparisonWidget(keyColumns); + final FrameComparisonWidget widget = createComparisonWidget( + frameWithoutComplexColumns, + keyColumns, + ByteRowKeyComparatorTest.NO_COMPLEX_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); + 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_readKey_someColumns() + public void test_noComplexColumns_readKey_someColumns() { final List keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.ASCENDING), @@ -122,18 +154,28 @@ public void test_readKey_someColumns() new KeyColumn("3", KeyOrder.ASCENDING) ); - final FrameComparisonWidget widget = createComparisonWidget(keyColumns); + final FrameComparisonWidget widget = createComparisonWidget( + frameWithoutComplexColumns, + keyColumns, + ByteRowKeyComparatorTest.NO_COMPLEX_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.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 < frame.numRows(); i++) { + for (int i = 0; i < frameWithoutComplexColumns.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.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN.get(i), + 0, + expectedKeyArray, + 0, + keyColumns.size() + ); Assert.assertEquals( KeyTestUtils.createKey(signature, expectedKeyArray), widget.readKey(i) @@ -142,7 +184,7 @@ public void test_readKey_someColumns() } @Test - public void test_readKey_allColumns() + public void test_noComplexColumns_readKey_allColumns() { final List keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.ASCENDING), @@ -151,18 +193,25 @@ public void test_readKey_allColumns() new KeyColumn("4", KeyOrder.ASCENDING) ); - final FrameComparisonWidget widget = createComparisonWidget(keyColumns); + final FrameComparisonWidget widget = createComparisonWidget( + frameWithoutComplexColumns, + keyColumns, + ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE + ); - for (int i = 0; i < frame.numRows(); i++) { + for (int i = 0; i < frameWithoutComplexColumns.numRows(); i++) { Assert.assertEquals( - KeyTestUtils.createKey(RowKeyComparatorTest.SIGNATURE, RowKeyComparatorTest.ALL_KEY_OBJECTS.get(i)), + KeyTestUtils.createKey( + ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE, + ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN.get(i) + ), widget.readKey(i) ); } } @Test - public void test_compare_frameToKey() + public void test_noComplexColumns_compare_frameToKey() { final List keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.ASCENDING), @@ -171,17 +220,21 @@ public void test_compare_frameToKey() new KeyColumn("4", KeyOrder.ASCENDING) ); - final FrameComparisonWidget widget = createComparisonWidget(keyColumns); + final FrameComparisonWidget widget = createComparisonWidget( + frameWithoutComplexColumns, + keyColumns, + ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE + ); // Compare self-to-self should be equal. - for (int i = 0; i < frame.numRows(); i++) { + for (int i = 0; i < frameWithoutComplexColumns.numRows(); i++) { Assert.assertEquals( 0, widget.compare( i, KeyTestUtils.createKey( - RowKeyComparatorTest.SIGNATURE, - RowKeyComparatorTest.ALL_KEY_OBJECTS.get(i) + ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE, + ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN.get(i) ) ) ); @@ -189,8 +242,8 @@ public void test_compare_frameToKey() // Check some other comparators. final RowKey firstKey = KeyTestUtils.createKey( - RowKeyComparatorTest.SIGNATURE, - RowKeyComparatorTest.ALL_KEY_OBJECTS.get(0) + ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE, + ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN.get(0) ); MatcherAssert.assertThat(widget.compare(0, firstKey), Matchers.equalTo(0)); @@ -202,17 +255,196 @@ public void test_compare_frameToKey() MatcherAssert.assertThat(widget.compare(6, firstKey), Matchers.greaterThan(0)); } - private FrameComparisonWidget createComparisonWidget(final List keyColumns) + @Test + public void test_isPartiallyNullKey_someColumns() + { + final List keyColumns = ImmutableList.of( + new KeyColumn("1", KeyOrder.ASCENDING), + new KeyColumn("2", KeyOrder.ASCENDING), + new KeyColumn("3", KeyOrder.ASCENDING) + ); + + final FrameComparisonWidget widget = createComparisonWidget( + frameWithComplexColumns, + keyColumns, + ByteRowKeyComparatorTest.SIGNATURE + ); + + for (int i = 0; i < frameWithComplexColumns.numRows(); i++) { + final boolean isAllNonNull = + Arrays.stream(ByteRowKeyComparatorTest.ALL_KEY_OBJECTS.get(i)).limit(3).allMatch(Objects::nonNull); + + // 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[]{0}) && widget.hasNonNullKeyParts(i, new int[]{2}) + ); + } + } + + @Test + public void test_isPartiallyNullKey_allColumns() + { + final List 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) + ); + + final FrameComparisonWidget widget = createComparisonWidget( + frameWithComplexColumns, + keyColumns, + ByteRowKeyComparatorTest.SIGNATURE + ); + + for (int i = 0; i < frameWithoutComplexColumns.numRows(); i++) { + final boolean isAllNonNull = + 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})); + } + } + + @Test + public void test_readKey_someColumns() + { + final List keyColumns = ImmutableList.of( + new KeyColumn("1", KeyOrder.ASCENDING), + new KeyColumn("2", KeyOrder.ASCENDING), + new KeyColumn("3", KeyOrder.ASCENDING) + ); + + final FrameComparisonWidget widget = createComparisonWidget( + frameWithComplexColumns, + keyColumns, + ByteRowKeyComparatorTest.SIGNATURE + ); + + final RowSignature signature = + RowSignature.builder() + .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 < frameWithComplexColumns.numRows(); i++) { + final Object[] expectedKeyArray = new Object[keyColumns.size()]; + System.arraycopy(ByteRowKeyComparatorTest.ALL_KEY_OBJECTS.get(i), 0, expectedKeyArray, 0, keyColumns.size()); + Assert.assertEquals( + KeyTestUtils.createKey(signature, expectedKeyArray), + widget.readKey(i) + ); + } + } + + @Test + public void test_readKey_allColumns() + { + final List 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) + ); + + final FrameComparisonWidget widget = createComparisonWidget( + frameWithComplexColumns, + keyColumns, + ByteRowKeyComparatorTest.SIGNATURE + ); + + for (int i = 0; i < frameWithComplexColumns.numRows(); i++) { + Assert.assertEquals( + KeyTestUtils.createKey(ByteRowKeyComparatorTest.SIGNATURE, ByteRowKeyComparatorTest.ALL_KEY_OBJECTS.get(i)), + widget.readKey(i) + ); + } + } + + @Test + public void test_compare_frameToKey() + { + final List 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) + ); + + final FrameComparisonWidget widget = createComparisonWidget( + frameWithComplexColumns, + keyColumns, + ByteRowKeyComparatorTest.SIGNATURE + ); + + // Compare self-to-self should be equal. + for (int i = 0; i < frameWithComplexColumns.numRows(); i++) { + Assert.assertEquals( + 0, + widget.compare( + i, + KeyTestUtils.createKey( + ByteRowKeyComparatorTest.SIGNATURE, + ByteRowKeyComparatorTest.ALL_KEY_OBJECTS.get(i) + ) + ) + ); + } + + // Check some other comparators. + final RowKey firstKey = KeyTestUtils.createKey( + 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.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 Frame frame, + final List 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()) ); diff --git a/processing/src/test/java/org/apache/druid/frame/key/RowKeyComparatorTest.java b/processing/src/test/java/org/apache/druid/frame/key/RowKeyComparatorTest.java index 18e524d1e38b..bc062df9070d 100644 --- a/processing/src/test/java/org/apache/druid/frame/key/RowKeyComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/frame/key/RowKeyComparatorTest.java @@ -22,46 +22,30 @@ 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 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 keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.DESCENDING), @@ -70,13 +54,13 @@ public void test_compare_AAAA() // AAAA = all ascending 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 keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.ASCENDING), @@ -85,13 +69,13 @@ public void test_compare_DDDD() // DDDD = all descending 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 keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.ASCENDING), @@ -100,13 +84,13 @@ public void test_compare_DAAD() 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 keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.DESCENDING), @@ -115,13 +99,13 @@ public void test_compare_ADDA() 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 keyColumns = ImmutableList.of( new KeyColumn("1", KeyOrder.DESCENDING), @@ -130,8 +114,103 @@ public void test_compare_DADA() 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 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 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 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 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 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 void test_equals() .verify(); } - private List sortUsingKeyComparator(final List keyColumns, final List objectss) + private static List sortUsingKeyComparator( + final List keyColumns, + final List objectss, + final RowSignature rowSignature + ) { final List 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 sortUsingObjectComparator(final List keyColumns, final List objectss) + private static List sortUsingObjectComparator( + final List keyColumns, + final List objectss, + final RowSignature rowSignature + ) { final List sortedObjectssCopy = objectss.stream().sorted( (o1, o2) -> { @@ -178,7 +265,7 @@ private List sortUsingObjectComparator(final List keyColumns, final List sortedKeys = new ArrayList<>(); for (final Object[] objects : sortedObjectssCopy) { - sortedKeys.add(KeyTestUtils.createKey(SIGNATURE, objects)); + sortedKeys.add(KeyTestUtils.createKey(rowSignature, objects)); } return sortedKeys; diff --git a/processing/src/test/java/org/apache/druid/frame/key/RowKeyComparisonRunLengthsTest.java b/processing/src/test/java/org/apache/druid/frame/key/RowKeyComparisonRunLengthsTest.java new file mode 100644 index 000000000000..4db0f25b787b --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/key/RowKeyComparisonRunLengthsTest.java @@ -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 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 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 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 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 keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.ASCENDING)); + final List 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 keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.ASCENDING)); + // Any known complex type + final List 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 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 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 expectedResults = expectedResultsBuilder.build(); + + final List> 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, 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, RowSignature> generateKeyColumns( + final List> columnTypeAndKeyOrder, + int index + ) + { + final List 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()); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java index 8211839223c9..5450bd98fd25 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java @@ -290,7 +290,7 @@ private OutputChannels verifySuperSorter( maxBytesPerFrame ) : new FileOutputChannelFactory(tempFolder, maxBytesPerFrame, null); final RowKeyReader keyReader = clusterBy.keyReader(signature); - final Comparator keyComparator = clusterBy.keyComparator(); + final Comparator keyComparator = clusterBy.keyComparator(signature); final SettableFuture clusterByPartitionsFuture = SettableFuture.create(); final SuperSorterProgressTracker superSorterProgressTracker = new SuperSorterProgressTracker(); diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java index 61b01ad0646e..770d79beb76f 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java @@ -37,6 +37,7 @@ 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 @@ @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 void test_double() } @Test - public void test_complex() + public void test_complex_hll() + { + testWithDataset(FrameWriterTestData.TEST_COMPLEX_HLL); + } + + @Test + public void test_complex_nested() { - // 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_NESTED); } @Test @@ -332,26 +343,12 @@ public void test_typePairs() // 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> rowSequence = unsortAndMakeRows(Arrays.asList(dataset1, dataset2)); - // Sort by all columns up to the first COMPLEX one. (Can't sort by COMPLEX.) final List 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 writeResult = writeFrame(rowSequence, signature, sortColumns); @@ -384,14 +381,9 @@ public void test_insufficientWriteCapacity() final Sequence> 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 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 @@ private Sequence> sortIfNeeded( } final RowSignature keySignature = KeyTestUtils.createKeySignature(keyColumns, signature); - final Comparator keyComparator = RowKeyComparator.create(keyColumns); + final Comparator keyComparator = RowKeyComparator.create(keyColumns, signature); return Sequences.sort( rows, @@ -514,10 +506,7 @@ private List computeSortColumns(final List sortColumnNames) return Collections.emptyList(); } else { return sortColumnNames.stream() - .map( - columnName -> - new KeyColumn(columnName, sortedness) - ) + .map(columnName -> new KeyColumn(columnName, sortedness)) .collect(Collectors.toList()); } } diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java index 72990e02086e..d3720cd607c1 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java @@ -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 TEST_COMPLEX = new Dataset<>( + public static final Dataset 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 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 { private final ColumnType type; diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java index 71c67deadb02..d43ba034c2ae 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java @@ -24,12 +24,9 @@ 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 void test_columnar_unsupportedColumnType() 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)") - ) - ); - } - @Test public void test_rowBased_sortColumnsNotPrefix() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 854ac2215985..1d6af0d1c253 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -1155,16 +1155,12 @@ public void assertResultsValid(final ResultMatchMode matchMode, final List types = new ArrayList<>(); - final boolean isMSQ = isMSQRowType(queryResults.signature); - - if (!isMSQ) { - for (int i = 0; i < queryResults.signature.getColumnNames().size(); i++) { - Optional 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 = queryResults.signature.getColumnType(i); + if (columnType.isPresent()) { + types.add(columnType.get().getType()); + } else { + types.add(null); } } @@ -1181,19 +1177,13 @@ public void assertResultsValid(final ResultMatchMode matchMode, final List colNames = signature.getColumnNames(); - return colNames.size() == 1 && "TASK".equals(colNames.get(0)); - } - public void assertResultsEquals(String sql, List expectedResults, List results) { int minSize = Math.min(results.size(), expectedResults.size()); @@ -1340,6 +1330,11 @@ protected void msqIncompatible() assumeFalse(testBuilder().config.isRunningMSQ(), "test case is not MSQ compatible"); } + protected boolean isRunningMSQ() + { + return testBuilder().config.isRunningMSQ(); + } + protected static boolean isRewriteJoinToFilter(final Map queryContext) { return (boolean) queryContext.getOrDefault( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 997c7172eda3..11266b92e6ca 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -88,13 +88,13 @@ 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> RAW_ROWS = ImmutableList.of( + public static final List> RAW_ROWS = ImmutableList.of( ImmutableMap.builder() .put("t", "2000-01-01") .put("string", "aaa") @@ -146,7 +146,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.builder() @@ -160,7 +160,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.builder() @@ -173,10 +173,10 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ).build(), null ); - private static final List ROWS = + public static final List ROWS = RAW_ROWS.stream().map(raw -> TestDataBuilder.createRow(raw, ALL_JSON_COLUMNS)).collect(Collectors.toList()); - private static final List ROWS_MIX = + public static final List ROWS_MIX = RAW_ROWS.stream().map(raw -> TestDataBuilder.createRow(raw, JSON_AND_SCALAR_MIX)).collect(Collectors.toList()); @Override @@ -1074,6 +1074,10 @@ public void testGroupByRootSingleTypeStringMixed2Sparse() @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]'), " @@ -2686,6 +2690,8 @@ public void testGroupByPathSelectorFilterCoalesce() @Test public void testJsonAndArrayAgg() { + // MSQ cannot handle non-primitive arrays + msqIncompatible(); cannotVectorize(); testQuery( "SELECT " @@ -5411,6 +5417,49 @@ public void testJoinOnNestedColumnThrows() public void testScanStringNotNullCast() { skipVectorize(); + final List 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)" @@ -5428,27 +5477,7 @@ public void testScanStringNotNullCast() .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() @@ -5903,6 +5932,8 @@ public void testGroupByAndFilterVariant() @Test public void testScanAllTypesAuto() { + // Variant types are not supported by MSQ. + msqIncompatible(); skipVectorize(); testQuery( "SELECT * FROM druid.all_auto", @@ -6841,6 +6872,8 @@ public void testJsonQueryArrays() @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") @@ -7060,6 +7093,10 @@ public void testUnnestJsonQueryArraysJsonValueSum() @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( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java index df14aeb105a0..5dc7cbd6f1a9 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java @@ -146,6 +146,26 @@ public QueryResults( this.capture = capture; } + public QueryResults( + final Map queryContext, + final String vectorizeOption, + final RowSignature rowSignature, + final List results, + final List> 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 queryContext, final String vectorizeOption, @@ -163,9 +183,9 @@ public QueryResults( this.sqlSignature = null; } - public QueryResults withResults(List newResults) + public QueryResults withSignatureAndResults(final RowSignature rowSignature, final List newResults) { - return new QueryResults(queryContext, vectorizeOption, sqlSignature, newResults, recordedQueries, capture); + return new QueryResults(queryContext, vectorizeOption, rowSignature, newResults, recordedQueries, capture); } }