You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2019/06/05 02:20:13 UTC

[arrow] branch master updated: ARROW-1837: [Java][Integration] Fix unsigned round trip integration tests

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

wesm 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 746805c  ARROW-1837: [Java][Integration] Fix unsigned round trip integration tests
746805c is described below

commit 746805cbf66739f79d874cc00147b1aa4de235a8
Author: Micah Kornfield <em...@gmail.com>
AuthorDate: Tue Jun 4 21:19:58 2019 -0500

    ARROW-1837: [Java][Integration] Fix unsigned round trip integration tests
    
    - Show unsigned values can be round-tripped between java and C++
      in integration tests.  This doesn't fully fix the problem because
      the UInt* APIs are mostly wrong because they can't represent the
      full range of unsigned values (return types are all too small
      because java only has signed types).
    
    - While I was at it, I fixed the issue with no batches.
    
    Author: Micah Kornfield <em...@gmail.com>
    Author: emkornfield <em...@gmail.com>
    
    Closes #4432 from emkornfield/fix_integration_tests and squashes the following commits:
    
    226c4af0 <Micah Kornfield> fixes
    27e47383 <emkornfield> Add missing comma to integration test
    d8ad3d85 <Micah Kornfield> Address PR feedback
    a6a23e9d <Micah Kornfield> ARROW-1837:  Fix unsigned round trip integration tests
---
 integration/integration_test.py                    | 10 ++--
 .../java/org/apache/arrow/tools/FileToStream.java  |  7 +--
 .../java/org/apache/arrow/tools/StreamToFile.java  |  9 ++-
 .../java/org/apache/arrow/vector/UInt1Vector.java  | 33 ++++++++++
 .../java/org/apache/arrow/vector/UInt2Vector.java  | 16 ++++-
 .../java/org/apache/arrow/vector/UInt4Vector.java  | 34 ++++++++++-
 .../java/org/apache/arrow/vector/UInt8Vector.java  | 37 ++++++++++++
 .../apache/arrow/vector/ipc/JsonFileReader.java    | 70 ++++++++++++++++++++--
 .../apache/arrow/vector/ipc/JsonFileWriter.java    | 16 +++++
 .../org/apache/arrow/vector/ipc/BaseFileTest.java  | 39 ++++++++++++
 .../org/apache/arrow/vector/ipc/TestJSONFile.java  | 30 ++++++++++
 11 files changed, 280 insertions(+), 21 deletions(-)

diff --git a/integration/integration_test.py b/integration/integration_test.py
index 0c5aee4..54e9487 100644
--- a/integration/integration_test.py
+++ b/integration/integration_test.py
@@ -194,13 +194,12 @@ class IntegerType(PrimitiveType):
         self.max_value = max_value
 
     def _get_generated_data_bounds(self):
-        signed_iinfo = np.iinfo('int' + str(self.bit_width))
         if self.is_signed:
+            signed_iinfo = np.iinfo('int' + str(self.bit_width))
             min_value, max_value = signed_iinfo.min, signed_iinfo.max
         else:
-            # ARROW-1837 Remove this hack and restore full unsigned integer
-            # range
-            min_value, max_value = 0, signed_iinfo.max
+            unsigned_iinfo = np.iinfo('uint' + str(self.bit_width))
+            min_value, max_value = 0, unsigned_iinfo.max
 
         lower_bound = max(min_value, self.min_value)
         upper_bound = min(max_value, self.max_value)
