You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by sm...@apache.org on 2016/03/09 01:26:50 UTC

[1/4] arrow git commit: ARROW-46: ListVector should initialize bits in allocateNew

Repository: arrow
Updated Branches:
  refs/heads/master 45cd9fd8d -> e822ea758


ARROW-46: ListVector should initialize bits in allocateNew


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/e822ea75
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/e822ea75
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/e822ea75

Branch: refs/heads/master
Commit: e822ea758dc18ade9d3386acfd1d38e7b05ba3dd
Parents: 31def7d
Author: Minji Kim <mi...@dremio.com>
Authored: Mon Mar 7 15:23:33 2016 -0800
Committer: Steven Phillips <st...@dremio.com>
Committed: Tue Mar 8 16:09:19 2016 -0800

----------------------------------------------------------------------
 .../apache/arrow/vector/complex/ListVector.java |  1 +
 .../apache/arrow/vector/TestValueVector.java    | 20 ++++++++++++++++++++
 2 files changed, 21 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/e822ea75/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
index 13610c4..3e60c76 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
@@ -72,6 +72,7 @@ public class ListVector extends BaseRepeatedValueVector {
   @Override
   public void allocateNew() throws OutOfMemoryException {
     super.allocateNewSafe();
+    bits.allocateNewSafe();
   }
 
   public void transferTo(ListVector target) {

http://git-wip-us.apache.org/repos/asf/arrow/blob/e822ea75/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
----------------------------------------------------------------------
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 4488d75..ac3eebe 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
@@ -24,10 +24,13 @@ import static org.junit.Assert.assertTrue;
 import java.nio.charset.Charset;
 
 import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.complex.ListVector;
 import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.complex.RepeatedListVector;
 import org.apache.arrow.vector.complex.RepeatedMapVector;
 import org.apache.arrow.vector.types.MaterializedField;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.Types.MinorType;
 import org.apache.arrow.vector.util.BasicTypeHelper;
 import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.holders.BitHolder;
@@ -518,4 +521,21 @@ public class TestValueVector {
     }
   }
 
+  @Test
+  public void testListVectorShouldNotThrowOversizedAllocationException() throws Exception {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH,
+            Types.optional(MinorType.LIST));
+    ListVector vector = new ListVector(field, allocator, null);
+    ListVector vectorFrom = new ListVector(field, allocator, null);
+    vectorFrom.allocateNew();
+
+    for (int i = 0; i < 10000; i++) {
+      vector.allocateNew();
+      vector.copyFromSafe(0, 0, vectorFrom);
+      vector.clear();
+    }
+
+    vectorFrom.clear();
+    vector.clear();
+  }
 }


[3/4] arrow git commit: ARROW-51: Add simple ValueVector tests

Posted by sm...@apache.org.
ARROW-51: Add simple ValueVector tests


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/31def7d8
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/31def7d8
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/31def7d8

Branch: refs/heads/master
Commit: 31def7d81a094dd051d2f4bbead78edaae25755a
Parents: 243ed4e
Author: Steven Phillips <st...@dremio.com>
Authored: Tue Mar 8 14:11:29 2016 -0800
Committer: Steven Phillips <st...@dremio.com>
Committed: Tue Mar 8 16:09:19 2016 -0800

----------------------------------------------------------------------
 .../apache/arrow/vector/TestValueVector.java    | 521 +++++++++++++++++++
 1 file changed, 521 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/31def7d8/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..4488d75
