Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Support sorting on complex columns in MSQ #16322

Merged
merged 29 commits into from
May 13, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
29 commits
Select commit Hold shift + click to select a range
1c46cb4
init
LakshSingla Apr 23, 2024
486d29a
working, almost
LakshSingla Apr 26, 2024
260206b
stuff working
LakshSingla May 3, 2024
b93b53e
Merge branch 'master' into msq-complex-sorting
LakshSingla May 3, 2024
ac8ba6b
tests, checkstyle
LakshSingla May 6, 2024
abbf49b
tests
LakshSingla May 6, 2024
b7ea7b8
more changes
LakshSingla May 6, 2024
f3caaf6
test comments
LakshSingla May 8, 2024
c852c35
changes
LakshSingla May 8, 2024
d2dd402
tests
LakshSingla May 8, 2024
a0d29d4
tests
LakshSingla May 8, 2024
ae87dab
tests
LakshSingla May 8, 2024
4b86b31
better comment
LakshSingla May 8, 2024
1ec5796
tests fix
LakshSingla May 8, 2024
b68948e
tests fix
LakshSingla May 8, 2024
08633f0
tests fix, test framework fix, comments
LakshSingla May 9, 2024
8d49678
Trigger Build
LakshSingla May 9, 2024
76fd60e
Merge branch 'master' into msq-complex-sorting
LakshSingla May 9, 2024
cd29bec
merge fix
LakshSingla May 9, 2024
690c5cf
convert list to array
LakshSingla May 9, 2024
d1d28e1
add back old tests
LakshSingla May 9, 2024
7f27724
preserve old tests, add new tests for complexcol + byte comparable col
LakshSingla May 9, 2024
ae77984
tests
LakshSingla May 10, 2024
2af2977
add benchmarks for nested data
LakshSingla May 10, 2024
b1c61cb
final set, have separate methods
LakshSingla May 10, 2024
eaa0593
some more final changes
LakshSingla May 10, 2024
2e23831
Merge branch 'master' into msq-complex-sorting
LakshSingla May 10, 2024
7356aa2
review comments
LakshSingla May 13, 2024
806b1ec
Update FrameWriterUtils.java
LakshSingla May 13, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
changes
  • Loading branch information
LakshSingla committed May 8, 2024
commit c852c35555d552a3fef2896637bf693af7734f79
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,8 @@ public int compare(final byte[] keyArray1, final byte[] keyArray2)
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;
Expand All @@ -134,43 +136,33 @@ public int compare(final byte[] keyArray1, final byte[] keyArray2)
complexTypeName
);


int cmp = FrameReaderUtils.compareComplexTypes(
cmp = FrameReaderUtils.compareComplexTypes(
keyArray1,
currentRunStartPosition1,
keyArray2,
currentRunStartPosition2,
columnType,
serde
);
if (cmp != 0) {
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
}
// We have only compared a single field here
fieldsComparedTillNow = nextField;
currentRunStartPosition1 = currentRunEndPosition1;
currentRunStartPosition2 = currentRunEndPosition2;
} else {
// The keys are byte comparable
int nextField = fieldsComparedTillNow + runLengthEntry.getRunLength();
final int currentRunEndPosition1 = RowKeyReader.fieldEndPosition(keyArray1, nextField - 1);
final int currentRunEndPosition2 = RowKeyReader.fieldEndPosition(keyArray2, nextField - 1);
int cmp = FrameReaderUtils.compareByteArraysUnsigned(
cmp = FrameReaderUtils.compareByteArraysUnsigned(
keyArray1,
currentRunStartPosition1,
currentRunEndPosition1 - currentRunStartPosition1,
keyArray2,
currentRunStartPosition2,
currentRunEndPosition2 - currentRunStartPosition2
);
if (cmp != 0) {
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
}
}

fieldsComparedTillNow = nextField;
currentRunStartPosition1 = currentRunEndPosition1;
currentRunStartPosition2 = currentRunEndPosition2;
if (cmp != 0) {
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
}

fieldsComparedTillNow = nextField;
currentRunStartPosition1 = currentRunEndPosition1;
currentRunStartPosition2 = currentRunEndPosition2;
}
return 0;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -210,6 +210,13 @@ public int compare(int row, RowKey key)
continue;
}

