This is an automated email from the ASF dual-hosted git repository.
karan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git
The following commit(s) were added to refs/heads/master by this push:
new a3edda3b63 Modify quantile sketches to add byte[] directly (#13351)
a3edda3b63 is described below
commit a3edda3b634d00942f8729930dd45e2dd6b2eaff
Author: Adarsh Sanjeev <[email protected]>
AuthorDate: Mon Nov 14 00:24:06 2022 +0530
Modify quantile sketches to add byte[] directly (#13351)
* Modify quantile sketchs to add byte[] directly
* Rename class and add test
---
.../statistics/QuantilesSketchKeyCollector.java | 26 ++---
.../QuantilesSketchKeyCollectorFactory.java | 34 +++---
...eyComparator.java => ByteRowKeyComparator.java} | 19 ++--
.../java/org/apache/druid/frame/key/ClusterBy.java | 8 ++
.../druid/frame/key/FrameComparisonWidgetImpl.java | 4 +-
.../java/org/apache/druid/frame/key/RowKey.java | 7 +-
.../apache/druid/frame/key/RowKeyComparator.java | 119 ++-------------------
...atorTest.java => ByteRowKeyComparatorTest.java} | 31 +++---
.../druid/frame/key/RowKeyComparatorTest.java | 5 +-
9 files changed, 78 insertions(+), 175 deletions(-)
diff --git
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java
index 2a5ee777a0..e7f0708d78 100644
---
a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java
+++
b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/statistics/QuantilesSketchKeyCollector.java
@@ -44,13 +44,13 @@ import java.util.NoSuchElementException;
*/
public class QuantilesSketchKeyCollector implements
KeyCollector<QuantilesSketchKeyCollector>
{
- private final Comparator<RowKey> comparator;
- private ItemsSketch<RowKey> sketch;
+ private final Comparator<byte[]> comparator;
+ private ItemsSketch<byte[]> sketch;
private double averageKeyLength;
QuantilesSketchKeyCollector(
- final Comparator<RowKey> comparator,
- @Nullable final ItemsSketch<RowKey> sketch,
+ final Comparator<byte[]> comparator,
+ @Nullable final ItemsSketch<byte[]> sketch,
double averageKeyLength
)
{
@@ -67,7 +67,7 @@ public class QuantilesSketchKeyCollector implements
KeyCollector<QuantilesSketch
estimatedTotalSketchSizeInBytes += key.estimatedObjectSizeBytes() * weight;
for (int i = 0; i < weight; i++) {
// Add the same key multiple times to make it "heavier".
- sketch.update(key);
+ sketch.update(key.array());
}
averageKeyLength = (estimatedTotalSketchSizeInBytes / sketch.getN());
}
@@ -75,7 +75,7 @@ public class QuantilesSketchKeyCollector implements
KeyCollector<QuantilesSketch
@Override
public void addAll(QuantilesSketchKeyCollector other)
{
- final ItemsUnion<RowKey> union = ItemsUnion.getInstance(
+ final ItemsUnion<byte[]> union = ItemsUnion.getInstance(
Math.max(sketch.getK(), other.sketch.getK()),
comparator
);
@@ -129,10 +129,10 @@ public class QuantilesSketchKeyCollector implements
KeyCollector<QuantilesSketch
@Override
public RowKey minKey()
{
- final RowKey minValue = sketch.getMinValue();
+ final byte[] minValue = sketch.getMinValue();
if (minValue != null) {
- return minValue;
+ return RowKey.wrap(minValue);
} else {
throw new NoSuchElementException();
}
@@ -152,20 +152,20 @@ public class QuantilesSketchKeyCollector implements
KeyCollector<QuantilesSketch
final int numPartitions = Ints.checkedCast(LongMath.divide(sketch.getN(),
targetWeight, RoundingMode.CEILING));
// numPartitions + 1, because the final quantile is the max, and we won't
build a partition based on that.
- final RowKey[] quantiles = sketch.getQuantiles(numPartitions + 1);
+ final byte[][] quantiles = sketch.getQuantiles(numPartitions + 1);
final List<ClusterByPartition> partitions = new ArrayList<>();
for (int i = 0; i < numPartitions; i++) {
final boolean isFinalPartition = i == numPartitions - 1;
if (isFinalPartition) {
- partitions.add(new ClusterByPartition(quantiles[i], null));
+ partitions.add(new ClusterByPartition(RowKey.wrap(quantiles[i]),
null));
} else {
- final ClusterByPartition partition = new
ClusterByPartition(quantiles[i], quantiles[i + 1]);
- final int cmp = comparator.compare(partition.getStart(),
partition.getEnd());
+ final int cmp = comparator.compare(quantiles[i], quantiles[i + 1]);
if (cmp < 0) {
// Skip partitions where start == end.
// I don't think start can be greater than end, but if that happens,
skip them too!
+ final ClusterByPartition partition = new
ClusterByPartition(RowKey.wrap(quantiles[i]), RowKey.wrap(quantiles[i + 1]));
partitions.add(partition);
}
}
@@ -177,7 +177,7 @@ public class QuantilesSketchKeyCollector implements
KeyCollector<QuantilesSketch
/**
* Retrieves the backing sketch. Exists for usage by {@link
QuantilesSketchKeyCollectorFactory}.
*/
- ItemsSketch<RowKey> getSketch()
+ ItemsSketch<byte[]> getSketch()
{
return sketch;
}
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 1682987bec..c6e98e31b2 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
@@ -28,7 +28,6 @@ import org.apache.datasketches.memory.Memory;
import org.apache.datasketches.memory.WritableMemory;
import org.apache.datasketches.quantiles.ItemsSketch;
import org.apache.druid.frame.key.ClusterBy;
-import org.apache.druid.frame.key.RowKey;
import org.apache.druid.java.util.common.StringUtils;
import java.io.IOException;
@@ -42,16 +41,16 @@ public class QuantilesSketchKeyCollectorFactory
@VisibleForTesting
static final int SKETCH_INITIAL_K = 1 << 15;
- private final Comparator<RowKey> comparator;
+ private final Comparator<byte[]> comparator;
- private QuantilesSketchKeyCollectorFactory(final Comparator<RowKey>
comparator)
+ private QuantilesSketchKeyCollectorFactory(final Comparator<byte[]>
comparator)
{
this.comparator = comparator;
}
static QuantilesSketchKeyCollectorFactory create(final ClusterBy clusterBy)
{
- return new QuantilesSketchKeyCollectorFactory(clusterBy.keyComparator());
+ return new
QuantilesSketchKeyCollectorFactory(clusterBy.byteKeyComparator());
}
@Override
@@ -78,7 +77,7 @@ public class QuantilesSketchKeyCollectorFactory
public QuantilesSketchKeyCollectorSnapshot
toSnapshot(QuantilesSketchKeyCollector collector)
{
final String encodedSketch =
-
StringUtils.encodeBase64String(collector.getSketch().toByteArray(RowKeySerde.INSTANCE));
+
StringUtils.encodeBase64String(collector.getSketch().toByteArray(ByteRowKeySerde.INSTANCE));
return new QuantilesSketchKeyCollectorSnapshot(encodedSketch,
collector.getAverageKeyLength());
}
@@ -87,26 +86,26 @@ public class QuantilesSketchKeyCollectorFactory
{
final String encodedSketch = snapshot.getEncodedSketch();
final byte[] bytes = StringUtils.decodeBase64String(encodedSketch);
- final ItemsSketch<RowKey> sketch =
- ItemsSketch.getInstance(Memory.wrap(bytes), comparator,
RowKeySerde.INSTANCE);
+ final ItemsSketch<byte[]> sketch =
+ ItemsSketch.getInstance(Memory.wrap(bytes), comparator,
ByteRowKeySerde.INSTANCE);
return new QuantilesSketchKeyCollector(comparator, sketch,
snapshot.getAverageKeyLength());
}
- private static class RowKeySerde extends ArrayOfItemsSerDe<RowKey>
+ private static class ByteRowKeySerde extends ArrayOfItemsSerDe<byte[]>
{
- private static final RowKeySerde INSTANCE = new RowKeySerde();
+ private static final ByteRowKeySerde INSTANCE = new ByteRowKeySerde();
- private RowKeySerde()
+ private ByteRowKeySerde()
{
}
@Override
- public byte[] serializeToByteArray(final RowKey[] items)
+ public byte[] serializeToByteArray(final byte[][] items)
{
int serializedSize = Integer.BYTES * items.length;
- for (final RowKey key : items) {
- serializedSize += key.array().length;
+ for (final byte[] key : items) {
+ serializedSize += key.length;
}
final byte[] serializedBytes = new byte[serializedSize];
@@ -114,8 +113,7 @@ public class QuantilesSketchKeyCollectorFactory
long keyWritePosition = (long) Integer.BYTES * items.length;
for (int i = 0; i < items.length; i++) {
- final RowKey key = items[i];
- final byte[] keyBytes = key.array();
+ final byte[] keyBytes = items[i];
writableMemory.putInt((long) Integer.BYTES * i, keyBytes.length);
writableMemory.putByteArray(keyWritePosition, keyBytes, 0,
keyBytes.length);
@@ -128,9 +126,9 @@ public class QuantilesSketchKeyCollectorFactory
}
@Override
- public RowKey[] deserializeFromMemory(final Memory mem, final int numItems)
+ public byte[][] deserializeFromMemory(final Memory mem, final int numItems)
{
- final RowKey[] keys = new RowKey[numItems];
+ final byte[][] keys = new byte[numItems][];
long keyPosition = (long) Integer.BYTES * numItems;
for (int i = 0; i < numItems; i++) {
@@ -138,7 +136,7 @@ public class QuantilesSketchKeyCollectorFactory
final byte[] keyBytes = new byte[keyLength];
mem.getByteArray(keyPosition, keyBytes, 0, keyLength);
- keys[i] = RowKey.wrap(keyBytes);
+ keys[i] = keyBytes;
keyPosition += keyLength;
}
diff --git
a/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparator.java
b/processing/src/main/java/org/apache/druid/frame/key/ByteRowKeyComparator.java
similarity index 91%
copy from
processing/src/main/java/org/apache/druid/frame/key/RowKeyComparator.java
copy to
processing/src/main/java/org/apache/druid/frame/key/ByteRowKeyComparator.java
index 7e82631139..5e00946736 100644
--- a/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparator.java
+++
b/processing/src/main/java/org/apache/druid/frame/key/ByteRowKeyComparator.java
@@ -30,17 +30,17 @@ import java.util.List;
import java.util.Objects;
/**
- * Comparator for {@link RowKey} instances.
+ * Comparator for byte arrays from {@link RowKey#key} instances.
*
* Comparison logic in this class is very similar to {@link
FrameComparisonWidget}, but is different because it works
* on byte[] instead of Frames.
*/
-public class RowKeyComparator implements Comparator<RowKey>
+public class ByteRowKeyComparator implements Comparator<byte[]>
{
private final int firstFieldPosition;
private final int[] ascDescRunLengths;
- private RowKeyComparator(
+ private ByteRowKeyComparator(
final int firstFieldPosition,
final int[] ascDescRunLengths
)
@@ -49,9 +49,9 @@ public class RowKeyComparator implements Comparator<RowKey>
this.ascDescRunLengths = ascDescRunLengths;
}
- public static RowKeyComparator create(final List<SortColumn> keyColumns)
+ public static ByteRowKeyComparator create(final List<SortColumn> keyColumns)
{
- return new RowKeyComparator(
+ return new ByteRowKeyComparator(
computeFirstFieldPosition(keyColumns.size()),
computeAscDescRunLengths(keyColumns)
);
@@ -100,14 +100,11 @@ public class RowKeyComparator implements
Comparator<RowKey>
@Override
@SuppressWarnings("SubtractionInCompareTo")
- public int compare(final RowKey key1, final RowKey key2)
+ public int compare(final byte[] keyArray1, final byte[] keyArray2)
{
// Similar logic to FrameComparaisonWidgetImpl, but implementation is
different enough that we need our own.
// Major difference is Frame v. Frame instead of byte[] v. byte[].
- final byte[] keyArray1 = key1.array();
- final byte[] keyArray2 = key2.array();
-
int comparableBytesStartPosition1 = firstFieldPosition;
int comparableBytesStartPosition2 = firstFieldPosition;
@@ -153,7 +150,7 @@ public class RowKeyComparator implements Comparator<RowKey>
if (o == null || getClass() != o.getClass()) {
return false;
}
- RowKeyComparator that = (RowKeyComparator) o;
+ ByteRowKeyComparator that = (ByteRowKeyComparator) o;
return firstFieldPosition == that.firstFieldPosition
&& Arrays.equals(ascDescRunLengths, that.ascDescRunLengths);
}
@@ -169,7 +166,7 @@ public class RowKeyComparator implements Comparator<RowKey>
@Override
public String toString()
{
- return "RowKeyComparator{" +
+ return "ByteRowKeyComparator{" +
"firstFieldPosition=" + firstFieldPosition +
", ascDescRunLengths=" + Arrays.toString(ascDescRunLengths) +
'}';
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 43c00cdaa0..367802e000 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
@@ -125,6 +125,14 @@ public class ClusterBy
return RowKeyComparator.create(columns);
}
+ /**
+ * Comparator that compares byte arrays of keys for this instance using the
given signature directly.
+ */
+ public Comparator<byte[]> byteKeyComparator()
+ {
+ return ByteRowKeyComparator.create(columns);
+ }
+
/**
* Comparator that compares bucket keys for this instance. Bucket keys are
retrieved by calling
* {@link RowKeyReader#trim(RowKey, int)} with {@link #getBucketByCount()}.
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 de0ccf9a9d..0aeed47760 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
@@ -89,8 +89,8 @@ public class FrameComparisonWidgetImpl implements
FrameComparisonWidget
frame.region(RowBasedFrameWriter.ROW_OFFSET_REGION),
frame.region(RowBasedFrameWriter.ROW_DATA_REGION),
sortColumns.size(),
-
RowKeyComparator.computeFirstFieldPosition(frameReader.signature().size()),
- RowKeyComparator.computeAscDescRunLengths(sortColumns)
+
ByteRowKeyComparator.computeFirstFieldPosition(frameReader.signature().size()),
+ ByteRowKeyComparator.computeAscDescRunLengths(sortColumns)
);
}
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 89328414b9..1c63afd201 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
@@ -32,9 +32,8 @@ public class RowKey
{
private static final RowKey EMPTY_KEY = new RowKey(new byte[0]);
- // Constant to account for hashcode and object overhead
- // 24 bytes (header) + 8 bytes (reference) + 8 bytes (hashCode long) + 4
bytes (safe estimate of hashCodeComputed)
- static final int OBJECT_OVERHEAD_SIZE_BYTES = 44;
+ // Constant to account for byte array overhead.
+ static final int OBJECT_OVERHEAD_SIZE_BYTES = 24;
private final byte[] key;
@@ -114,7 +113,7 @@ public class RowKey
}
/**
- * Estimate number of bytes taken by an object of {@link RowKey}. Only
returns an estimate and does not account for
+ * Estimate number of bytes taken by the key array. Only returns an estimate
and does not account for
* platform or JVM specific implementation.
*/
public int estimatedObjectSizeBytes()
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 7e82631139..c4d73696b1 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,129 +19,32 @@
package org.apache.druid.frame.key;
-import com.google.common.primitives.Ints;
-import it.unimi.dsi.fastutil.ints.IntArrayList;
-import it.unimi.dsi.fastutil.ints.IntList;
-import org.apache.druid.frame.read.FrameReaderUtils;
-
-import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
-import java.util.Objects;
/**
* Comparator for {@link RowKey} instances.
*
- * Comparison logic in this class is very similar to {@link
FrameComparisonWidget}, but is different because it works
- * on byte[] instead of Frames.
+ * Delegates the comparing to a {@link ByteRowKeyComparator}.
*/
public class RowKeyComparator implements Comparator<RowKey>
{
- private final int firstFieldPosition;
- private final int[] ascDescRunLengths;
+ private final ByteRowKeyComparator byteRowKeyComparatorDelegate;
- private RowKeyComparator(
- final int firstFieldPosition,
- final int[] ascDescRunLengths
- )
+ private RowKeyComparator(final ByteRowKeyComparator
byteRowKeyComparatorDelegate)
{
- this.firstFieldPosition = firstFieldPosition;
- this.ascDescRunLengths = ascDescRunLengths;
+ this.byteRowKeyComparatorDelegate = byteRowKeyComparatorDelegate;
}
public static RowKeyComparator create(final List<SortColumn> keyColumns)
{
- return new RowKeyComparator(
- computeFirstFieldPosition(keyColumns.size()),
- computeAscDescRunLengths(keyColumns)
- );
- }
-
- /**
- * Compute the offset into each key where the first field starts.
- *
- * Public so {@link FrameComparisonWidgetImpl} can use it.
- */
- 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<SortColumn>
sortColumns)
- {
- final IntList ascDescRunLengths = new IntArrayList(4);
-
- boolean descending = false;
- int runLength = 0;
-
- for (final SortColumn column : sortColumns) {
- if (column.descending() != descending) {
- ascDescRunLengths.add(runLength);
- runLength = 0;
- descending = !descending;
- }
-
- runLength++;
- }
-
- if (runLength > 0) {
- ascDescRunLengths.add(runLength);
- }
-
- return ascDescRunLengths.toIntArray();
+ return new RowKeyComparator(ByteRowKeyComparator.create(keyColumns));
}
@Override
- @SuppressWarnings("SubtractionInCompareTo")
public int compare(final RowKey key1, final RowKey key2)
{
- // Similar logic to FrameComparaisonWidgetImpl, but implementation is
different enough that we need our own.
- // Major difference is Frame v. Frame instead of byte[] v. byte[].
-
- final byte[] keyArray1 = key1.array();
- final byte[] keyArray2 = key2.array();
-
- int comparableBytesStartPosition1 = firstFieldPosition;
- int comparableBytesStartPosition2 = firstFieldPosition;
-
- boolean ascending = true;
- int field = 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);
-
- int cmp = FrameReaderUtils.compareByteArraysUnsigned(
- keyArray1,
- comparableBytesStartPosition1,
- comparableBytesEndPosition1 - comparableBytesStartPosition1,
- keyArray2,
- comparableBytesStartPosition2,
- comparableBytesEndPosition2 - comparableBytesStartPosition2
- );
-
- if (cmp != 0) {
- return ascending ? cmp : -cmp;
- }
-
- field = nextField;
- comparableBytesStartPosition1 = comparableBytesEndPosition1;
- comparableBytesStartPosition2 = comparableBytesEndPosition2;
- }
-
- ascending = !ascending;
- }
-
- return 0;
+ return byteRowKeyComparatorDelegate.compare(key1.array(), key2.array());
}
@Override
@@ -154,24 +57,20 @@ public class RowKeyComparator implements Comparator<RowKey>
return false;
}
RowKeyComparator that = (RowKeyComparator) o;
- return firstFieldPosition == that.firstFieldPosition
- && Arrays.equals(ascDescRunLengths, that.ascDescRunLengths);
+ return
byteRowKeyComparatorDelegate.equals(that.byteRowKeyComparatorDelegate);
}
@Override
public int hashCode()
{
- int result = Objects.hash(firstFieldPosition);
- result = 31 * result + Arrays.hashCode(ascDescRunLengths);
- return result;
+ return byteRowKeyComparatorDelegate.hashCode();
}
@Override
public String toString()
{
return "RowKeyComparator{" +
- "firstFieldPosition=" + firstFieldPosition +
- ", ascDescRunLengths=" + Arrays.toString(ascDescRunLengths) +
+ "byteRowKeyComparatorDelegate=" + byteRowKeyComparatorDelegate +
'}';
}
}
diff --git
a/processing/src/test/java/org/apache/druid/frame/key/RowKeyComparatorTest.java
b/processing/src/test/java/org/apache/druid/frame/key/ByteRowKeyComparatorTest.java
similarity index 85%
copy from
processing/src/test/java/org/apache/druid/frame/key/RowKeyComparatorTest.java
copy to
processing/src/test/java/org/apache/druid/frame/key/ByteRowKeyComparatorTest.java
index a41226ca80..fc9755642e 100644
---
a/processing/src/test/java/org/apache/druid/frame/key/RowKeyComparatorTest.java
+++
b/processing/src/test/java/org/apache/druid/frame/key/ByteRowKeyComparatorTest.java
@@ -33,7 +33,7 @@ import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
-public class RowKeyComparatorTest extends InitializedNullHandlingTest
+public class ByteRowKeyComparatorTest extends InitializedNullHandlingTest
{
static final RowSignature SIGNATURE =
RowSignature.builder()
@@ -71,7 +71,7 @@ public class RowKeyComparatorTest extends
InitializedNullHandlingTest
);
Assert.assertEquals(
sortUsingObjectComparator(sortColumns, ALL_KEY_OBJECTS),
- sortUsingKeyComparator(sortColumns, ALL_KEY_OBJECTS)
+ sortUsingByteKeyComparator(sortColumns, ALL_KEY_OBJECTS)
);
}
@@ -86,7 +86,7 @@ public class RowKeyComparatorTest extends
InitializedNullHandlingTest
);
Assert.assertEquals(
sortUsingObjectComparator(sortColumns, ALL_KEY_OBJECTS),
- sortUsingKeyComparator(sortColumns, ALL_KEY_OBJECTS)
+ sortUsingByteKeyComparator(sortColumns, ALL_KEY_OBJECTS)
);
}
@@ -101,7 +101,7 @@ public class RowKeyComparatorTest extends
InitializedNullHandlingTest
);
Assert.assertEquals(
sortUsingObjectComparator(sortColumns, ALL_KEY_OBJECTS),
- sortUsingKeyComparator(sortColumns, ALL_KEY_OBJECTS)
+ sortUsingByteKeyComparator(sortColumns, ALL_KEY_OBJECTS)
);
}
@@ -116,7 +116,7 @@ public class RowKeyComparatorTest extends
InitializedNullHandlingTest
);
Assert.assertEquals(
sortUsingObjectComparator(sortColumns, ALL_KEY_OBJECTS),
- sortUsingKeyComparator(sortColumns, ALL_KEY_OBJECTS)
+ sortUsingByteKeyComparator(sortColumns, ALL_KEY_OBJECTS)
);
}
@@ -131,26 +131,25 @@ public class RowKeyComparatorTest extends
InitializedNullHandlingTest
);
Assert.assertEquals(
sortUsingObjectComparator(sortColumns, ALL_KEY_OBJECTS),
- sortUsingKeyComparator(sortColumns, ALL_KEY_OBJECTS)
+ sortUsingByteKeyComparator(sortColumns, ALL_KEY_OBJECTS)
);
}
@Test
public void test_equals()
{
- EqualsVerifier.forClass(RowKeyComparator.class).usingGetClass().verify();
+ EqualsVerifier.forClass(ByteRowKeyComparator.class)
+ .usingGetClass()
+ .verify();
}
- private List<RowKey> sortUsingKeyComparator(final List<SortColumn>
sortColumns, final List<Object[]> objectss)
+ private List<RowKey> sortUsingByteKeyComparator(final List<SortColumn>
sortColumns, final List<Object[]> objectss)
{
- final List<RowKey> sortedKeys = new ArrayList<>();
-
- for (final Object[] objects : objectss) {
- sortedKeys.add(KeyTestUtils.createKey(SIGNATURE, objects));
- }
-
- sortedKeys.sort(RowKeyComparator.create(sortColumns));
- return sortedKeys;
+ return objectss.stream()
+ .map(objects -> KeyTestUtils.createKey(SIGNATURE,
objects).array())
+ .sorted(ByteRowKeyComparator.create(sortColumns))
+ .map(RowKey::wrap)
+ .collect(Collectors.toList());
}
private List<RowKey> sortUsingObjectComparator(final List<SortColumn>
sortColumns, final List<Object[]> objectss)
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 a41226ca80..33b6d054e4 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
@@ -138,7 +138,10 @@ public class RowKeyComparatorTest extends
InitializedNullHandlingTest
@Test
public void test_equals()
{
- EqualsVerifier.forClass(RowKeyComparator.class).usingGetClass().verify();
+ EqualsVerifier.forClass(RowKeyComparator.class)
+ .withNonnullFields("byteRowKeyComparatorDelegate")
+ .usingGetClass()
+ .verify();
}
private List<RowKey> sortUsingKeyComparator(final List<SortColumn>
sortColumns, final List<Object[]> objectss)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]