--- /dev/null
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
@@ -0,0 +1,521 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.charset.Charset;
+
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.RepeatedListVector;
+import org.apache.arrow.vector.complex.RepeatedMapVector;
+import org.apache.arrow.vector.types.MaterializedField;
+import org.apache.arrow.vector.util.BasicTypeHelper;
+import org.apache.arrow.vector.util.OversizedAllocationException;
+import org.apache.arrow.vector.holders.BitHolder;
+import org.apache.arrow.vector.holders.IntHolder;
+import org.apache.arrow.vector.holders.NullableFloat4Holder;
+import org.apache.arrow.vector.holders.NullableUInt4Holder;
+import org.apache.arrow.vector.holders.NullableVar16CharHolder;
+import org.apache.arrow.vector.holders.NullableVarCharHolder;
+import org.apache.arrow.vector.holders.RepeatedFloat4Holder;
+import org.apache.arrow.vector.holders.RepeatedIntHolder;
+import org.apache.arrow.vector.holders.RepeatedVarBinaryHolder;
+import org.apache.arrow.vector.holders.UInt4Holder;
+import org.apache.arrow.vector.holders.VarCharHolder;
+import org.apache.arrow.memory.BufferAllocator;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class TestValueVector {
+  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestValueVector.class);
+
+  private final static String EMPTY_SCHEMA_PATH = "";
+
+  private BufferAllocator allocator;
+
+  @Before
+  public void init() {
+    allocator = new RootAllocator(Long.MAX_VALUE);
+  }
+
+  private final static Charset utf8Charset = Charset.forName("UTF-8");
+  private final static byte[] STR1 = new String("AAAAA1").getBytes(utf8Charset);
+  private final static byte[] STR2 = new String("BBBBBBBBB2").getBytes(utf8Charset);
+  private final static byte[] STR3 = new String("CCCC3").getBytes(utf8Charset);
+
+  @After
+  public void terminate() throws Exception {
+    allocator.close();
+  }
+
+  @Test(expected = OversizedAllocationException.class)
+  public void testFixedVectorReallocation() {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, UInt4Holder.TYPE);
+    final UInt4Vector vector = new UInt4Vector(field, allocator);
+    // edge case 1: buffer size = max value capacity
+    final int expectedValueCapacity = BaseValueVector.MAX_ALLOCATION_SIZE / 4;
+    try {
+      vector.allocateNew(expectedValueCapacity);
+      assertEquals(expectedValueCapacity, vector.getValueCapacity());
+      vector.reAlloc();
+      assertEquals(expectedValueCapacity * 2, vector.getValueCapacity());
+    } finally {
+      vector.close();
+    }
+
+    // common case: value count < max value capacity
+    try {
+      vector.allocateNew(BaseValueVector.MAX_ALLOCATION_SIZE / 8);
+      vector.reAlloc(); // value allocation reaches to MAX_VALUE_ALLOCATION
+      vector.reAlloc(); // this should throw an IOOB
+    } finally {
+      vector.close();
+    }
+  }
+
+  @Test(expected = OversizedAllocationException.class)
+  public void testBitVectorReallocation() {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, UInt4Holder.TYPE);
+    final BitVector vector = new BitVector(field, allocator);
+    // edge case 1: buffer size ~ max value capacity
+    final int expectedValueCapacity = 1 << 29;
+    try {
+      vector.allocateNew(expectedValueCapacity);
+      assertEquals(expectedValueCapacity, vector.getValueCapacity());
+      vector.reAlloc();
+      assertEquals(expectedValueCapacity * 2, vector.getValueCapacity());
+    } finally {
+      vector.close();
+    }
+
+    // common: value count < MAX_VALUE_ALLOCATION
+    try {
+      vector.allocateNew(expectedValueCapacity);
+      for (int i=0; i<3;i++) {
+        vector.reAlloc(); // expand buffer size
+      }
+      assertEquals(Integer.MAX_VALUE, vector.getValueCapacity());
+      vector.reAlloc(); // buffer size ~ max allocation
+      assertEquals(Integer.MAX_VALUE, vector.getValueCapacity());
+      vector.reAlloc(); // overflow
+    } finally {
+      vector.close();
+    }
+  }
+
+
+  @Test(expected = OversizedAllocationException.class)
+  public void testVariableVectorReallocation() {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, UInt4Holder.TYPE);
+    final VarCharVector vector = new VarCharVector(field, allocator);
+    // edge case 1: value count = MAX_VALUE_ALLOCATION
+    final int expectedAllocationInBytes = BaseValueVector.MAX_ALLOCATION_SIZE;
+    final int expectedOffsetSize = 10;
+    try {
+      vector.allocateNew(expectedAllocationInBytes, 10);
+      assertTrue(expectedOffsetSize <= vector.getValueCapacity());
+      assertTrue(expectedAllocationInBytes <= vector.getBuffer().capacity());
+      vector.reAlloc();
+      assertTrue(expectedOffsetSize * 2 <= vector.getValueCapacity());
+      assertTrue(expectedAllocationInBytes * 2 <= vector.getBuffer().capacity());
+    } finally {
+      vector.close();
+    }
+
+    // common: value count < MAX_VALUE_ALLOCATION
+    try {
+      vector.allocateNew(BaseValueVector.MAX_ALLOCATION_SIZE / 2, 0);
+      vector.reAlloc(); // value allocation reaches to MAX_VALUE_ALLOCATION
+      vector.reAlloc(); // this tests if it overflows
+    } finally {
+      vector.close();
+    }
+  }
+
+  @Test
+  public void testFixedType() {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, UInt4Holder.TYPE);
+
+    // Create a new value vector for 1024 integers.
+    try (final UInt4Vector vector = new UInt4Vector(field, allocator)) {
+      final UInt4Vector.Mutator m = vector.getMutator();
+      vector.allocateNew(1024);
+
+      // Put and set a few values
+      m.setSafe(0, 100);
+      m.setSafe(1, 101);
+      m.setSafe(100, 102);
+      m.setSafe(1022, 103);
+      m.setSafe(1023, 104);
+
+      final UInt4Vector.Accessor accessor = vector.getAccessor();
+      assertEquals(100, accessor.get(0));
+      assertEquals(101, accessor.get(1));
+      assertEquals(102, accessor.get(100));
+      assertEquals(103, accessor.get(1022));
+      assertEquals(104, accessor.get(1023));
+    }
+  }
+
+  @Test
+  public void testNullableVarLen2() {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableVarCharHolder.TYPE);
+
+    // Create a new value vector for 1024 integers.
+    try (final NullableVarCharVector vector = new NullableVarCharVector(field, allocator)) {
+      final NullableVarCharVector.Mutator m = vector.getMutator();
+      vector.allocateNew(1024 * 10, 1024);
+
+      m.set(0, STR1);
+      m.set(1, STR2);
+      m.set(2, STR3);
+
+      // Check the sample strings.
+      final NullableVarCharVector.Accessor accessor = vector.getAccessor();
+      assertArrayEquals(STR1, accessor.get(0));
+      assertArrayEquals(STR2, accessor.get(1));
+      assertArrayEquals(STR3, accessor.get(2));
+
+      // Ensure null value throws.
+      boolean b = false;
+      try {
+        vector.getAccessor().get(3);
+      } catch (IllegalStateException e) {
+        b = true;
+      } finally {
+        assertTrue(b);
+      }
+    }
+  }
+
+  @Test
+  public void testRepeatedIntVector() {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, RepeatedIntHolder.TYPE);
+
+    // Create a new value vector.
+    try (final RepeatedIntVector vector1 = new RepeatedIntVector(field, allocator)) {
+
+      // Populate the vector.
+      final int[] values = {2, 3, 5, 7, 11, 13, 17, 19, 23, 27}; // some tricksy primes
+      final int nRecords = 7;
+      final int nElements = values.length;
+      vector1.allocateNew(nRecords, nRecords * nElements);
+      final RepeatedIntVector.Mutator mutator = vector1.getMutator();
+      for (int recordIndex = 0; recordIndex < nRecords; ++recordIndex) {
+        mutator.startNewValue(recordIndex);
+        for (int elementIndex = 0; elementIndex < nElements; ++elementIndex) {
+          mutator.add(recordIndex, recordIndex * values[elementIndex]);
+        }
+      }
+      mutator.setValueCount(nRecords);
+
+      // Verify the contents.
+      final RepeatedIntVector.Accessor accessor1 = vector1.getAccessor();
+      assertEquals(nRecords, accessor1.getValueCount());
+      for (int recordIndex = 0; recordIndex < nRecords; ++recordIndex) {
+        for (int elementIndex = 0; elementIndex < nElements; ++elementIndex) {
+          final int value = accessor1.get(recordIndex, elementIndex);
+          assertEquals(recordIndex * values[elementIndex], value);
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testNullableFixedType() {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableUInt4Holder.TYPE);
+
+    // Create a new value vector for 1024 integers.
+    try (final NullableUInt4Vector vector = new NullableUInt4Vector(field, allocator)) {
+      final NullableUInt4Vector.Mutator m = vector.getMutator();
+      vector.allocateNew(1024);
+
+      // Put and set a few values
+      m.set(0, 100);
+      m.set(1, 101);
+      m.set(100, 102);
+      m.set(1022, 103);
+      m.set(1023, 104);
+
+      final NullableUInt4Vector.Accessor accessor = vector.getAccessor();
+      assertEquals(100, accessor.get(0));
+      assertEquals(101, accessor.get(1));
+      assertEquals(102, accessor.get(100));
+      assertEquals(103, accessor.get(1022));
+      assertEquals(104, accessor.get(1023));
+
+      // Ensure null values throw
+      {
+        boolean b = false;
+        try {
+          accessor.get(3);
+        } catch (IllegalStateException e) {
+          b = true;
+        } finally {
+          assertTrue(b);
+        }
+      }
+
+      vector.allocateNew(2048);
+      {
+        boolean b = false;
+        try {
+          accessor.get(0);
+        } catch (IllegalStateException e) {
+          b = true;
+        } finally {
+          assertTrue(b);
+        }
+      }
+
+      m.set(0, 100);
+      m.set(1, 101);
+      m.set(100, 102);
+      m.set(1022, 103);
+      m.set(1023, 104);
+      assertEquals(100, accessor.get(0));
+      assertEquals(101, accessor.get(1));
+      assertEquals(102, accessor.get(100));
+      assertEquals(103, accessor.get(1022));
+      assertEquals(104, accessor.get(1023));
+
+      // Ensure null values throw.
+      {
+        boolean b = false;
+        try {
+          vector.getAccessor().get(3);
+        } catch (IllegalStateException e) {
+          b = true;
+        } finally {
+          assertTrue(b);
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testNullableFloat() {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableFloat4Holder.TYPE);
+
+    // Create a new value vector for 1024 integers
+    try (final NullableFloat4Vector vector = (NullableFloat4Vector) BasicTypeHelper.getNewVector(field, allocator)) {
+      final NullableFloat4Vector.Mutator m = vector.getMutator();
+      vector.allocateNew(1024);
+
+      // Put and set a few values.
+      m.set(0, 100.1f);
+      m.set(1, 101.2f);
+      m.set(100, 102.3f);
+      m.set(1022, 103.4f);
+      m.set(1023, 104.5f);
+
+      final NullableFloat4Vector.Accessor accessor = vector.getAccessor();
+      assertEquals(100.1f, accessor.get(0), 0);
+      assertEquals(101.2f, accessor.get(1), 0);
+      assertEquals(102.3f, accessor.get(100), 0);
+      assertEquals(103.4f, accessor.get(1022), 0);
+      assertEquals(104.5f, accessor.get(1023), 0);
+
+      // Ensure null values throw.
+      {
+        boolean b = false;
+        try {
+          vector.getAccessor().get(3);
+        } catch (IllegalStateException e) {
+          b = true;
+        } finally {
+          assertTrue(b);
+        }
+      }
+
+      vector.allocateNew(2048);
+      {
+        boolean b = false;
+        try {
+          accessor.get(0);
+        } catch (IllegalStateException e) {
+          b = true;
+        } finally {
+          assertTrue(b);
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testBitVector() {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, BitHolder.TYPE);
+
+    // Create a new value vector for 1024 integers
+    try (final BitVector vector = new BitVector(field, allocator)) {
+      final BitVector.Mutator m = vector.getMutator();
+      vector.allocateNew(1024);
+
+      // Put and set a few values
+      m.set(0, 1);
+      m.set(1, 0);
+      m.set(100, 0);
+      m.set(1022, 1);
+
+      final BitVector.Accessor accessor = vector.getAccessor();
+      assertEquals(1, accessor.get(0));
+      assertEquals(0, accessor.get(1));
+      assertEquals(0, accessor.get(100));
+      assertEquals(1, accessor.get(1022));
+
+      // test setting the same value twice
+      m.set(0, 1);
+      m.set(0, 1);
+      m.set(1, 0);
+      m.set(1, 0);
+      assertEquals(1, accessor.get(0));
+      assertEquals(0, accessor.get(1));
+
+      // test toggling the values
+      m.set(0, 0);
+      m.set(1, 1);
+      assertEquals(0, accessor.get(0));
+      assertEquals(1, accessor.get(1));
+
+      // Ensure unallocated space returns 0
+      assertEquals(0, accessor.get(3));
+    }
+  }
+
+  @Test
+  public void testReAllocNullableFixedWidthVector() {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableFloat4Holder.TYPE);
+
+    // Create a new value vector for 1024 integers
+    try (final NullableFloat4Vector vector = (NullableFloat4Vector) BasicTypeHelper.getNewVector(field, allocator)) {
+      final NullableFloat4Vector.Mutator m = vector.getMutator();
+      vector.allocateNew(1024);
+
+      assertEquals(1024, vector.getValueCapacity());
+
+      // Put values in indexes that fall within the initial allocation
+      m.setSafe(0, 100.1f);
+      m.setSafe(100, 102.3f);
+      m.setSafe(1023, 104.5f);
+
+      // Now try to put values in space that falls beyond the initial allocation
+      m.setSafe(2000, 105.5f);
+
+      // Check valueCapacity is more than initial allocation
+      assertEquals(1024 * 2, vector.getValueCapacity());
+
+      final NullableFloat4Vector.Accessor accessor = vector.getAccessor();
+      assertEquals(100.1f, accessor.get(0), 0);
+      assertEquals(102.3f, accessor.get(100), 0);
+      assertEquals(104.5f, accessor.get(1023), 0);
+      assertEquals(105.5f, accessor.get(2000), 0);
+
+      // Set the valueCount to be more than valueCapacity of current allocation. This is possible for NullableValueVectors
+      // as we don't call setSafe for null values, but we do call setValueCount when all values are inserted into the
+      // vector
+      m.setValueCount(vector.getValueCapacity() + 200);
+    }
+  }
+
+  @Test
+  public void testReAllocNullableVariableWidthVector() {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableVarCharHolder.TYPE);
+
+    // Create a new value vector for 1024 integers
+    try (final NullableVarCharVector vector = (NullableVarCharVector) BasicTypeHelper.getNewVector(field, allocator)) {
+      final NullableVarCharVector.Mutator m = vector.getMutator();
+      vector.allocateNew();
+
+      int initialCapacity = vector.getValueCapacity();
+
+      // Put values in indexes that fall within the initial allocation
+      m.setSafe(0, STR1, 0, STR1.length);
+      m.setSafe(initialCapacity - 1, STR2, 0, STR2.length);
+
+      // Now try to put values in space that falls beyond the initial allocation
+      m.setSafe(initialCapacity + 200, STR3, 0, STR3.length);
+
+      // Check valueCapacity is more than initial allocation
+      assertEquals((initialCapacity + 1) * 2 - 1, vector.getValueCapacity());
+
+      final NullableVarCharVector.Accessor accessor = vector.getAccessor();
+      assertArrayEquals(STR1, accessor.get(0));
+      assertArrayEquals(STR2, accessor.get(initialCapacity - 1));
+      assertArrayEquals(STR3, accessor.get(initialCapacity + 200));
+
+      // Set the valueCount to be more than valueCapacity of current allocation. This is possible for NullableValueVectors
+      // as we don't call setSafe for null values, but we do call setValueCount when the current batch is processed.
+      m.setValueCount(vector.getValueCapacity() + 200);
+    }
+  }
+
+  @Test
+  public void testVVInitialCapacity() throws Exception {
+    final MaterializedField[] fields = new MaterializedField[9];
+    final ValueVector[] valueVectors = new ValueVector[9];
+
+    fields[0] = MaterializedField.create(EMPTY_SCHEMA_PATH, BitHolder.TYPE);
+    fields[1] = MaterializedField.create(EMPTY_SCHEMA_PATH, IntHolder.TYPE);
+    fields[2] = MaterializedField.create(EMPTY_SCHEMA_PATH, VarCharHolder.TYPE);
+    fields[3] = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableVar16CharHolder.TYPE);
+    fields[4] = MaterializedField.create(EMPTY_SCHEMA_PATH, RepeatedFloat4Holder.TYPE);
+    fields[5] = MaterializedField.create(EMPTY_SCHEMA_PATH, RepeatedVarBinaryHolder.TYPE);
+
+    fields[6] = MaterializedField.create(EMPTY_SCHEMA_PATH, MapVector.TYPE);
+    fields[6].addChild(fields[0] /*bit*/);
+    fields[6].addChild(fields[2] /*varchar*/);
+
+    fields[7] = MaterializedField.create(EMPTY_SCHEMA_PATH, RepeatedMapVector.TYPE);
+    fields[7].addChild(fields[1] /*int*/);
+    fields[7].addChild(fields[3] /*optional var16char*/);
+
+    fields[8] = MaterializedField.create(EMPTY_SCHEMA_PATH, RepeatedListVector.TYPE);
+    fields[8].addChild(fields[1] /*int*/);
+
+    final int initialCapacity = 1024;
+
+    try {
+      for (int i = 0; i < valueVectors.length; i++) {
+        valueVectors[i] = BasicTypeHelper.getNewVector(fields[i], allocator);
+        valueVectors[i].setInitialCapacity(initialCapacity);
+        valueVectors[i].allocateNew();
+      }
+
+      for (int i = 0; i < valueVectors.length; i++) {
+        final ValueVector vv = valueVectors[i];
+        final int vvCapacity = vv.getValueCapacity();
+
+        // this can't be equality because Nullables will be allocated using power of two sized buffers (thus need 1025
+        // spots in one vector > power of two is 2048, available capacity will be 2048 => 2047)
+        assertTrue(String.format("Incorrect value capacity for %s [%d]", vv.getField(), vvCapacity),
+                initialCapacity <= vvCapacity);
+      }
+    } finally {
+      for (ValueVector v : valueVectors) {
+        v.close();
+      }
+    }
+  }
+
+}


[2/4] arrow git commit: ARROW-18: Fix decimal precision and scale in MapWriters

Posted by sm...@apache.org.
ARROW-18: Fix decimal precision and scale in MapWriters


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/243ed4e9
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/243ed4e9
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/243ed4e9

Branch: refs/heads/master
Commit: 243ed4e91d5ed922b205f7ac5fa8f9f821a07fbb
Parents: 1650026
Author: Steven Phillips <st...@dremio.com>
Authored: Mon Feb 29 19:33:44 2016 -0800
Committer: Steven Phillips <st...@dremio.com>
Committed: Tue Mar 8 16:09:19 2016 -0800

----------------------------------------------------------------------
 java/vector/src/main/codegen/templates/MapWriters.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/243ed4e9/java/vector/src/main/codegen/templates/MapWriters.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/codegen/templates/MapWriters.java b/java/vector/src/main/codegen/templates/MapWriters.java
index 7001367..42f3982 100644
--- a/java/vector/src/main/codegen/templates/MapWriters.java
+++ b/java/vector/src/main/codegen/templates/MapWriters.java
@@ -206,7 +206,7 @@ public class ${mode}MapWriter extends AbstractFieldWriter {
   }
 
   public ${minor.class}Writer ${lowerName}(String name, int scale, int precision) {
-    final MajorType ${upperName}_TYPE = new MajorType(MinorType.${upperName}, DataMode.OPTIONAL, scale, precision, null, null);
+    final MajorType ${upperName}_TYPE = new MajorType(MinorType.${upperName}, DataMode.OPTIONAL, precision, scale, 0, null);
   <#else>
   private static final MajorType ${upperName}_TYPE = Types.optional(MinorType.${upperName});
   @Override


[4/4] arrow git commit: ARROW-17: set some vector fields to package level access for Drill compatibility

Posted by sm...@apache.org.
ARROW-17: set some vector fields to package level access for Drill compatibility


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/16500262
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/16500262
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/16500262

Branch: refs/heads/master
Commit: 1650026285bea52288c7f24720c3caf7cd3ce2a8
Parents: 45cd9fd
Author: Steven Phillips <st...@dremio.com>
Authored: Mon Feb 29 19:32:12 2016 -0800
Committer: Steven Phillips <st...@dremio.com>
Committed: Tue Mar 8 16:09:19 2016 -0800

----------------------------------------------------------------------
 .../main/codegen/templates/BasicTypeHelper.java |  1 +
 .../codegen/templates/NullableValueVectors.java |  6 ++-
 .../codegen/templates/RepeatedValueVectors.java |  2 +-
 .../src/main/codegen/templates/UnionVector.java |  4 +-
 .../templates/VariableLengthVectors.java        |  2 +-
 .../java/org/apache/arrow/vector/BitVector.java |  4 +-
 .../apache/arrow/vector/complex/ListVector.java |  4 +-
 .../apache/arrow/vector/complex/MapVector.java  |  2 +-
 .../vector/complex/RepeatedListVector.java      |  3 +-
 .../arrow/vector/complex/RepeatedMapVector.java |  2 +-
 .../org/apache/arrow/vector/types/Types.java    | 54 ++++++++++++++++----
 11 files changed, 60 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/16500262/java/vector/src/main/codegen/templates/BasicTypeHelper.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/codegen/templates/BasicTypeHelper.java b/java/vector/src/main/codegen/templates/BasicTypeHelper.java
index bb6446e..0bae715 100644
--- a/java/vector/src/main/codegen/templates/BasicTypeHelper.java
+++ b/java/vector/src/main/codegen/templates/BasicTypeHelper.java
@@ -231,6 +231,7 @@ public class BasicTypeHelper {
     return getNewVector(field, allocator, null);
   }
   public static ValueVector getNewVector(MaterializedField field, BufferAllocator allocator, CallBack callBack){
+    field = field.clone();
     MajorType type = field.getType();
 
     switch (type.getMinorType()) {

http://git-wip-us.apache.org/repos/asf/arrow/blob/16500262/java/vector/src/main/codegen/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/codegen/templates/NullableValueVectors.java b/java/vector/src/main/codegen/templates/NullableValueVectors.java
index 6893a25..b0029f7 100644
--- a/java/vector/src/main/codegen/templates/NullableValueVectors.java
+++ b/java/vector/src/main/codegen/templates/NullableValueVectors.java
@@ -45,8 +45,10 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   private final FieldReader reader = new Nullable${minor.class}ReaderImpl(Nullable${minor.class}Vector.this);
 
   private final MaterializedField bitsField = MaterializedField.create("$bits$", new MajorType(MinorType.UINT1, DataMode.REQUIRED));
-  private final UInt1Vector bits = new UInt1Vector(bitsField, allocator);
-  private final ${valuesName} values = new ${minor.class}Vector(field, allocator);
+  private final MaterializedField valuesField = MaterializedField.create("$values$", new MajorType(field.getType().getMinorType(), DataMode.REQUIRED, field.getPrecision(), field.getScale()));
+
+  final UInt1Vector bits = new UInt1Vector(bitsField, allocator);
+  final ${valuesName} values = new ${minor.class}Vector(valuesField, allocator);
 
   private final Mutator mutator = new Mutator();
   private final Accessor accessor = new Accessor();

http://git-wip-us.apache.org/repos/asf/arrow/blob/16500262/java/vector/src/main/codegen/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/codegen/templates/RepeatedValueVectors.java b/java/vector/src/main/codegen/templates/RepeatedValueVectors.java
index 5ac80f5..ceae53b 100644
--- a/java/vector/src/main/codegen/templates/RepeatedValueVectors.java
+++ b/java/vector/src/main/codegen/templates/RepeatedValueVectors.java
@@ -42,7 +42,7 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
   //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Repeated${minor.class}Vector.class);
 
   // we maintain local reference to concrete vector type for performance reasons.
-  private ${minor.class}Vector values;
+  ${minor.class}Vector values;
   private final FieldReader reader = new Repeated${minor.class}ReaderImpl(Repeated${minor.class}Vector.this);
   private final Mutator mutator = new Mutator();
   private final Accessor accessor = new Accessor();

http://git-wip-us.apache.org/repos/asf/arrow/blob/16500262/java/vector/src/main/codegen/templates/UnionVector.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/codegen/templates/UnionVector.java b/java/vector/src/main/codegen/templates/UnionVector.java
index ba94ac2..6042a5b 100644
--- a/java/vector/src/main/codegen/templates/UnionVector.java
+++ b/java/vector/src/main/codegen/templates/UnionVector.java
@@ -51,9 +51,9 @@ public class UnionVector implements ValueVector {
   private BufferAllocator allocator;
   private Accessor accessor = new Accessor();
   private Mutator mutator = new Mutator();
-  private int valueCount;
+  int valueCount;
 
-  private MapVector internalMap;
+  MapVector internalMap;
   private UInt1Vector typeVector;
 
   private MapVector mapVector;

http://git-wip-us.apache.org/repos/asf/arrow/blob/16500262/java/vector/src/main/codegen/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/codegen/templates/VariableLengthVectors.java b/java/vector/src/main/codegen/templates/VariableLengthVectors.java
index 13d53b8..84fb3eb 100644
--- a/java/vector/src/main/codegen/templates/VariableLengthVectors.java
+++ b/java/vector/src/main/codegen/templates/VariableLengthVectors.java
@@ -57,7 +57,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
 
   public final static String OFFSETS_VECTOR_NAME = "$offsets$";
   private final MaterializedField offsetsField = MaterializedField.create(OFFSETS_VECTOR_NAME, new MajorType(MinorType.UINT4, DataMode.REQUIRED));
-  private final UInt${type.width}Vector offsetVector = new UInt${type.width}Vector(offsetsField, allocator);
+  final UInt${type.width}Vector offsetVector = new UInt${type.width}Vector(offsetsField, allocator);
   private final FieldReader reader = new ${minor.class}ReaderImpl(${minor.class}Vector.this);
 
   private final Accessor accessor;

http://git-wip-us.apache.org/repos/asf/arrow/blob/16500262/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java
index 952e902..c5bcb2d 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java
@@ -41,7 +41,7 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
   private final Accessor accessor = new Accessor();
   private final Mutator mutator = new Mutator();
 
-  private int valueCount;
+  int valueCount;
   private int allocationSizeInBytes = INITIAL_VALUE_ALLOCATION;
   private int allocationMonitor = 0;
 
@@ -64,7 +64,7 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
     return getSizeFromCount(valueCount);
   }
 
-  private int getSizeFromCount(int valueCount) {
+  int getSizeFromCount(int valueCount) {
     return (int) Math.ceil(valueCount / 8.0);
   }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/16500262/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
index 8387c9e..13610c4 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
@@ -47,8 +47,8 @@ import com.google.common.collect.ObjectArrays;
 
 public class ListVector extends BaseRepeatedValueVector {
 
-  private UInt4Vector offsets;
-  private final UInt1Vector bits;
+  UInt4Vector offsets;
+  final UInt1Vector bits;
   private Mutator mutator = new Mutator();
   private Accessor accessor = new Accessor();
   private UnionListWriter writer;

http://git-wip-us.apache.org/repos/asf/arrow/blob/16500262/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
index 1bbce73..cc0953a 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
@@ -52,7 +52,7 @@ public class MapVector extends AbstractMapVector {
   private final SingleMapReaderImpl reader = new SingleMapReaderImpl(MapVector.this);
   private final Accessor accessor = new Accessor();
   private final Mutator mutator = new Mutator();
-  private int valueCount;
+  int valueCount;
 
   public MapVector(String path, BufferAllocator allocator, CallBack callBack){
     this(MaterializedField.create(path, TYPE), allocator, callBack);

http://git-wip-us.apache.org/repos/asf/arrow/blob/16500262/java/vector/src/main/java/org/apache/arrow/vector/complex/RepeatedListVector.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/RepeatedListVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/RepeatedListVector.java
index 778fe81..f337f9c 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/RepeatedListVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/RepeatedListVector.java
@@ -49,7 +49,7 @@ public class RepeatedListVector extends AbstractContainerVector
 
   public final static MajorType TYPE = new MajorType(MinorType.LIST, DataMode.REPEATED);
   private final RepeatedListReaderImpl reader = new RepeatedListReaderImpl(null, this);
-  private final DelegateRepeatedVector delegate;
+  final DelegateRepeatedVector delegate;
 
   protected static class DelegateRepeatedVector extends BaseRepeatedValueVector {
 
@@ -313,7 +313,6 @@ public class RepeatedListVector extends AbstractContainerVector
     if (result.isCreated() && callBack != null) {
       callBack.doWork();
     }
-    this.field = delegate.getField();
     return result;
   }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/16500262/java/vector/src/main/java/org/apache/arrow/vector/complex/RepeatedMapVector.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/RepeatedMapVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/RepeatedMapVector.java
index e7eacd3..686414e 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/RepeatedMapVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/RepeatedMapVector.java
@@ -53,7 +53,7 @@ public class RepeatedMapVector extends AbstractMapVector
 
   public final static MajorType TYPE = new MajorType(MinorType.MAP, DataMode.REPEATED);
 
-  private final UInt4Vector offsets;   // offsets to start of each record (considering record indices are 0-indexed)
+  final UInt4Vector offsets;   // offsets to start of each record (considering record indices are 0-indexed)
   private final RepeatedMapReaderImpl reader = new RepeatedMapReaderImpl(RepeatedMapVector.this);
   private final RepeatedMapAccessor accessor = new RepeatedMapAccessor();
   private final Mutator mutator = new Mutator();

http://git-wip-us.apache.org/repos/asf/arrow/blob/16500262/java/vector/src/main/java/org/apache/arrow/vector/types/Types.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/types/Types.java b/java/vector/src/main/java/org/apache/arrow/vector/types/Types.java
index cef892c..88999cb 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/types/Types.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/types/Types.java
@@ -20,6 +20,7 @@ package org.apache.arrow.vector.types;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 public class Types {
   public enum MinorType {
@@ -73,26 +74,35 @@ public class Types {
   public static class MajorType {
     private MinorType minorType;
     private DataMode mode;
-    private Integer precision;
-    private Integer scale;
-    private Integer timezone;
+    private int precision;
+    private int scale;
+    private int timezone;
+    private int width;
     private List<MinorType> subTypes;
 
     public MajorType(MinorType minorType, DataMode mode) {
-      this(minorType, mode, null, null, null, null);
+      this(minorType, mode, 0, 0, 0, 0, null);
     }
 
-    public MajorType(MinorType minorType, DataMode mode, Integer precision, Integer scale) {
-      this(minorType, mode, precision, scale, null, null);
+    public MajorType(MinorType minorType, DataMode mode, int precision, int scale) {
+      this(minorType, mode, precision, scale, 0, 0, null);
     }
 
-    public MajorType(MinorType minorType, DataMode mode, Integer precision, Integer scale, Integer timezone, List<MinorType> subTypes) {
+    public MajorType(MinorType minorType, DataMode mode, int precision, int scale, int timezone, List<MinorType> subTypes) {
+      this(minorType, mode, precision, scale, timezone, 0, subTypes);
+    }
+
+    public MajorType(MinorType minorType, DataMode mode, int precision, int scale, int timezone, int width, List<MinorType> subTypes) {
       this.minorType = minorType;
       this.mode = mode;
       this.precision = precision;
       this.scale = scale;
       this.timezone = timezone;
+      this.width = width;
       this.subTypes = subTypes;
+      if (subTypes == null) {
+        this.subTypes = new ArrayList<>();
+      }
     }
 
     public MinorType getMinorType() {
@@ -103,21 +113,45 @@ public class Types {
       return mode;
     }
 
-    public Integer getPrecision() {
+    public int getPrecision() {
       return precision;
     }
 
-    public Integer getScale() {
+    public int getScale() {
       return scale;
     }
 
-    public Integer getTimezone() {
+    public int getTimezone() {
       return timezone;
     }
 
     public List<MinorType> getSubTypes() {
       return subTypes;
     }
+
+    public int getWidth() {
+      return width;
+    }
+
+
+    @Override
+    public boolean equals(Object other) {
+      if (other == null) {
+        return false;
+      }
+      if (!(other instanceof MajorType)) {
+        return false;
+      }
+      MajorType that = (MajorType) other;
+      return this.minorType == that.minorType &&
+              this.mode == that.mode &&
+              this.precision == that.precision &&
+              this.scale == that.scale &&
+              this.timezone == that.timezone &&
+              this.width == that.width &&
+              Objects.equals(this.subTypes, that.subTypes);
+    }
+
   }
 
   public static MajorType required(MinorType minorType) {