// Index of the next field that will get considered. Excludes the current field that we are comparing right now
final int nextField = fieldsComparedTillNow + runLengthEntry.getRunLength();
final int comparableBytesEndPositionInKey = RowKeyReader.fieldEndPosition(keyArray, nextField - 1);
final int comparableBytesEndPositionInRow = getFieldEndPositionInRow(rowPosition, nextField - 1);

final int cmp;

if (!runLengthEntry.isByteComparable()) {
// Only complex types are not byte comparable. Nested arrays aren't supported in MSQ
assert runLengthEntry.getRunLength() == 1;
Expand All @@ -228,48 +235,31 @@ public int compare(int row, RowKey key)
complexTypeName
);

// Index of the next field that will get considered. Excludes the current field that we are comparing right now
final int nextField = fieldsComparedTillNow + 1;
final int comparableBytesEndPositionInKey = RowKeyReader.fieldEndPosition(keyArray, nextField - 1);
final int comparableBytesEndPositionInRow = getFieldEndPositionInRow(rowPosition, nextField - 1);

int cmp = FrameReaderUtils.compareComplexTypes(
cmp = FrameReaderUtils.compareComplexTypes(
dataRegion,
rowPosition + comparableBytesStartPositionInRow,
keyArray,
comparableBytesStartPositionInKey,
columnType,
serde
);

if (cmp != 0) {
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
}

fieldsComparedTillNow = nextField;
comparableBytesStartPositionInRow = comparableBytesEndPositionInRow;
comparableBytesStartPositionInKey = comparableBytesEndPositionInKey;
} else {
int nextField = fieldsComparedTillNow + runLengthEntry.getRunLength();
final long comparableBytesEndPositionInRow = getFieldEndPositionInRow(rowPosition, nextField - 1);
final int comparableBytesEndPositionInKey = RowKeyReader.fieldEndPosition(keyArray, nextField - 1);

int cmp = FrameReaderUtils.compareMemoryToByteArrayUnsigned(
cmp = FrameReaderUtils.compareMemoryToByteArrayUnsigned(
dataRegion,
rowPosition + comparableBytesStartPositionInRow,
comparableBytesEndPositionInRow - comparableBytesStartPositionInRow,
keyArray,
comparableBytesStartPositionInKey,
comparableBytesEndPositionInKey - comparableBytesStartPositionInKey
);
if (cmp != 0) {
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
}
fieldsComparedTillNow = nextField;
comparableBytesStartPositionInRow = comparableBytesEndPositionInRow;
comparableBytesStartPositionInKey = comparableBytesEndPositionInKey;
}

if (cmp != 0) {
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
}
fieldsComparedTillNow = nextField;
comparableBytesStartPositionInRow = comparableBytesEndPositionInRow;
comparableBytesStartPositionInKey = comparableBytesEndPositionInKey;
}
return 0;
}
Expand Down Expand Up @@ -300,6 +290,12 @@ public int compare(final int row, final FrameComparisonWidget otherWidget, final
continue;
}

final int nextField = fieldsComparedTillNow + runLengthEntry.getRunLength();
final int comparableBytesEndPositionInRow = getFieldEndPositionInRow(rowPosition, nextField - 1);
final int otherComparableBytesEndPositionInRow = otherWidgetImpl.getFieldEndPositionInRow(otherRowPosition, nextField - 1);

final int cmp;

if (!runLengthEntry.isByteComparable()) {
// Only complex types are not byte comparable. Nested arrays aren't supported in MSQ
assert runLengthEntry.getRunLength() == 1;
Expand Down Expand Up @@ -331,47 +327,32 @@ public int compare(final int row, final FrameComparisonWidget otherWidget, final
)
);

