fix NPE in StringGroupByColumnSelectorStrategy#bufferComparator (#10325)

* fix NPE in StringGroupByColumnSelectorStrategy#bufferComparator

* Add tests

* javadocs
This commit is contained in:
Suneet Saldanha 2020-09-04 13:23:40 -07:00 committed by GitHub
parent d7fcff3aba
commit 91a153820e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 115 additions and 4 deletions

View File

@ -143,7 +143,9 @@ public interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy
/**
* Return BufferComparator for values written using this strategy when limit is pushed down to segment scan.
* @param keyBufferPosition starting offset for this column's value within the grouping key
* @param stringComparator stringComparator from LimitSpec for this column
* @param stringComparator stringComparator from LimitSpec for this column. If this is null, implementations
* will use the {@link org.apache.druid.query.ordering.StringComparators#LEXICOGRAPHIC}
* comparator.
* @return BufferComparator for comparing values written
*/
Grouper.BufferComparator bufferComparator(int keyBufferPosition, @Nullable StringComparator stringComparator);

View File

@ -157,8 +157,8 @@ public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelecto
capabilities != null &&
capabilities.hasBitmapIndexes() &&
capabilities.areDictionaryValuesSorted().and(capabilities.areDictionaryValuesUnique()).isTrue();
if (canCompareInts && (stringComparator == null || StringComparators.LEXICOGRAPHIC.equals(stringComparator))) {
final StringComparator comparator = stringComparator == null ? StringComparators.LEXICOGRAPHIC : stringComparator;
if (canCompareInts && StringComparators.LEXICOGRAPHIC.equals(comparator)) {
return (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> Integer.compare(
lhsBuffer.getInt(lhsPosition + keyBufferPosition),
rhsBuffer.getInt(rhsPosition + keyBufferPosition)
@ -168,7 +168,7 @@ public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelecto
return (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> {
String lhsStr = dictionaryLookup.apply(lhsBuffer.getInt(lhsPosition + keyBufferPosition));
String rhsStr = dictionaryLookup.apply(rhsBuffer.getInt(rhsPosition + keyBufferPosition));
return stringComparator.compare(lhsStr, rhsStr);
return comparator.compare(lhsStr, rhsStr);
};
}
}

View File

@ -0,0 +1,109 @@
/*
* 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.query.groupby.epinephelinae.column;
import it.unimi.dsi.fastutil.ints.Int2ObjectArrayMap;
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
import org.apache.druid.query.groupby.epinephelinae.Grouper;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import java.nio.ByteBuffer;
import java.util.function.IntFunction;
@RunWith(MockitoJUnitRunner.class)
public class StringGroupByColumnSelectorStrategyTest
{
// The dictionary has been constructed such that the values are not sorted lexicographically
// so we can tell when the comparator uses a lexicographic comparison and when it uses the indexes.
private static final Int2ObjectMap<String> DICTIONARY = new Int2ObjectArrayMap<>(
new int[] {0, 1, 2},
new String[] {"A", "F1", "D"}
);
private final ByteBuffer lhsBuffer = ByteBuffer.allocate(4);
private final ByteBuffer rhsBuffer = ByteBuffer.allocate(4);
@Mock
private ColumnCapabilities capabilities;
private final IntFunction<String> dictionaryLookup = DICTIONARY::get;
private StringGroupByColumnSelectorStrategy target;
@Before
public void setUp()
{
lhsBuffer.putInt(1);
rhsBuffer.putInt(2);
Mockito.doReturn(true).when(capabilities).hasBitmapIndexes();
Mockito.doReturn(ColumnCapabilities.Capable.TRUE).when(capabilities).areDictionaryValuesSorted();
Mockito.doReturn(ColumnCapabilities.Capable.TRUE).when(capabilities).areDictionaryValuesUnique();
target = new StringGroupByColumnSelectorStrategy(dictionaryLookup, capabilities);
}
@Test
public void testBufferComparatorCannotCompareIntsAndNullStringComparatorShouldUseLexicographicComparator()
{
Mockito.when(capabilities.areDictionaryValuesSorted()).thenReturn(ColumnCapabilities.Capable.FALSE);
// The comparator is not using the short circuit so it isn't comparing indexes.
Grouper.BufferComparator comparator = target.bufferComparator(0, null);
Assert.assertTrue(comparator.compare(lhsBuffer, rhsBuffer, 0, 0) > 0);
Assert.assertTrue(comparator.compare(rhsBuffer, lhsBuffer, 0, 0) < 0);
}
@Test
public void testBufferComparatorCanCompareIntsAndNullStringComparatorShouldUseLexicographicComparator()
{
Grouper.BufferComparator comparator = target.bufferComparator(0, null);
Assert.assertTrue(comparator.compare(lhsBuffer, rhsBuffer, 0, 0) < 0);
Assert.assertTrue(comparator.compare(rhsBuffer, lhsBuffer, 0, 0) > 0);
}
@Test
public void testBufferComparatorCanCompareIntsAndLexicographicStringComparatorShouldUseLexicographicComparator()
{
Grouper.BufferComparator comparator = target.bufferComparator(0, StringComparators.LEXICOGRAPHIC);
Assert.assertTrue(comparator.compare(lhsBuffer, rhsBuffer, 0, 0) < 0);
Assert.assertTrue(comparator.compare(rhsBuffer, lhsBuffer, 0, 0) > 0);
}
@Test
public void testBufferComparatorCanCompareIntsAndStrLenStringComparatorShouldUseLexicographicComparator()
{
Grouper.BufferComparator comparator = target.bufferComparator(0, StringComparators.STRLEN);
Assert.assertTrue(comparator.compare(lhsBuffer, rhsBuffer, 0, 0) > 0);
Assert.assertTrue(comparator.compare(rhsBuffer, lhsBuffer, 0, 0) < 0);
}
@After
public void tearDown()
{
lhsBuffer.clear();
rhsBuffer.clear();
}
}