fix NPE in StringGroupByColumnSelectorStrategy#bufferComparator (#10325)

* fix NPE in StringGroupByColumnSelectorStrategy#bufferComparator

* Add tests

* javadocs
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java
index 4e91360..5825009 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorStrategy.java
@@ -143,7 +143,9 @@
   /**
    * 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);
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java
index 78bfdc6..a128d10 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategy.java
@@ -157,8 +157,8 @@
         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 @@
       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);
       };
     }
   }
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategyTest.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategyTest.java
new file mode 100644
index 0000000..86524bb
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/column/StringGroupByColumnSelectorStrategyTest.java
@@ -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();
+  }
+}