int nextField = fieldsComparedTillNow + 1;
final int comparableBytesEndPositionInRow = getFieldEndPositionInRow(rowPosition, nextField - 1);
final int otherComparableBytesEndPositionInRow = getFieldEndPositionInRow(otherRowPosition, nextField - 1);

int cmp = FrameReaderUtils.compareComplexTypes(
cmp = FrameReaderUtils.compareComplexTypes(
dataRegion,
rowPosition + comparableBytesStartPositionInRow,
otherWidgetImpl.dataRegion,
otherRowPosition + otherComparableBytesStartPositionInRow,
columnType1,
serde
);

if (cmp != 0) {
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
}

fieldsComparedTillNow = nextField;
comparableBytesStartPositionInRow = comparableBytesEndPositionInRow;
otherComparableBytesStartPositionInRow = otherComparableBytesEndPositionInRow;
} else {
int nextField = fieldsComparedTillNow + runLengthEntry.getRunLength();
final int comparableBytesEndPositionInRow = getFieldEndPositionInRow(rowPosition, nextField - 1);
final int otherComparableBytesEndPositionInRow =
otherWidgetImpl.getFieldEndPositionInRow(otherRowPosition, nextField - 1);

int cmp = FrameReaderUtils.compareMemoryUnsigned(
cmp = FrameReaderUtils.compareMemoryUnsigned(
dataRegion,
rowPosition + comparableBytesStartPositionInRow,
comparableBytesEndPositionInRow - comparableBytesStartPositionInRow,
otherWidgetImpl.getDataRegion(),
otherRowPosition + otherComparableBytesStartPositionInRow,
otherComparableBytesEndPositionInRow - otherComparableBytesStartPositionInRow
);
if (cmp != 0) {
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
}
fieldsComparedTillNow = nextField;
comparableBytesStartPositionInRow = comparableBytesEndPositionInRow;
otherComparableBytesStartPositionInRow = otherComparableBytesEndPositionInRow;
}

if (cmp != 0) {
return runLengthEntry.getOrder() == KeyOrder.ASCENDING ? cmp : -cmp;
}

fieldsComparedTillNow = nextField;
comparableBytesStartPositionInRow = comparableBytesEndPositionInRow;
otherComparableBytesStartPositionInRow = otherComparableBytesEndPositionInRow;
}

