You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by em...@apache.org on 2020/08/15 04:27:16 UTC

[arrow] branch master updated: ARROW-9377: [Java] Support unsigned dictionary indices

This is an automated email from the ASF dual-hosted git repository.

emkornfield pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 0d0a0cf  ARROW-9377: [Java] Support unsigned dictionary indices
0d0a0cf is described below

commit 0d0a0cfef367d2aefc0596b0f627ba2b594f7306
Author: liyafan82 <fa...@foxmail.com>
AuthorDate: Fri Aug 14 21:26:47 2020 -0700

    ARROW-9377: [Java] Support unsigned dictionary indices
    
    See https://issues.apache.org/jira/browse/ARROW-9377
    
    Closes #7817 from liyafan82/fly_0721_uns
    
    Authored-by: liyafan82 <fa...@foxmail.com>
    Signed-off-by: Micah Kornfield <em...@gmail.com>
---
 .../java/org/apache/arrow/vector/UInt1Vector.java  |  14 +-
 .../java/org/apache/arrow/vector/UInt2Vector.java  |   6 +
 .../java/org/apache/arrow/vector/UInt4Vector.java  |  15 +-
 .../java/org/apache/arrow/vector/UInt8Vector.java  |   6 +
 .../apache/arrow/vector/TestDictionaryVector.java  | 104 +++++++++
 .../org/apache/arrow/vector/TestValueVector.java   |  43 ++++
 .../vector/ipc/TestUIntDictionaryRoundTrip.java    | 246 +++++++++++++++++++++
 7 files changed, 430 insertions(+), 4 deletions(-)

