Add a way to retrieve UTF-8 bytes directly via DimensionDictionarySelector. (#11172)

* Add a way to retrieve UTF-8 bytes directly via DimensionDictionarySelector.

The idea is that certain operations (like count distinct on strings) will
be faster if they are able to run directly on UTF-8 bytes instead of on
Java Strings decoded by "lookupName".

* Add license header.

* Updates suggested by robots.
diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionDictionarySelector.java b/processing/src/main/java/org/apache/druid/segment/DimensionDictionarySelector.java
index 10cb466..40d8771 100644
--- a/processing/src/main/java/org/apache/druid/segment/DimensionDictionarySelector.java
+++ b/processing/src/main/java/org/apache/druid/segment/DimensionDictionarySelector.java
@@ -22,6 +22,7 @@
 import org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop;
 
 import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
 
 /**
  * Interface containing dictionary-related methods common to {@link DimensionSelector},
@@ -57,8 +58,9 @@
   int getValueCardinality();
 
   /**
-   * The Name is the String name of the actual field.  It is assumed that storage layers convert names
-   * into id values which can then be used to get the string value.  For example
+   * Returns the value for a particular dictionary id as a Java String.
+   *
+   * For example, if a column has four rows:
    *
    * A,B
    * A
@@ -77,15 +79,55 @@
    * lookupName(0) => A
    * lookupName(1) => B
    *
-   * @param id id to lookup the field name for
+   * Performance note: if you want a {@code java.lang.String}, always use this method. It will be at least as fast
+   * as calling {@link #lookupNameUtf8} and decoding the bytes. However, if you want UTF-8 bytes, then check if
+   * {@link #supportsLookupNameUtf8()} returns true, and if it does, use {@link #lookupNameUtf8} instead.
    *
-   * @return the field name for the given id
+   * @param id id to lookup the dictionary value for
+   *
+   * @return dictionary value for the given id, or null if the value is itself null
    */
   @CalledFromHotLoop
   @Nullable
   String lookupName(int id);
 
   /**
+   * Returns the value for a particular dictionary id as UTF-8 bytes.
+   *
+   * The returned buffer is in big-endian order. It is not reused, so callers may modify the position, limit, byte
+   * order, etc of the buffer.
+   *
+   * The returned buffer may point to the original data, so callers must take care not to use it outside the valid
+   * lifetime of this selector. In particular, if the original data came from a reference-counted segment, callers must
+   * not use the returned ByteBuffer after releasing their reference to the relevant {@link ReferenceCountingSegment}.
+   *
+   * Performance note: if you want UTF-8 bytes, and {@link #supportsLookupNameUtf8()} returns true, always use this
+   * method. It will be at least as fast as calling {@link #lookupName} and encoding the bytes. However, if you want a
+   * {@code java.lang.String}, then use {@link #lookupName} instead of this method.
+   *
+   * @param id id to lookup the dictionary value for
+   *
+   * @return dictionary value for the given id, or null if the value is itself null
+   *
+   * @throws UnsupportedOperationException if {@link #supportsLookupNameUtf8()} is false
+   */
+  @Nullable
+  default ByteBuffer lookupNameUtf8(int id)
+  {
+    // If UTF-8 isn't faster, it's better to throw an exception rather than delegate to "lookupName" and do the
+    // conversion. Callers should check "supportsLookupNameUtf8" to make sure they're calling the fastest method.
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Returns whether this selector supports {@link #lookupNameUtf8}.
+   */
+  default boolean supportsLookupNameUtf8()
+  {
+    return false;
+  }
+
+  /**
    * Returns true if it is possible to {@link #lookupName(int)} by ids from 0 to {@link #getValueCardinality()}
    * before the rows with those ids are returned.
    *
diff --git a/processing/src/main/java/org/apache/druid/segment/IndexIO.java b/processing/src/main/java/org/apache/druid/segment/IndexIO.java
index df43f63..dd4425b 100644
--- a/processing/src/main/java/org/apache/druid/segment/IndexIO.java
+++ b/processing/src/main/java/org/apache/druid/segment/IndexIO.java
@@ -356,6 +356,7 @@
       }
 
       Map<String, GenericIndexed<String>> dimValueLookups = new HashMap<>();
+      Map<String, GenericIndexed<ByteBuffer>> dimValueUtf8Lookups = new HashMap<>();
       Map<String, VSizeColumnarMultiInts> dimColumns = new HashMap<>();
       Map<String, GenericIndexed<ImmutableBitmap>> bitmaps = new HashMap<>();
 
@@ -369,7 +370,9 @@
             fileDimensionName
         );
 
-        dimValueLookups.put(dimension, GenericIndexed.read(dimBuffer, GenericIndexed.STRING_STRATEGY));
+        // Duplicate the first buffer since we are reading the dictionary twice.
+        dimValueLookups.put(dimension, GenericIndexed.read(dimBuffer.duplicate(), GenericIndexed.STRING_STRATEGY));
+        dimValueUtf8Lookups.put(dimension, GenericIndexed.read(dimBuffer, GenericIndexed.BYTE_BUFFER_STRATEGY));
         dimColumns.put(dimension, VSizeColumnarMultiInts.readFromByteBuffer(dimBuffer));
       }
 
@@ -399,6 +402,7 @@
           timestamps,
           metrics,
           dimValueLookups,
+          dimValueUtf8Lookups,
           dimColumns,
           bitmaps,
           spatialIndexed,
@@ -441,6 +445,7 @@
             .setDictionaryEncodedColumnSupplier(
                 new DictionaryEncodedColumnSupplier(
                     index.getDimValueLookup(dimension),
+                    index.getDimValueUtf8Lookup(dimension),
                     null,
                     Suppliers.ofInstance(index.getDimColumn(dimension)),
                     columnConfig.columnCacheSizeBytes()
diff --git a/processing/src/main/java/org/apache/druid/segment/MMappedIndex.java b/processing/src/main/java/org/apache/druid/segment/MMappedIndex.java
index 1496842..737a2be 100644
--- a/processing/src/main/java/org/apache/druid/segment/MMappedIndex.java
+++ b/processing/src/main/java/org/apache/druid/segment/MMappedIndex.java
@@ -29,6 +29,7 @@
 
 import javax.annotation.Nullable;
 
+import java.nio.ByteBuffer;
 import java.util.Map;
 
 /**
@@ -42,6 +43,7 @@
   final CompressedColumnarLongsSupplier timestamps;
   final Map<String, MetricHolder> metrics;
   final Map<String, GenericIndexed<String>> dimValueLookups;
+  final Map<String, GenericIndexed<ByteBuffer>> dimValueUtf8Lookups;
   final Map<String, VSizeColumnarMultiInts> dimColumns;
   final Map<String, GenericIndexed<ImmutableBitmap>> invertedIndexes;
   final Map<String, ImmutableRTree> spatialIndexes;
@@ -54,6 +56,7 @@
       CompressedColumnarLongsSupplier timestamps,
       Map<String, MetricHolder> metrics,
       Map<String, GenericIndexed<String>> dimValueLookups,
+      Map<String, GenericIndexed<ByteBuffer>> dimValueUtf8Lookups,
       Map<String, VSizeColumnarMultiInts> dimColumns,
       Map<String, GenericIndexed<ImmutableBitmap>> invertedIndexes,
       Map<String, ImmutableRTree> spatialIndexes,
@@ -66,6 +69,7 @@
     this.timestamps = timestamps;
     this.metrics = metrics;
     this.dimValueLookups = dimValueLookups;
+    this.dimValueUtf8Lookups = dimValueUtf8Lookups;
     this.dimColumns = dimColumns;
     this.invertedIndexes = invertedIndexes;
     this.spatialIndexes = spatialIndexes;
@@ -98,6 +102,11 @@
     return dimValueLookups.get(dimension);
   }
 
+  public GenericIndexed<ByteBuffer> getDimValueUtf8Lookup(String dimension)
+  {
+    return dimValueUtf8Lookups.get(dimension);
+  }
+
   public VSizeColumnarMultiInts getDimColumn(String dimension)
   {
     return dimColumns.get(dimension);
diff --git a/processing/src/main/java/org/apache/druid/segment/column/DictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/column/DictionaryEncodedColumn.java
index 229d7e3..2a6d71a 100644
--- a/processing/src/main/java/org/apache/druid/segment/column/DictionaryEncodedColumn.java
+++ b/processing/src/main/java/org/apache/druid/segment/column/DictionaryEncodedColumn.java
@@ -31,6 +31,7 @@
 import javax.annotation.Nullable;
 
 /**
+ *
  */
 public interface DictionaryEncodedColumn<ActualType extends Comparable<? super ActualType>> extends BaseColumn
 {
@@ -42,6 +43,7 @@
 
   IndexedInts getMultiValueRow(int rowNum);
 
+  @Nullable
   ActualType lookupName(int id);
 
   int lookupId(ActualType name);
diff --git a/processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java
index e42fa3b..5b56476 100644
--- a/processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java
+++ b/processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java
@@ -31,6 +31,7 @@
 import org.apache.druid.segment.data.CachingIndexed;
 import org.apache.druid.segment.data.ColumnarInts;
 import org.apache.druid.segment.data.ColumnarMultiInts;
+import org.apache.druid.segment.data.Indexed;
 import org.apache.druid.segment.data.IndexedInts;
 import org.apache.druid.segment.data.ReadableOffset;
 import org.apache.druid.segment.data.SingleIndexedInt;
@@ -45,9 +46,11 @@
 
 import javax.annotation.Nullable;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.BitSet;
 
 /**
+ *
  */
 public class StringDictionaryEncodedColumn implements DictionaryEncodedColumn<String>
 {
@@ -55,17 +58,20 @@
   private final ColumnarInts column;
   @Nullable
   private final ColumnarMultiInts multiValueColumn;
-  private final CachingIndexed<String> cachedLookups;
+  private final CachingIndexed<String> cachedDictionary;
+  private final Indexed<ByteBuffer> dictionaryUtf8;
 
   public StringDictionaryEncodedColumn(
       @Nullable ColumnarInts singleValueColumn,
       @Nullable ColumnarMultiInts multiValueColumn,
-      CachingIndexed<String> cachedLookups
+      CachingIndexed<String> dictionary,
+      Indexed<ByteBuffer> dictionaryUtf8
   )
   {
     this.column = singleValueColumn;
     this.multiValueColumn = multiValueColumn;
-    this.cachedLookups = cachedLookups;
+    this.cachedDictionary = dictionary;
+    this.dictionaryUtf8 = dictionaryUtf8;
   }
 
   @Override
@@ -96,19 +102,39 @@
   @Nullable
   public String lookupName(int id)
   {
-    return cachedLookups.get(id);
+    return cachedDictionary.get(id);
+  }
+
+
+  /**
+   * Returns the value for a particular dictionary id as UTF-8 bytes.
+   *
+   * The returned buffer is in big-endian order. It is not reused, so callers may modify the position, limit, byte
+   * order, etc of the buffer.
+   *
+   * The returned buffer points to the original data, so callers must take care not to use it outside the valid
+   * lifetime of this column.
+   *
+   * @param id id to lookup the dictionary value for
+   *
+   * @return dictionary value for the given id, or null if the value is itself null
+   */
+  @Nullable
+  public ByteBuffer lookupNameUtf8(int id)
+  {
+    return dictionaryUtf8.get(id);
   }
 
   @Override
   public int lookupId(String name)
   {
-    return cachedLookups.indexOf(name);
+    return cachedDictionary.indexOf(name);
   }
 
   @Override
   public int getCardinality()
   {
-    return cachedLookups.size();
+    return cachedDictionary.size();
   }
 
   @Override
@@ -141,6 +167,19 @@
         return extractionFn == null ? value : extractionFn.apply(value);
       }
 
+      @Nullable
+      @Override
+      public ByteBuffer lookupNameUtf8(int id)
+      {
+        return StringDictionaryEncodedColumn.this.lookupNameUtf8(id);
+      }
+
+      @Override
+      public boolean supportsLookupNameUtf8()
+      {
+        return true;
+      }
+
       @Override
       public boolean nameLookupPossibleInAdvance()
       {
@@ -369,6 +408,19 @@
         return StringDictionaryEncodedColumn.this.lookupName(id);
       }
 
+      @Nullable
+      @Override
+      public ByteBuffer lookupNameUtf8(int id)
+      {
+        return StringDictionaryEncodedColumn.this.lookupNameUtf8(id);
+      }
+
+      @Override
+      public boolean supportsLookupNameUtf8()
+      {
+        return true;
+      }
+
       @Override
       public boolean nameLookupPossibleInAdvance()
       {
@@ -454,6 +506,19 @@
         return StringDictionaryEncodedColumn.this.lookupName(id);
       }
 
+      @Nullable
+      @Override
+      public ByteBuffer lookupNameUtf8(int id)
+      {
+        return StringDictionaryEncodedColumn.this.lookupNameUtf8(id);
+      }
+
+      @Override
+      public boolean supportsLookupNameUtf8()
+      {
+        return true;
+      }
+
       @Override
       public boolean nameLookupPossibleInAdvance()
       {
@@ -542,7 +607,7 @@
   @Override
   public void close() throws IOException
   {
-    CloseQuietly.close(cachedLookups);
+    CloseQuietly.close(cachedDictionary);
 
     if (column != null) {
       column.close();
diff --git a/processing/src/main/java/org/apache/druid/segment/data/GenericIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/GenericIndexed.java
index 35c9a3f..74e1d76 100644
--- a/processing/src/main/java/org/apache/druid/segment/data/GenericIndexed.java
+++ b/processing/src/main/java/org/apache/druid/segment/data/GenericIndexed.java
@@ -93,6 +93,51 @@
 
   private static final SerializerUtils SERIALIZER_UTILS = new SerializerUtils();
 
+  /**
+   * An ObjectStrategy that returns a big-endian ByteBuffer pointing to the original data.
+   *
+   * The returned ByteBuffer is a fresh read-only instance, so it is OK for callers to modify its position, limit, etc.
+   * However, it does point to the original data, so callers must take care not to use it if the original data may
+   * have been freed.
+   */
+  public static final ObjectStrategy<ByteBuffer> BYTE_BUFFER_STRATEGY = new ObjectStrategy<ByteBuffer>()
+  {
+    @Override
+    public Class<ByteBuffer> getClazz()
+    {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public ByteBuffer fromByteBuffer(final ByteBuffer buffer, final int numBytes)
+    {
+      final ByteBuffer dup = buffer.asReadOnlyBuffer();
+      dup.limit(buffer.position() + numBytes);
+      return dup;
+    }
+
+    @Override
+    @Nullable
+    public byte[] toBytes(@Nullable ByteBuffer buf)
+    {
+      if (buf == null) {
+        return null;
+      }
+
+      // This method doesn't have javadocs and I'm not sure if it is OK to modify the "val" argument. Copy defensively.
+      final ByteBuffer dup = buf.duplicate();
+      final byte[] bytes = new byte[dup.remaining()];
+      dup.get(bytes);
+      return bytes;
+    }
+
+    @Override
+    public int compare(ByteBuffer o1, ByteBuffer o2)
+    {
+      return o1.compareTo(o2);
+    }
+  };
+
   public static final ObjectStrategy<String> STRING_STRATEGY = new ObjectStrategy<String>()
   {
     @Override
diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java
index 833164b..7fdec0e 100644
--- a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java
+++ b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java
@@ -303,11 +303,19 @@
 
         final boolean hasMultipleValues = Feature.MULTI_VALUE.isSet(rFlags) || Feature.MULTI_VALUE_V3.isSet(rFlags);
 
+        // Duplicate the first buffer since we are reading the dictionary twice.
         final GenericIndexed<String> rDictionary = GenericIndexed.read(
-            buffer,
+            buffer.duplicate(),
             GenericIndexed.STRING_STRATEGY,
             builder.getFileMapper()
         );
+
+        final GenericIndexed<ByteBuffer> rDictionaryUtf8 = GenericIndexed.read(
+            buffer,
+            GenericIndexed.BYTE_BUFFER_STRATEGY,
+            builder.getFileMapper()
+        );
+
         builder.setType(ValueType.STRING);
 
         final WritableSupplier<ColumnarInts> rSingleValuedColumn;
@@ -325,6 +333,7 @@
 
         DictionaryEncodedColumnSupplier dictionaryEncodedColumnSupplier = new DictionaryEncodedColumnSupplier(
             rDictionary,
+            rDictionaryUtf8,
             rSingleValuedColumn,
             rMultiValuedColumn,
             columnConfig.columnCacheSizeBytes()
diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java
index d0fb888..f48dcfc 100644
--- a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java
+++ b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java
@@ -28,24 +28,28 @@
 import org.apache.druid.segment.data.GenericIndexed;
 
 import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
 
 /**
  */
 public class DictionaryEncodedColumnSupplier implements Supplier<DictionaryEncodedColumn<?>>
 {
   private final GenericIndexed<String> dictionary;
+  private final GenericIndexed<ByteBuffer> dictionaryUtf8;
   private final @Nullable Supplier<ColumnarInts> singleValuedColumn;
   private final @Nullable Supplier<ColumnarMultiInts> multiValuedColumn;
   private final int lookupCacheSize;
 
   public DictionaryEncodedColumnSupplier(
       GenericIndexed<String> dictionary,
+      GenericIndexed<ByteBuffer> dictionaryUtf8,
       @Nullable Supplier<ColumnarInts> singleValuedColumn,
       @Nullable Supplier<ColumnarMultiInts> multiValuedColumn,
       int lookupCacheSize
   )
   {
     this.dictionary = dictionary;
+    this.dictionaryUtf8 = dictionaryUtf8;
     this.singleValuedColumn = singleValuedColumn;
     this.multiValuedColumn = multiValuedColumn;
     this.lookupCacheSize = lookupCacheSize;
@@ -57,7 +61,8 @@
     return new StringDictionaryEncodedColumn(
         singleValuedColumn != null ? singleValuedColumn.get() : null,
         multiValuedColumn != null ? multiValuedColumn.get() : null,
-        new CachingIndexed<>(dictionary, lookupCacheSize)
+        new CachingIndexed<>(dictionary, lookupCacheSize),
+        dictionaryUtf8.singleThreaded()
     );
   }
 }
diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java
new file mode 100644
index 0000000..ebcf3a9
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java
@@ -0,0 +1,240 @@
+/*
+ * 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.segment;
+
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.guava.Yielders;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorCursor;
+import org.apache.druid.testing.InitializedNullHandlingTest;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.runners.Enclosed;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+
+@RunWith(Enclosed.class)
+public class QueryableIndexStorageAdapterTest
+{
+  @RunWith(Parameterized.class)
+  public static class DimensionDictionarySelectorTest extends InitializedNullHandlingTest
+  {
+    private final boolean vectorize;
+
+    private DimensionDictionarySelector qualitySelector;
+    private DimensionDictionarySelector placementishSelector;
+    private DimensionDictionarySelector partialNullSelector;
+
+    private Closer closer = Closer.create();
+
+    @Parameterized.Parameters(name = "vectorize = {0}")
+    public static Collection<?> constructorFeeder()
+    {
+      return Arrays.asList(new Object[]{false}, new Object[]{true});
+    }
+
+    public DimensionDictionarySelectorTest(boolean vectorize)
+    {
+      this.vectorize = vectorize;
+    }
+
+    @Before
+    public void setUp()
+    {
+      final QueryableIndex index = TestIndex.getMMappedTestIndex();
+      final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index);
+
+      if (vectorize) {
+        final VectorCursor cursor = closer.register(
+            adapter.makeVectorCursor(
+                null,
+                Intervals.ETERNITY,
+                VirtualColumns.EMPTY,
+                false,
+                QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE,
+                null
+            )
+        );
+
+        final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
+
+        qualitySelector =
+            columnSelectorFactory.makeSingleValueDimensionSelector(DefaultDimensionSpec.of("quality"));
+        placementishSelector =
+            columnSelectorFactory.makeMultiValueDimensionSelector(DefaultDimensionSpec.of("placementish"));
+        partialNullSelector =
+            columnSelectorFactory.makeSingleValueDimensionSelector(DefaultDimensionSpec.of("partial_null_column"));
+      } else {
+        final Sequence<Cursor> cursors = adapter.makeCursors(
+            null,
+            Intervals.ETERNITY,
+            VirtualColumns.EMPTY,
+            Granularities.ALL,
+            false,
+            null
+        );
+
+        final Yielder<Cursor> yielder = closer.register(Yielders.each(cursors));
+        final Cursor cursor = yielder.get();
+        final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
+
+        qualitySelector =
+            columnSelectorFactory.makeDimensionSelector(DefaultDimensionSpec.of("quality"));
+        placementishSelector =
+            columnSelectorFactory.makeDimensionSelector(DefaultDimensionSpec.of("placementish"));
+        partialNullSelector =
+            columnSelectorFactory.makeDimensionSelector(DefaultDimensionSpec.of("partial_null_column"));
+      }
+    }
+
+    @After
+    public void tearDown() throws IOException
+    {
+      closer.close();
+    }
+
+    @Test
+    public void test_getCardinality_quality()
+    {
+      Assert.assertEquals(9, qualitySelector.getValueCardinality());
+    }
+
+    @Test
+    public void test_getCardinality_placementish()
+    {
+      Assert.assertEquals(9, placementishSelector.getValueCardinality());
+    }
+
+    @Test
+    public void test_getCardinality_partialNullColumn()
+    {
+      Assert.assertEquals(2, partialNullSelector.getValueCardinality());
+    }
+
+    @Test
+    public void test_lookupName_quality()
+    {
+      Assert.assertEquals("automotive", qualitySelector.lookupName(0));
+      Assert.assertEquals("business", qualitySelector.lookupName(1));
+      Assert.assertEquals("entertainment", qualitySelector.lookupName(2));
+      Assert.assertEquals("health", qualitySelector.lookupName(3));
+      Assert.assertEquals("mezzanine", qualitySelector.lookupName(4));
+      Assert.assertEquals("news", qualitySelector.lookupName(5));
+      Assert.assertEquals("premium", qualitySelector.lookupName(6));
+      Assert.assertEquals("technology", qualitySelector.lookupName(7));
+      Assert.assertEquals("travel", qualitySelector.lookupName(8));
+    }
+
+    @Test
+    public void test_lookupName_placementish()
+    {
+      Assert.assertEquals("a", placementishSelector.lookupName(0));
+      Assert.assertEquals("b", placementishSelector.lookupName(1));
+      Assert.assertEquals("e", placementishSelector.lookupName(2));
+      Assert.assertEquals("h", placementishSelector.lookupName(3));
+      Assert.assertEquals("m", placementishSelector.lookupName(4));
+      Assert.assertEquals("n", placementishSelector.lookupName(5));
+      Assert.assertEquals("p", placementishSelector.lookupName(6));
+      Assert.assertEquals("preferred", placementishSelector.lookupName(7));
+      Assert.assertEquals("t", placementishSelector.lookupName(8));
+    }
+
+    @Test
+    public void test_lookupName_partialNull()
+    {
+      Assert.assertNull(partialNullSelector.lookupName(0));
+      Assert.assertEquals("value", partialNullSelector.lookupName(1));
+    }
+
+    @Test
+    public void test_lookupNameUtf8_quality()
+    {
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("automotive")), qualitySelector.lookupNameUtf8(0));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("business")), qualitySelector.lookupNameUtf8(1));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("entertainment")), qualitySelector.lookupNameUtf8(2));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("health")), qualitySelector.lookupNameUtf8(3));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("mezzanine")), qualitySelector.lookupNameUtf8(4));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("news")), qualitySelector.lookupNameUtf8(5));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("premium")), qualitySelector.lookupNameUtf8(6));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("technology")), qualitySelector.lookupNameUtf8(7));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("travel")), qualitySelector.lookupNameUtf8(8));
+    }
+
+    @Test
+    public void test_lookupNameUtf8_placementish()
+    {
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("a")), placementishSelector.lookupNameUtf8(0));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("b")), placementishSelector.lookupNameUtf8(1));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("e")), placementishSelector.lookupNameUtf8(2));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("h")), placementishSelector.lookupNameUtf8(3));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("m")), placementishSelector.lookupNameUtf8(4));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("n")), placementishSelector.lookupNameUtf8(5));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("p")), placementishSelector.lookupNameUtf8(6));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("preferred")), placementishSelector.lookupNameUtf8(7));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("t")), placementishSelector.lookupNameUtf8(8));
+    }
+
+    @Test
+    public void test_lookupNameUtf8_partialNull()
+    {
+      Assert.assertNull(partialNullSelector.lookupNameUtf8(0));
+      Assert.assertEquals(ByteBuffer.wrap(StringUtils.toUtf8("value")), partialNullSelector.lookupNameUtf8(1));
+    }
+
+    @Test
+    public void test_lookupNameUtf8_buffersAreNotShared()
+    {
+      // Different buffer on different calls; enables callers to safely modify position, limit as promised in
+      // the javadocs.
+      Assert.assertNotSame(qualitySelector.lookupNameUtf8(0), qualitySelector.lookupNameUtf8(0));
+    }
+
+    @Test
+    public void test_supportsLookupNameUtf8_quality()
+    {
+      Assert.assertTrue(partialNullSelector.supportsLookupNameUtf8());
+    }
+
+    @Test
+    public void test_supportsLookupNameUtf8_placementish()
+    {
+      Assert.assertTrue(partialNullSelector.supportsLookupNameUtf8());
+    }
+
+    @Test
+    public void test_supportsLookupNameUtf8_partialNull()
+    {
+      Assert.assertTrue(partialNullSelector.supportsLookupNameUtf8());
+    }
+  }
+}
diff --git a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java
index df5a35e..0b1004e 100644
--- a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java
@@ -38,6 +38,7 @@
 import org.junit.Test;
 
 import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 
 public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTest
@@ -68,6 +69,14 @@
     // Emulate multi-valued dimension
     final DictionaryEncodedColumnSupplier columnSupplier = new DictionaryEncodedColumnSupplier(
         GenericIndexed.fromIterable(ImmutableList.of("v1", "v2", "v3"), GenericIndexed.STRING_STRATEGY),
+        GenericIndexed.fromIterable(
+            ImmutableList.of(
+                ByteBuffer.wrap(StringUtils.toUtf8("v1")),
+                ByteBuffer.wrap(StringUtils.toUtf8("v2")),
+                ByteBuffer.wrap(StringUtils.toUtf8("v3"))
+            ),
+            GenericIndexed.BYTE_BUFFER_STRATEGY
+        ),
         null,
         () -> VSizeColumnarMultiInts.fromIterable(ImmutableList.of(VSizeColumnarInts.fromArray(new int[]{1}))),
         0
@@ -83,6 +92,14 @@
     // Emulate multi-valued dimension
     final DictionaryEncodedColumnSupplier columnSupplier = new DictionaryEncodedColumnSupplier(
         GenericIndexed.fromIterable(ImmutableList.of("v1", "v2", "v3"), GenericIndexed.STRING_STRATEGY),
+        GenericIndexed.fromIterable(
+            ImmutableList.of(
+                ByteBuffer.wrap(StringUtils.toUtf8("v1")),
+                ByteBuffer.wrap(StringUtils.toUtf8("v2")),
+                ByteBuffer.wrap(StringUtils.toUtf8("v3"))
+            ),
+            GenericIndexed.BYTE_BUFFER_STRATEGY
+        ),
         null,
         () -> VSizeColumnarMultiInts.fromIterable(ImmutableList.of(VSizeColumnarInts.fromArray(new int[]{1}))),
         0
diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java
index 6698c3a..cb54592 100644
--- a/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java
@@ -20,6 +20,7 @@
 package org.apache.druid.segment.filter;
 
 import com.google.common.collect.ImmutableList;
+import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.SimpleAscendingOffset;
 import org.apache.druid.segment.data.GenericIndexed;
@@ -33,6 +34,8 @@
 import org.junit.Before;
 import org.junit.Test;
 
+import java.nio.ByteBuffer;
+
 public class ValueMatchersTest extends InitializedNullHandlingTest
 {
   private DictionaryEncodedColumnSupplier supplierSingleConstant;
@@ -44,18 +47,33 @@
   {
     supplierSingleConstant = new DictionaryEncodedColumnSupplier(
         GenericIndexed.fromIterable(ImmutableList.of("value"), GenericIndexed.STRING_STRATEGY),
+        GenericIndexed.fromIterable(
+            ImmutableList.of(ByteBuffer.wrap(StringUtils.toUtf8("value"))),
+            GenericIndexed.BYTE_BUFFER_STRATEGY
+        ),
         () -> VSizeColumnarInts.fromArray(new int[]{0}),
         null,
         0
     );
     supplierSingle = new DictionaryEncodedColumnSupplier(
         GenericIndexed.fromIterable(ImmutableList.of("value", "value2"), GenericIndexed.STRING_STRATEGY),
+        GenericIndexed.fromIterable(
+            ImmutableList.of(
+                ByteBuffer.wrap(StringUtils.toUtf8("value")),
+                ByteBuffer.wrap(StringUtils.toUtf8("value2"))
+            ),
+            GenericIndexed.BYTE_BUFFER_STRATEGY
+        ),
         () -> VSizeColumnarInts.fromArray(new int[]{0, 0, 1, 0, 1}),
         null,
         0
     );
     supplierMulti = new DictionaryEncodedColumnSupplier(
         GenericIndexed.fromIterable(ImmutableList.of("value"), GenericIndexed.STRING_STRATEGY),
+        GenericIndexed.fromIterable(
+            ImmutableList.of(ByteBuffer.wrap(StringUtils.toUtf8("value"))),
+            GenericIndexed.BYTE_BUFFER_STRATEGY
+        ),
         null,
         () -> VSizeColumnarMultiInts.fromIterable(
             ImmutableList.of(