return 0;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,128 @@
/*
* 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.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.junit.Assert;
import org.junit.Test;

import java.util.Collections;
import java.util.List;

public class RowKeyComparisonRunLengthsTest
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In a situation like this, I like to have one test that does a whole space of things. Here I suggest a test case that checks all possible length-3 keys where we vary each key element's type between string and complex, and direction between asc and desc. So that's 4 possibilities for each key element (string asc, string desc, complex asc, complex desc) and therefore 64 keys we're testing. It will run fast since it's only 64 cases, and it gets good coverage of different possibilities for runs and orderings.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was considering having a parameterized test, however, how do I add the expected results for each test case generically, without repeating the logic in RowKeyComparisonRunLengths.

For example, if the input test case is "complex ASC, string DESC, string DESC"; how can I get the expected value of the result, without repeating what I have written in RowKeyComparisonRunLength? One way would be iterating the results of all 64 combinations, which can be done, but I wanted to confirm if that's what you referred to in the comment.

The test case would read something like:

assertionHelper(
  HelperUtils.createCheckerFor(new RunLength(false, 1, ASC), new RunLength(true, 2, DESC))
  RowKeyComparisonRunLength.create("complex ASC, string DESC, string DESC")
);
.... (total 64 entries)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was thinking of having all 64 test cases enumerated.

Copy link
Contributor Author

@LakshSingla LakshSingla May 10, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added the test cases - the tests are generated programmatically in the loop (mapping each index to the corresponding permutation of the key column), and the expectations are enumerated corresponding to each test case

{

@Test
public void testRunLengthsWithNoKeyColumns()
{
final List<KeyColumn> keyColumns = Collections.emptyList();
final RowSignature signature = RowSignature.empty();
final RowKeyComparisonRunLengths runLengths = RowKeyComparisonRunLengths.create(keyColumns, signature);
Assert.assertEquals(0, runLengths.getRunLengthEntries().size());
}

@Test
public void testRunLengthsWithInvalidOrder()
{
final List<KeyColumn> keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.NONE));
final RowSignature signature = RowSignature.builder()
.add("a", ColumnType.LONG)
.build();
Assert.assertThrows(DruidException.class, () -> RowKeyComparisonRunLengths.create(keyColumns, signature));
}

@Test
public void testRunLengthsWithIncompleteRowSignature()
{
final List<KeyColumn> keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.NONE));
final RowSignature signature = RowSignature.empty();
Assert.assertThrows(DruidException.class, () -> RowKeyComparisonRunLengths.create(keyColumns, signature));
}

@Test
public void testRunLengthsWithEmptyType()
{
final List<KeyColumn> keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.NONE));
final RowSignature signature1 = RowSignature.builder()
.add("a", null)
.build();
Assert.assertThrows(DruidException.class, () -> RowKeyComparisonRunLengths.create(keyColumns, signature1));

final RowSignature signature2 = RowSignature.builder()
.add("a", ColumnType.UNKNOWN_COMPLEX)
.build();
Assert.assertThrows(DruidException.class, () -> RowKeyComparisonRunLengths.create(keyColumns, signature2));
}

@Test
public void testRunLengthsWithByteComparableTypes()
{
final List<KeyColumn> keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.ASCENDING));
final List<ColumnType> byteComparableTypes = ImmutableList.of(
ColumnType.LONG,
ColumnType.FLOAT,
ColumnType.DOUBLE,
ColumnType.STRING,
ColumnType.LONG_ARRAY,
ColumnType.FLOAT_ARRAY,
ColumnType.DOUBLE_ARRAY,
ColumnType.STRING_ARRAY
);

for (final ColumnType columnType : byteComparableTypes) {
final RowSignature signature = RowSignature.builder()
.add("a", columnType)
.build();
final RowKeyComparisonRunLengths runLengths = RowKeyComparisonRunLengths.create(keyColumns, signature);
Assert.assertEquals(1, runLengths.getRunLengthEntries().size());
Assert.assertTrue(runLengths.getRunLengthEntries().get(0).isByteComparable());
Assert.assertEquals(1, runLengths.getRunLengthEntries().get(0).getRunLength());
Assert.assertEquals(KeyOrder.ASCENDING, runLengths.getRunLengthEntries().get(0).getOrder());
}
}

@Test
public void testRunLengthsWithNonByteComparableTypes()
{
final List<KeyColumn> keyColumns = Collections.singletonList(new KeyColumn("a", KeyOrder.ASCENDING));
// Any known complex type
final List<ColumnType> byteComparableTypes = ImmutableList.of(ColumnType.NESTED_DATA);

for (final ColumnType columnType : byteComparableTypes) {
final RowSignature signature = RowSignature.builder()
.add("a", columnType)
.build();
final RowKeyComparisonRunLengths runLengths = RowKeyComparisonRunLengths.create(keyColumns, signature);
Assert.assertEquals(1, runLengths.getRunLengthEntries().size());
Assert.assertFalse(runLengths.getRunLengthEntries().get(0).isByteComparable());
Assert.assertEquals(1, runLengths.getRunLengthEntries().get(0).getRunLength());
Assert.assertEquals(KeyOrder.ASCENDING, runLengths.getRunLengthEntries().get(0).getOrder());
}
}

@Test
public void testRunLengthsWithMultipleColumns()
{
//
}

}
Loading