diff --git a/java/vector/src/main/java/org/apache/arrow/vector/UInt1Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/UInt1Vector.java
index f91d171..4a2e5b1 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/UInt1Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt1Vector.java
@@ -36,6 +36,16 @@ import org.apache.arrow.vector.util.TransferPair;
  * maintained to track which elements in the vector are null.
  */
 public final class UInt1Vector extends BaseFixedWidthVector implements BaseIntVector {
+  /**
+   * The mask to use when promoting the unsigned byte value to an integer.
+   */
+  public static final int PROMOTION_MASK = 0xFF;
+
+  /**
+   * The maximum 8-bit unsigned integer.
+   */
+  public static final byte MAX_UINT1 = (byte) 0XFF;
+
   private static final byte TYPE_WIDTH = 1;
   private final FieldReader reader;
 
@@ -83,7 +93,7 @@ public final class UInt1Vector extends BaseFixedWidthVector implements BaseIntVe
    */
   public static short getNoOverflow(final ArrowBuf buffer, final int index) {
     byte b = buffer.getByte(index * TYPE_WIDTH);
-    return (short) (0xFF & b);
+    return (short) (PROMOTION_MASK & b);
   }
 
 
@@ -315,7 +325,7 @@ public final class UInt1Vector extends BaseFixedWidthVector implements BaseIntVe
 
   @Override
   public long getValueAsLong(int index) {
-    return this.get(index);
+    return this.get(index) & PROMOTION_MASK;
   }
 
 
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/UInt2Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/UInt2Vector.java
index de024ff..660194b 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/UInt2Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt2Vector.java
@@ -36,6 +36,12 @@ import org.apache.arrow.vector.util.TransferPair;
  * maintained to track which elements in the vector are null.
  */
 public final class UInt2Vector extends BaseFixedWidthVector implements BaseIntVector {
+
+  /**
+   * The maximum 16-bit unsigned integer.
+   */
+  public static final char MAX_UINT2 = (char) 0XFFFF;
+
   private static final byte TYPE_WIDTH = 2;
   private final FieldReader reader;
 
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/UInt4Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/UInt4Vector.java
index d749478..8166dfd 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/UInt4Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt4Vector.java
@@ -36,6 +36,17 @@ import org.apache.arrow.vector.util.TransferPair;
  * maintained to track which elements in the vector are null.
  */
 public final class UInt4Vector extends BaseFixedWidthVector implements BaseIntVector {
+
+  /**
+   * The mask to use when promoting the unsigned int value to a long int.
+   */
+  public static final long PROMOTION_MASK = 0x00000000FFFFFFFFL;
+
+  /**
+   * The maximum 32-bit unsigned integer.
+   */
+  public static final int MAX_UINT4 = 0XFFFFFFFF;
+
   private static final byte TYPE_WIDTH = 4;
   private final FieldReader reader;
 
@@ -83,7 +94,7 @@ public final class UInt4Vector extends BaseFixedWidthVector implements BaseIntVe
    */
   public static long getNoOverflow(final ArrowBuf buffer, final int index) {
     long l = buffer.getInt((long) index * TYPE_WIDTH);
-    return (0x00000000FFFFFFFFL) & l;
+    return PROMOTION_MASK & l;
   }
 
   /**
@@ -286,7 +297,7 @@ public final class UInt4Vector extends BaseFixedWidthVector implements BaseIntVe
 
   @Override
   public long getValueAsLong(int index) {
-    return this.get(index);
+    return this.get(index) & PROMOTION_MASK;
   }
 
   private class TransferImpl implements TransferPair {
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/UInt8Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/UInt8Vector.java
index c04af54..48f8c50 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/UInt8Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt8Vector.java
@@ -38,6 +38,12 @@ import org.apache.arrow.vector.util.TransferPair;
  * maintained to track which elements in the vector are null.
  */
 public final class UInt8Vector extends BaseFixedWidthVector implements BaseIntVector {
+
+  /**
+   * The maximum 64-bit unsigned long integer.
+   */
+  public static final long MAX_UINT8 = 0XFFFFFFFFFFFFFFFFL;
+
   private static final byte TYPE_WIDTH = 8;
   private final FieldReader reader;
 
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java
index 812ee38..e8fc444 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java
@@ -20,6 +20,7 @@ package org.apache.arrow.vector;
 import static org.apache.arrow.vector.TestUtils.newVarBinaryVector;
 import static org.apache.arrow.vector.TestUtils.newVarCharVector;
 import static org.apache.arrow.vector.testing.ValueVectorDataPopulator.setVector;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -29,6 +30,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.function.ToIntBiFunction;
 
 import org.apache.arrow.memory.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
@@ -878,6 +880,108 @@ public class TestDictionaryVector {
     }
   }
 
+  private void testDictionary(Dictionary dictionary, ToIntBiFunction<ValueVector, Integer> valGetter) {
+    try (VarCharVector vector = new VarCharVector("vector", allocator)) {
+      setVector(vector, "1", "3", "5", "7", "9");
+      try (ValueVector encodedVector = DictionaryEncoder.encode(vector, dictionary)) {
+
+        // verify encoded result
+        assertEquals(vector.getValueCount(), encodedVector.getValueCount());
+        assertEquals(valGetter.applyAsInt(encodedVector, 0), 1);
+        assertEquals(valGetter.applyAsInt(encodedVector, 1), 3);
+        assertEquals(valGetter.applyAsInt(encodedVector, 2), 5);
+        assertEquals(valGetter.applyAsInt(encodedVector, 3), 7);
+        assertEquals(valGetter.applyAsInt(encodedVector, 4), 9);
+
+        try (ValueVector decodedVector = DictionaryEncoder.decode(encodedVector, dictionary)) {
+          assertTrue(decodedVector instanceof VarCharVector);
+          assertEquals(vector.getValueCount(), decodedVector.getValueCount());
+          assertArrayEquals("1".getBytes(), ((VarCharVector) decodedVector).get(0));
+          assertArrayEquals("3".getBytes(), ((VarCharVector) decodedVector).get(1));
+          assertArrayEquals("5".getBytes(), ((VarCharVector) decodedVector).get(2));
+          assertArrayEquals("7".getBytes(), ((VarCharVector) decodedVector).get(3));
+          assertArrayEquals("9".getBytes(), ((VarCharVector) decodedVector).get(4));
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testDictionaryUInt1() {
+    try (VarCharVector dictionaryVector = new VarCharVector("dict vector", allocator)) {
+      setVector(dictionaryVector, "0", "1", "2", "3", "4", "5", "6", "7", "8", "9");
+      Dictionary dictionary1 = new Dictionary(dictionaryVector,
+          new DictionaryEncoding(/*id=*/10L, /*ordered=*/false,
+              /*indexType=*/new ArrowType.Int(/*bitWidth*/8, /*isSigned*/false)));
+      testDictionary(dictionary1, (vector, index) -> ((UInt1Vector) vector).get(index));
+    }
+  }
+
+  @Test
+  public void testDictionaryUInt2() {
+    try (VarCharVector dictionaryVector = new VarCharVector("dict vector", allocator)) {
+      setVector(dictionaryVector, "0", "1", "2", "3", "4", "5", "6", "7", "8", "9");
+      Dictionary dictionary2 = new Dictionary(dictionaryVector,
+          new DictionaryEncoding(/*id=*/20L, /*ordered=*/false,
+              /*indexType=*/new ArrowType.Int(/*indexType=*/16, /*isSigned*/false)));
+      testDictionary(dictionary2, (vector, index) -> ((UInt2Vector) vector).get(index));
+    }
+  }
+
+  @Test
+  public void testDictionaryUInt4() {
+    try (VarCharVector dictionaryVector = new VarCharVector("dict vector", allocator)) {
+      setVector(dictionaryVector, "0", "1", "2", "3", "4", "5", "6", "7", "8", "9");
+      Dictionary dictionary4 = new Dictionary(dictionaryVector,
+          new DictionaryEncoding(/*id=*/30L, /*ordered=*/false,
+              /*indexType=*/new ArrowType.Int(/*indexType=*/32, /*isSigned*/false)));
+      testDictionary(dictionary4, (vector, index) -> ((UInt4Vector) vector).get(index));
+    }
+  }
+
+  @Test
+  public void testDictionaryUInt8() {
+    try (VarCharVector dictionaryVector = new VarCharVector("dict vector", allocator)) {
+      setVector(dictionaryVector, "0", "1", "2", "3", "4", "5", "6", "7", "8", "9");
+      Dictionary dictionary8 = new Dictionary(dictionaryVector,
+              new DictionaryEncoding(/*id=*/40L, /*ordered=*/false,
+                  /*indexType=*/new ArrowType.Int(/*indexType=*/64, /*isSigned*/false)));
+      testDictionary(dictionary8, (vector, index) -> (int) ((UInt8Vector) vector).get(index));
+    }
+  }
+
+  @Test
+  public void testDictionaryUIntOverflow() {
+    // the size is within the range of UInt1, but outside the range of TinyInt.
+    final int vecLength = 256;
+    try (VarCharVector dictionaryVector = new VarCharVector("dict vector", allocator)) {
+      dictionaryVector.allocateNew(vecLength * 3, vecLength);
+      for (int i = 0; i < vecLength; i++) {
+        dictionaryVector.set(i, String.valueOf(i).getBytes());
+      }
+      dictionaryVector.setValueCount(vecLength);
+
+      Dictionary dictionary = new Dictionary(dictionaryVector,
+          new DictionaryEncoding(/*id=*/10L, /*ordered=*/false,
+              /*indexType=*/new ArrowType.Int(/*indexType=*/8, /*isSigned*/false)));
+
+      try (VarCharVector vector = new VarCharVector("vector", allocator)) {
+        setVector(vector, "255");
+        try (UInt1Vector encodedVector = (UInt1Vector) DictionaryEncoder.encode(vector, dictionary)) {
+
+          // verify encoded result
+          assertEquals(1, encodedVector.getValueCount());
+          assertEquals(255, encodedVector.getValueAsLong(0));
+
+          try (VarCharVector decodedVector = (VarCharVector) DictionaryEncoder.decode(encodedVector, dictionary)) {
+            assertEquals(1, decodedVector.getValueCount());
+            assertArrayEquals("255".getBytes(), decodedVector.get(0));
+          }
+        }
+      }
+    }
+  }
+
   private int[] convertListToIntArray(JsonStringArrayList list) {
     int[] values = new int[list.size()];
     for (int i = 0; i < list.size(); i++) {
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
index 0cd9e80..0be5e1e 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
@@ -2977,4 +2977,47 @@ public class TestValueVector {
       assertEquals(0, vector.getOffsetBuffer().capacity());
     }
   }
+
+  @Test
+  public void testSetGetUInt1() {
+    try (UInt1Vector vector = new UInt1Vector("vector", allocator)) {
+      vector.allocateNew(2);
+
+      vector.setWithPossibleTruncate(0, UInt1Vector.MAX_UINT1);
+      vector.setUnsafeWithPossibleTruncate(1, UInt1Vector.MAX_UINT1);
+      vector.setValueCount(2);
+
+      assertEquals(UInt1Vector.MAX_UINT1 & UInt1Vector.PROMOTION_MASK, vector.getValueAsLong(0));
+      assertEquals(UInt1Vector.MAX_UINT1 & UInt1Vector.PROMOTION_MASK, vector.getValueAsLong(1));
+    }
+  }
+
+  @Test
+  public void testSetGetUInt2() {
+    try (UInt2Vector vector = new UInt2Vector("vector", allocator)) {
+      vector.allocateNew(2);
+
+      vector.setWithPossibleTruncate(0, UInt2Vector.MAX_UINT2);
+      vector.setUnsafeWithPossibleTruncate(1, UInt2Vector.MAX_UINT2);
+      vector.setValueCount(2);
+
+      assertEquals(UInt2Vector.MAX_UINT2, vector.getValueAsLong(0));
+      assertEquals(UInt2Vector.MAX_UINT2, vector.getValueAsLong(1));
+    }
+  }
+
+  @Test
+  public void testSetGetUInt4() {
+    try (UInt4Vector vector = new UInt4Vector("vector", allocator)) {
+      vector.allocateNew(2);
+
+      vector.setWithPossibleTruncate(0, UInt4Vector.MAX_UINT4);
+      vector.setUnsafeWithPossibleTruncate(1, UInt4Vector.MAX_UINT4);
+      vector.setValueCount(2);
+
+      long expected = UInt4Vector.MAX_UINT4 & UInt4Vector.PROMOTION_MASK;
+      assertEquals(expected, vector.getValueAsLong(0));
+      assertEquals(expected, vector.getValueAsLong(1));
+    }
+  }
 }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestUIntDictionaryRoundTrip.java b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestUIntDictionaryRoundTrip.java
new file mode 100644
index 0000000..6aa7a0c
--- /dev/null
+++ b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestUIntDictionaryRoundTrip.java
@@ -0,0 +1,246 @@
+/*
+ * 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.arrow.vector.ipc;
+
+import static org.apache.arrow.vector.testing.ValueVectorDataPopulator.setVector;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.channels.Channels;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+import java.util.function.ToIntBiFunction;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.UInt1Vector;
+import org.apache.arrow.vector.UInt2Vector;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.UInt8Vector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.dictionary.Dictionary;
+import org.apache.arrow.vector.dictionary.DictionaryProvider;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.DictionaryEncoding;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Test the round-trip of dictionary encoding,
+ * with unsigned integer as indices.
+ */
+@RunWith(Parameterized.class)
+public class TestUIntDictionaryRoundTrip {
+
+  private final boolean streamMode;
+
+  public TestUIntDictionaryRoundTrip(boolean streamMode) {
+    this.streamMode = streamMode;
+  }
+
+  private BufferAllocator allocator;
+
+  private DictionaryProvider.MapDictionaryProvider dictionaryProvider;
+
+  @Before
+  public void init() {
+    allocator = new RootAllocator(Long.MAX_VALUE);
+    dictionaryProvider = new DictionaryProvider.MapDictionaryProvider();
+  }
+
+  @After
+  public void terminate() throws Exception {
+    allocator.close();
+  }
+
+  private byte[] writeData(FieldVector encodedVector) throws IOException {
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    VectorSchemaRoot root =
+        new VectorSchemaRoot(
+            Arrays.asList(encodedVector.getField()), Arrays.asList(encodedVector), encodedVector.getValueCount());
+    try (ArrowWriter writer = streamMode ?
+        new ArrowStreamWriter(root, dictionaryProvider, out) :
+        new ArrowFileWriter(root, dictionaryProvider, Channels.newChannel(out))) {
+      writer.start();
+      writer.writeBatch();
+      writer.end();
+
+      return out.toByteArray();
+    }
+  }
+
+  private void readData(
+      byte[] data,
+      Field expectedField,
+      ToIntBiFunction<ValueVector, Integer> valGetter,
+      long dictionaryID,
+      int[] expectedIndices,
+      String[] expectedDictItems) throws IOException {
+    try (ArrowReader reader = streamMode ?
+             new ArrowStreamReader(new ByteArrayInputStream(data), allocator) :
+             new ArrowFileReader(new SeekableReadChannel(new ByteArrayReadableSeekableByteChannel(data)), allocator)) {
+
+      // verify schema
+      Schema readSchema = reader.getVectorSchemaRoot().getSchema();
+      assertEquals(1, readSchema.getFields().size());
+      assertEquals(expectedField, readSchema.getFields().get(0));
+
+      // verify vector schema root
+      assertTrue(reader.loadNextBatch());
+      VectorSchemaRoot root = reader.getVectorSchemaRoot();
+
+      assertEquals(1, root.getFieldVectors().size());
+      ValueVector encodedVector = root.getVector(0);
+      assertEquals(expectedIndices.length, encodedVector.getValueCount());
+
+      for (int i = 0; i < expectedIndices.length; i++) {
+        assertEquals(expectedIndices[i], valGetter.applyAsInt(encodedVector, i));
+      }
+
+      // verify dictionary
+      Map<Long, Dictionary> dictVectors = reader.getDictionaryVectors();
+      assertEquals(1, dictVectors.size());
+      Dictionary dictionary = dictVectors.get(dictionaryID);
+      assertNotNull(dictionary);
+
+      assertTrue(dictionary.getVector() instanceof VarCharVector);
+      VarCharVector dictVector = (VarCharVector) dictionary.getVector();
+      assertEquals(expectedDictItems.length, dictVector.getValueCount());
+      for (int i = 0; i < dictVector.getValueCount(); i++) {
+        assertArrayEquals(expectedDictItems[i].getBytes(), dictVector.get(i));
+      }
+    }
+  }
+
+  private ValueVector createEncodedVector(int bitWidth, VarCharVector dictionaryVector) {
+    final DictionaryEncoding dictionaryEncoding =
+        new DictionaryEncoding(bitWidth, false, new ArrowType.Int(bitWidth, false));
+    Dictionary dictionary = new Dictionary(dictionaryVector, dictionaryEncoding);
+    dictionaryProvider.put(dictionary);
+
+    final FieldType type =
+        new FieldType(true, dictionaryEncoding.getIndexType(), dictionaryEncoding, null);
+    final Field field = new Field("encoded", type, null);
+    return field.createVector(allocator);
+  }
+
+  @Test
+  public void testUInt1RoundTrip() throws IOException {
+    final int vectorLength = UInt1Vector.MAX_UINT1 & UInt1Vector.PROMOTION_MASK;
+    try (VarCharVector dictionaryVector = new VarCharVector("dictionary", allocator);
+         UInt1Vector encodedVector1 = (UInt1Vector) createEncodedVector(8, dictionaryVector)) {
+      int[] indices = new int[vectorLength];
+      String[] dictionaryItems = new String[vectorLength];
+      for (int i = 0; i < vectorLength; i++) {
+        encodedVector1.setSafe(i, (byte) i);
+        indices[i] = i;
+        dictionaryItems[i] = String.valueOf(i);
+      }
+      encodedVector1.setValueCount(vectorLength);
+      setVector(dictionaryVector, dictionaryItems);
+      byte[] data = writeData(encodedVector1);
+      readData(
+          data, encodedVector1.getField(), (vector, index) -> (int) ((UInt1Vector) vector).getValueAsLong(index),
+          8L, indices, dictionaryItems);
+    }
+  }
+
+  @Test
+  public void testUInt2RoundTrip() throws IOException {
+    try (VarCharVector dictionaryVector = new VarCharVector("dictionary", allocator);
+        UInt2Vector encodedVector2 = (UInt2Vector) createEncodedVector(16, dictionaryVector)) {
+      int[] indices = new int[]{1, 3, 5, 7, 9, UInt2Vector.MAX_UINT2};
+      String[] dictItems = new String[UInt2Vector.MAX_UINT2];
+      for (int i = 0; i < UInt2Vector.MAX_UINT2; i++) {
+        dictItems[i] = String.valueOf(i);
+      }
+
+      setVector(encodedVector2, (char) 1, (char) 3, (char) 5, (char) 7, (char) 9, UInt2Vector.MAX_UINT2);
+      setVector(dictionaryVector, dictItems);
+
+      byte[] data = writeData(encodedVector2);
+      readData(data, encodedVector2.getField(), (vector, index) -> (int) ((UInt2Vector) vector).getValueAsLong(index),
+          16L, indices, dictItems);
+    }
+  }
+
+  @Test
+  public void testUInt4RoundTrip() throws IOException {
+    final int dictLength = 10;
+    try (VarCharVector dictionaryVector = new VarCharVector("dictionary", allocator);
+        UInt4Vector encodedVector4 = (UInt4Vector) createEncodedVector(32, dictionaryVector)) {
+      int[] indices = new int[]{1, 3, 5, 7, 9};
+      String[] dictItems = new String[dictLength];
+      for (int i = 0; i < dictLength; i++) {
+        dictItems[i] = String.valueOf(i);
+      }
+
+      setVector(encodedVector4, 1, 3, 5, 7, 9);
+      setVector(dictionaryVector, dictItems);
+
+      setVector(encodedVector4, 1, 3, 5, 7, 9);
+      byte[] data = writeData(encodedVector4);
+      readData(data, encodedVector4.getField(), (vector, index) -> (int) ((UInt4Vector) vector).getValueAsLong(index),
+          32L, indices, dictItems);
+    }
+  }
+
+  @Test
+  public void testUInt8RoundTrip() throws IOException {
+    final int dictLength = 10;
+    try (VarCharVector dictionaryVector = new VarCharVector("dictionary", allocator);
+        UInt8Vector encodedVector8 = (UInt8Vector) createEncodedVector(64, dictionaryVector)) {
+      int[] indices = new int[]{1, 3, 5, 7, 9};
+      String[] dictItems = new String[dictLength];
+      for (int i = 0; i < dictLength; i++) {
+        dictItems[i] = String.valueOf(i);
+      }
+
+      setVector(encodedVector8, 1L, 3L, 5L, 7L, 9L);
+      setVector(dictionaryVector, dictItems);
+
+      byte[] data = writeData(encodedVector8);
+      readData(data, encodedVector8.getField(), (vector, index) -> (int) ((UInt8Vector) vector).getValueAsLong(index),
+          64L, indices, dictItems);
+    }
+  }
+
+  @Parameterized.Parameters(name = "stream mode = {0}")
+  public static Collection<Object[]> getRepeat() {
+    return Arrays.asList(
+        new Object[]{true},
+        new Object[]{false}
+    );
+  }
+}