@@ -1030,8 +1029,7 @@ def get_generated_json_files(tempdir=None, flight=False):
         return
 
     file_objs = [
-        (generate_primitive_case([], name='primitive_no_batches')
-         .skip_category('Java')),
+        generate_primitive_case([], name='primitive_no_batches'),
         generate_primitive_case([17, 20], name='primitive'),
         generate_primitive_case([0, 0, 0], name='primitive_zerolength'),
         generate_decimal_case(),
diff --git a/java/tools/src/main/java/org/apache/arrow/tools/FileToStream.java b/java/tools/src/main/java/org/apache/arrow/tools/FileToStream.java
index c6cb332..a89fd9d 100644
--- a/java/tools/src/main/java/org/apache/arrow/tools/FileToStream.java
+++ b/java/tools/src/main/java/org/apache/arrow/tools/FileToStream.java
@@ -44,12 +44,11 @@ public class FileToStream {
     try (ArrowFileReader reader = new ArrowFileReader(in.getChannel(), allocator)) {
       VectorSchemaRoot root = reader.getVectorSchemaRoot();
       // load the first batch before instantiating the writer so that we have any dictionaries
-      if (!reader.loadNextBatch()) {
-        throw new IOException("Unable to read first record batch");
-      }
+      // only writeBatches if we loaded one in the first palce.
+      boolean writeBatches = reader.loadNextBatch();
       try (ArrowStreamWriter writer = new ArrowStreamWriter(root, reader, out)) {
         writer.start();
-        while (true) {
+        while (writeBatches) {
           writer.writeBatch();
           if (!reader.loadNextBatch()) {
             break;
diff --git a/java/tools/src/main/java/org/apache/arrow/tools/StreamToFile.java b/java/tools/src/main/java/org/apache/arrow/tools/StreamToFile.java
index 25dd2fd..bcf4d1d 100644
--- a/java/tools/src/main/java/org/apache/arrow/tools/StreamToFile.java
+++ b/java/tools/src/main/java/org/apache/arrow/tools/StreamToFile.java
@@ -42,13 +42,12 @@ public class StreamToFile {
     BufferAllocator allocator = new RootAllocator(Integer.MAX_VALUE);
     try (ArrowStreamReader reader = new ArrowStreamReader(in, allocator)) {
       VectorSchemaRoot root = reader.getVectorSchemaRoot();
-      // load the first batch before instantiating the writer so that we have any dictionaries
-      if (!reader.loadNextBatch()) {
-        throw new IOException("Unable to read first record batch");
-      }
+      // load the first batch before instantiating the writer so that we have any dictionaries.
+      // Only writeBatches if we load the first one.
+      boolean writeBatches = reader.loadNextBatch();
       try (ArrowFileWriter writer = new ArrowFileWriter(root, reader, Channels.newChannel(out))) {
         writer.start();
-        while (true) {
+        while (writeBatches) {
           writer.writeBatch();
           if (!reader.loadNextBatch()) {
             break;
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 4e135cb..c513334 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
@@ -28,6 +28,8 @@ import org.apache.arrow.vector.types.Types.MinorType;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
+import io.netty.buffer.ArrowBuf;
+
 /**
  * UInt1Vector implements a fixed width (1 bytes) vector of
  * integer values which could be null. A validity buffer (bit vector) is
@@ -62,6 +64,23 @@ public class UInt1Vector extends BaseFixedWidthVector {
    |          vector value retrieval methods                        |
    |                                                                |
    *----------------------------------------------------------------*/
+  /**
+   * Given a data buffer, get the value stored at a particular position
+   * in the vector.
+   *
+   * <p>To avoid overflow, the returned type is one step up from the signed
+   * type.
+   *
+   * <p>This method is mainly meant for integration tests.
+   *
+   * @param buffer data buffer
+   * @param index position of the element.
+   * @return value stored at the index.
+   */
+  public static short getNoOverflow(final ArrowBuf buffer, final int index) {
+    byte b =  buffer.getByte(index * TYPE_WIDTH);
+    return (short)(0xFF & b);
+  }
 
 
   /**
@@ -108,6 +127,20 @@ public class UInt1Vector extends BaseFixedWidthVector {
   }
 
   /**
+   * Returns the value stored at index without the potential for overflow.
+   *
+   * @param index   position of element
+   * @return element at given index
+   */
+  public Short getObjectNoOverflow(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
+      return getNoOverflow(valueBuffer, index);
+    }
+  }
+
+  /**
    * Copies the value at fromIndex to thisIndex (including validity).
    */
   public void copyFrom(int fromIndex, int thisIndex, UInt1Vector from) {
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 43a6397..631050d 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
@@ -28,6 +28,8 @@ import org.apache.arrow.vector.types.Types.MinorType;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
+import io.netty.buffer.ArrowBuf;
+
 /**
  * UInt2Vector implements a fixed width (2 bytes) vector of
  * integer values which could be null. A validity buffer (bit vector) is
@@ -62,7 +64,19 @@ public class UInt2Vector extends BaseFixedWidthVector {
    |          vector value retrieval methods                        |
    |                                                                |
    *----------------------------------------------------------------*/
-
+  /**
+   * Given a data buffer, get the value stored at a particular position
+   * in the vector.
+   *
+   * <p>This method is mainly meant for integration tests.
+   *
+   * @param buffer data buffer
+   * @param index position of the element.
+   * @return value stored at the index.
+   */
+  public static char get(final ArrowBuf buffer, final int index) {
+    return buffer.getChar(index * TYPE_WIDTH);
+  }
 
   /**
    * Get the element at the given index from the vector.
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 652fef8..84e6b8f 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
@@ -28,6 +28,8 @@ import org.apache.arrow.vector.types.Types.MinorType;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
+import io.netty.buffer.ArrowBuf;
+
 /**
  * UInt4Vector implements a fixed width (4 bytes) vector of
  * integer values which could be null. A validity buffer (bit vector) is
@@ -62,7 +64,23 @@ public class UInt4Vector extends BaseFixedWidthVector {
    |          vector value retrieval methods                        |
    |                                                                |
    *----------------------------------------------------------------*/
-
+  /**
+   * Given a data buffer, get the value stored at a particular position
+   * in the vector.
+   *
+   * <p>To avoid overflow, the returned type is one step up from the signed
+   * type.
+   *
+   * <p>This method is mainly meant for integration tests.
+   *
+   * @param buffer data buffer
+   * @param index position of the element.
+   * @return value stored at the index.
+   */
+  public static long getNoOverflow(final ArrowBuf buffer, final int index) {
+    long l =  buffer.getInt(index * TYPE_WIDTH);
+    return ((long)0xFFFFFFFF) & l;
+  }
 
   /**
    * Get the element at the given index from the vector.
@@ -108,6 +126,20 @@ public class UInt4Vector extends BaseFixedWidthVector {
   }
 
   /**
+   * Same as {@link #get(int)}.
+   *
+   * @param index   position of element
+   * @return element at given index
+   */
+  public Long getObjectNoOverflow(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
+      return getNoOverflow(valueBuffer, index);
+    }
+  }
+
+  /**
    * Copies a value and validity setting to the thisIndex position from the given vector
    * at fromIndex.
    */
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 a39a3bf..0f8da38 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
@@ -19,6 +19,8 @@ package org.apache.arrow.vector;
 
 import static org.apache.arrow.vector.NullCheckingForGet.NULL_CHECKING_ENABLED;
 
+import java.math.BigInteger;
+
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.complex.impl.UInt8ReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
@@ -28,6 +30,8 @@ import org.apache.arrow.vector.types.Types.MinorType;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
+import io.netty.buffer.ArrowBuf;
+
 /**
  * UInt8Vector implements a fixed width vector (8 bytes) of
  * integer values which could be null. A validity buffer (bit vector) is
@@ -62,6 +66,25 @@ public class UInt8Vector extends BaseFixedWidthVector {
    |          vector value retrieval methods                        |
    |                                                                |
    *----------------------------------------------------------------*/
+  private static final BigInteger SAFE_CONVERSION_MASK = new BigInteger("ffffffffffffffff", 16);
+
+  /**
+   * Given a data buffer, get the value stored at a particular position
+   * in the vector.
+   *
+   * <p>To avoid overflow, the returned type is one step up from the signed
+   * type.
+   *
+   * <p>This method is mainly meant for integration tests.
+   *
+   * @param buffer data buffer
+   * @param index position of the element.
+   * @return value stored at the index.
+   */
+  public static BigInteger getNoOverflow(final ArrowBuf buffer, final int index) {
+    BigInteger l =  BigInteger.valueOf(buffer.getLong(index * TYPE_WIDTH));
+    return SAFE_CONVERSION_MASK.and(l);
+  }
 
 
   /**
@@ -108,6 +131,20 @@ public class UInt8Vector extends BaseFixedWidthVector {
   }
 
   /**
+   * Returns the value stored at index without the potential for overflow.
+   *
+   * @param index   position of element
+   * @return element at given index
+   */
+  public BigInteger getObjectNoOverflow(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
+      return getNoOverflow(valueBuffer, index);
+    }
+  }
+
+  /**
    * Copy a value and validity setting from fromIndex in <code>from</code> to this
    * Vector at thisIndex.
    */
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java b/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java
index 3d4d4b1..5ae970a 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java
@@ -24,6 +24,7 @@ import static org.apache.arrow.vector.BufferLayout.BufferType.*;
 import java.io.File;
 import java.io.IOException;
 import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -324,6 +325,67 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
       }
     };
 
+    BufferReader UINT1 = new BufferReader() {
+      @Override
+      protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
+        final int size = count * TinyIntVector.TYPE_WIDTH;
+        ArrowBuf buf = allocator.buffer(size);
+
+        for (int i = 0; i < count; i++) {
+          parser.nextToken();
+          buf.writeByte(parser.getShortValue() & 0xFF);
+        }
+
+        return buf;
+      }
+    };
+
+    BufferReader UINT2 = new BufferReader() {
+      @Override
+      protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
+        final int size = count * SmallIntVector.TYPE_WIDTH;
+        ArrowBuf buf = allocator.buffer(size);
+
+        for (int i = 0; i < count; i++) {
+          parser.nextToken();
+          buf.writeShort(parser.getIntValue() & 0xFFFF);
+        }
+
+        return buf;
+      }
+    };
+
+    BufferReader UINT4 = new BufferReader() {
+      @Override
+      protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
+        final int size = count * IntVector.TYPE_WIDTH;
+        ArrowBuf buf = allocator.buffer(size);
+
+        for (int i = 0; i < count; i++) {
+          parser.nextToken();
+          buf.writeInt((int)parser.getLongValue());
+        }
+
+        return buf;
+      }
+    };
+
+    BufferReader UINT8 = new BufferReader() {
+      @Override
+      protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
+        final int size = count * BigIntVector.TYPE_WIDTH;
+        ArrowBuf buf = allocator.buffer(size);
+
+        for (int i = 0; i < count; i++) {
+          parser.nextToken();
+          BigInteger value = parser.getBigIntegerValue();
+          buf.writeLong(value.longValue());
+        }
+
+        return buf;
+      }
+    };
+
     BufferReader FLOAT4 = new BufferReader() {
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
@@ -470,16 +532,16 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
           reader = helper.INT8;
           break;
         case UINT1:
-          reader = helper.INT1;
+          reader = helper.UINT1;
           break;
         case UINT2:
-          reader = helper.INT2;
+          reader = helper.UINT2;
           break;
         case UINT4:
-          reader = helper.INT4;
+          reader = helper.UINT4;
           break;
         case UINT8:
-          reader = helper.INT8;
+          reader = helper.UINT8;
           break;
         case FLOAT4:
           reader = helper.FLOAT4;
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileWriter.java b/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileWriter.java
index 937550c..dfd57c5 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileWriter.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileWriter.java
@@ -58,6 +58,10 @@ import org.apache.arrow.vector.TimeStampSecTZVector;
 import org.apache.arrow.vector.TimeStampSecVector;
 import org.apache.arrow.vector.TinyIntVector;
 import org.apache.arrow.vector.TypeLayout;
+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.VectorSchemaRoot;
 import org.apache.arrow.vector.dictionary.Dictionary;
 import org.apache.arrow.vector.dictionary.DictionaryProvider;
@@ -271,6 +275,18 @@ public class JsonFileWriter implements AutoCloseable {
         case BIGINT:
           generator.writeNumber(BigIntVector.get(buffer, index));
           break;
+        case UINT1:
+          generator.writeNumber(UInt1Vector.getNoOverflow(buffer, index));
+          break;
+        case UINT2:
+          generator.writeNumber(UInt2Vector.get(buffer, index));
+          break;
+        case UINT4:
+          generator.writeNumber(UInt4Vector.getNoOverflow(buffer, index));
+          break;
+        case UINT8:
+          generator.writeNumber(UInt8Vector.getNoOverflow(buffer, index));
+          break;
         case FLOAT4:
           generator.writeNumber(Float4Vector.get(buffer, index));
           break;
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java b/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java
index 447ed70..e44996d 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java
@@ -40,6 +40,10 @@ import org.apache.arrow.vector.DecimalVector;
 import org.apache.arrow.vector.FieldVector;
 import org.apache.arrow.vector.IntVector;
 import org.apache.arrow.vector.TimeMilliVector;
+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.VarBinaryVector;
 import org.apache.arrow.vector.VarCharVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
@@ -59,6 +63,10 @@ import org.apache.arrow.vector.complex.writer.TimeMilliWriter;
 import org.apache.arrow.vector.complex.writer.TimeStampMilliTZWriter;
 import org.apache.arrow.vector.complex.writer.TimeStampMilliWriter;
 import org.apache.arrow.vector.complex.writer.TimeStampNanoWriter;
+import org.apache.arrow.vector.complex.writer.UInt1Writer;
+import org.apache.arrow.vector.complex.writer.UInt2Writer;
+import org.apache.arrow.vector.complex.writer.UInt4Writer;
+import org.apache.arrow.vector.complex.writer.UInt8Writer;
 import org.apache.arrow.vector.dictionary.Dictionary;
 import org.apache.arrow.vector.dictionary.DictionaryEncoder;
 import org.apache.arrow.vector.dictionary.DictionaryProvider;
@@ -94,15 +102,37 @@ public class BaseFileTest {
     allocator.close();
   }
 
+
+  private static short [] uint1Values = new short[]{0, 255, 1, 128, 2};
+  private static char [] uint2Values = new char[]{0, Character.MAX_VALUE, 1, Short.MAX_VALUE * 2, 2};
+  private static long [] uint4Values = new long[]{0, Integer.MAX_VALUE + 1, 1, Integer.MAX_VALUE * 2, 2};
+  private static BigInteger[] uint8Values = new BigInteger[]{BigInteger.valueOf(0),
+      BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.valueOf(2)), BigInteger.valueOf(2),
+      BigInteger.valueOf(Long.MAX_VALUE).add(BigInteger.valueOf(1)), BigInteger.valueOf(2)};
+
   protected void writeData(int count, StructVector parent) {
     ComplexWriter writer = new ComplexWriterImpl("root", parent);
     StructWriter rootWriter = writer.rootAsStruct();
     IntWriter intWriter = rootWriter.integer("int");
+    UInt1Writer uint1Writer = rootWriter.uInt1("uint1");
+    UInt2Writer uint2Writer = rootWriter.uInt2("uint2");
+    UInt4Writer uint4Writer = rootWriter.uInt4("uint4");
+    UInt8Writer uint8Writer = rootWriter.uInt8("uint8");
     BigIntWriter bigIntWriter = rootWriter.bigInt("bigInt");
     Float4Writer float4Writer = rootWriter.float4("float");
     for (int i = 0; i < count; i++) {
       intWriter.setPosition(i);
       intWriter.writeInt(i);
+      uint1Writer.setPosition(i);
+      // TODO: Fix add safe write methods on uint methods.
+      uint1Writer.setPosition(i);
+      uint1Writer.writeUInt1((byte)uint1Values[i % uint1Values.length] );
+      uint2Writer.setPosition(i);
+      uint2Writer.writeUInt2((char)uint2Values[i % uint2Values.length] );
+      uint4Writer.setPosition(i);
+      uint4Writer.writeUInt4((int)uint4Values[i % uint4Values.length] );
+      uint8Writer.setPosition(i);
+      uint8Writer.writeUInt8(uint8Values[i % uint8Values.length].longValue());
       bigIntWriter.setPosition(i);
       bigIntWriter.writeBigInt(i);
       float4Writer.setPosition(i);
@@ -111,9 +141,18 @@ public class BaseFileTest {
     writer.setValueCount(count);
   }
 
+
   protected void validateContent(int count, VectorSchemaRoot root) {
     for (int i = 0; i < count; i++) {
       Assert.assertEquals(i, root.getVector("int").getObject(i));
+      Assert.assertEquals((Short)uint1Values[i % uint1Values.length],
+          ((UInt1Vector)root.getVector("uint1")).getObjectNoOverflow(i));
+      Assert.assertEquals("Failed for index: " + i, (Character)uint2Values[i % uint2Values.length],
+          (Character)((UInt2Vector)root.getVector("uint2")).get(i));
+      Assert.assertEquals("Failed for index: " + i, (Long)uint4Values[i % uint4Values.length],
+          ((UInt4Vector)root.getVector("uint4")).getObjectNoOverflow(i));
+      Assert.assertEquals("Failed for index: " + i, uint8Values[i % uint8Values.length],
+          ((UInt8Vector)root.getVector("uint8")).getObjectNoOverflow(i));
       Assert.assertEquals(Long.valueOf(i), root.getVector("bigInt").getObject(i));
       Assert.assertEquals(i == 0 ? Float.NaN : i, root.getVector("float").getObject(i));
     }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestJSONFile.java b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestJSONFile.java
index f72f5c2..67ac375 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestJSONFile.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestJSONFile.java
@@ -24,6 +24,8 @@ import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.FieldVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
 import org.apache.arrow.vector.complex.StructVector;
+import org.apache.arrow.vector.complex.impl.ComplexWriterImpl;
+import org.apache.arrow.vector.complex.writer.BaseWriter;
 import org.apache.arrow.vector.dictionary.DictionaryProvider;
 import org.apache.arrow.vector.dictionary.DictionaryProvider.MapDictionaryProvider;
 import org.apache.arrow.vector.types.pojo.Schema;
@@ -37,6 +39,34 @@ public class TestJSONFile extends BaseFileTest {
   private static final Logger LOGGER = LoggerFactory.getLogger(TestJSONFile.class);
 
   @Test
+  public void testNoBatches() throws IOException {
+    File file = new File("target/no_batches.json");
+
+    try (BufferAllocator originalVectorAllocator =
+             allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
+         StructVector parent = StructVector.empty("parent", originalVectorAllocator)) {
+      BaseWriter.ComplexWriter writer = new ComplexWriterImpl("root", parent);
+      BaseWriter.StructWriter rootWriter = writer.rootAsStruct();
+      rootWriter.integer("int");
+      rootWriter.uInt1("uint1");
+      rootWriter.bigInt("bigInt");
+      rootWriter.float4("float");
+      JsonFileWriter jsonWriter = new JsonFileWriter(file, JsonFileWriter.config().pretty(true));
+      jsonWriter.start(new VectorSchemaRoot(parent.getChild("root")).getSchema(), null);
+      jsonWriter.close();
+    }
+
+    // read
+    try (
+        BufferAllocator readerAllocator = allocator.newChildAllocator("reader", 0, Integer.MAX_VALUE);
+        JsonFileReader reader = new JsonFileReader(file, readerAllocator)
+    ) {
+      Schema schema = reader.start();
+      LOGGER.debug("reading schema: " + schema);
+    }
+  }
+
+  @Test
   public void testWriteRead() throws IOException {
     File file = new File("target/mytest.json");
     int count = COUNT;