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

arrow git commit: ARROW-246: [Java] UnionVector doesn't call allocateNew() when creating it's vectorType

Repository: arrow
Updated Branches:
  refs/heads/master 3a2dfba59 -> 56835c338


ARROW-246: [Java] UnionVector doesn't call allocateNew() when creating it's vectorType


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

Branch: refs/heads/master
Commit: 56835c338f01aebcace01312e82431306e7fd578
Parents: 3a2dfba
Author: adeneche <ad...@apache.org>
Authored: Mon Aug 1 15:28:08 2016 -0700
Committer: AbdelHakim Deneche <ad...@Jacquess-MacBook-Pro.local>
Committed: Tue Aug 2 18:31:20 2016 -0700

----------------------------------------------------------------------
 .../src/main/codegen/templates/UnionVector.java |   2 +
 .../arrow/vector/DirtyBufferAllocator.java      | 120 +++++++++++++++++++
 .../apache/arrow/vector/TestUnionVector.java    |  88 ++++++++++++++
 3 files changed, 210 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/56835c33/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 4829448..692436d 100644
--- a/java/vector/src/main/codegen/templates/UnionVector.java
+++ b/java/vector/src/main/codegen/templates/UnionVector.java
@@ -73,6 +73,8 @@ public class UnionVector implements ValueVector {
     this.allocator = allocator;
     this.internalMap = new MapVector("internal", allocator, callBack);
     this.typeVector = internalMap.addOrGet("types", new MajorType(MinorType.UINT1, DataMode.REQUIRED), UInt1Vector.class);
+    this.typeVector.allocateNew();
+    this.typeVector.zeroVector();
     this.field.addChild(internalMap.getField().clone());
     this.majorType = field.getType();
     this.callBack = callBack;

http://git-wip-us.apache.org/repos/asf/arrow/blob/56835c33/java/vector/src/test/java/org/apache/arrow/vector/DirtyBufferAllocator.java
----------------------------------------------------------------------
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/DirtyBufferAllocator.java b/java/vector/src/test/java/org/apache/arrow/vector/DirtyBufferAllocator.java
new file mode 100644
index 0000000..cc6b9ec
--- /dev/null
+++ b/java/vector/src/test/java/org/apache/arrow/vector/DirtyBufferAllocator.java
@@ -0,0 +1,120 @@
+/**
+ * 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 org.apache.arrow.memory.AllocationReservation;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.BufferManager;
+
+import io.netty.buffer.ArrowBuf;
+import io.netty.buffer.ByteBufAllocator;
+
+/**
+ * Wrapper around a buffer delegate that populates any allocated buffer with a constant
+ * value. Useful for testing if value vectors are properly resetting their buffers.
+ */
+public class DirtyBufferAllocator implements BufferAllocator {
+
+  private final BufferAllocator delegate;
+  private final byte fillValue;
+
+  DirtyBufferAllocator(final BufferAllocator delegate, final byte fillValue) {
+    this.delegate = delegate;
+    this.fillValue = fillValue;
+  }
+
+  @Override
+  public ArrowBuf buffer(int size) {
+    return buffer(size, null);
+  }
+
+  @Override
+  public ArrowBuf buffer(int size, BufferManager manager) {
+    ArrowBuf buffer = delegate.buffer(size, manager);
+    // contaminate the buffer
+    for (int i = 0; i < buffer.capacity(); i++) {
+      buffer.setByte(i, fillValue);
+    }
+
+    return buffer;
+  }
+
+  @Override
+  public ByteBufAllocator getAsByteBufAllocator() {
+    return delegate.getAsByteBufAllocator();
+  }
+
+  @Override
+  public BufferAllocator newChildAllocator(String name, long initReservation, long maxAllocation) {
+    return delegate.newChildAllocator(name, initReservation, maxAllocation);
+  }
+
+  @Override
+  public void close() {
+    delegate.close();
+  }
+
+  @Override
+  public long getAllocatedMemory() {
+    return delegate.getAllocatedMemory();
+  }
+
+  @Override
+  public void setLimit(long newLimit) {
+    delegate.setLimit(newLimit);
+  }
+
+  @Override
+  public long getLimit() {
+    return delegate.getLimit();
+  }
+
+  @Override
+  public long getPeakMemoryAllocation() {
+    return delegate.getPeakMemoryAllocation();
+  }
+
+  @Override
+  public AllocationReservation newReservation() {
+    return delegate.newReservation();
+  }
+
+  @Override
+  public ArrowBuf getEmpty() {
+    return delegate.getEmpty();
+  }
+
+  @Override
+  public String getName() {
+    return delegate.getName();
+  }
+
+  @Override
+  public boolean isOverLimit() {
+    return delegate.isOverLimit();
+  }
+
+  @Override
+  public String toVerboseString() {
+    return delegate.toVerboseString();
+  }
+
+  @Override
+  public void assertOpen() {
+    delegate.assertOpen();
+  }}

http://git-wip-us.apache.org/repos/asf/arrow/blob/56835c33/java/vector/src/test/java/org/apache/arrow/vector/TestUnionVector.java
----------------------------------------------------------------------
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestUnionVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestUnionVector.java
new file mode 100644
index 0000000..8f19b31
--- /dev/null
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestUnionVector.java
@@ -0,0 +1,88 @@
+/**
+ * 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.assertEquals;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.complex.UnionVector;
+import org.apache.arrow.vector.holders.NullableUInt4Holder;
+import org.apache.arrow.vector.holders.UInt4Holder;
+import org.apache.arrow.vector.types.MaterializedField;
+import org.apache.arrow.vector.types.Types;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestUnionVector {
+  private final static String EMPTY_SCHEMA_PATH = "";
+
+  private BufferAllocator allocator;
+
+  @Before
+  public void init() {
+    allocator = new RootAllocator(Long.MAX_VALUE);
+  }
+
+  @After
+  public void terminate() throws Exception {
+    allocator.close();
+  }
+
+  @Test
+  public void testUnionVector() throws Exception {
+    final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, UInt4Holder.TYPE);
+
+    final BufferAllocator alloc = new DirtyBufferAllocator(allocator, (byte) 100);
+
+    UnionVector unionVector = new UnionVector(field, alloc, null);
+
+    final NullableUInt4Holder uInt4Holder = new NullableUInt4Holder();
+    uInt4Holder.value = 100;
+    uInt4Holder.isSet = 1;
+
+    try {
+      // write some data
+      final UnionVector.Mutator mutator = unionVector.getMutator();
+      mutator.setType(0, Types.MinorType.UINT4);
+      mutator.setSafe(0, uInt4Holder);
+      mutator.setType(2, Types.MinorType.UINT4);
+      mutator.setSafe(2, uInt4Holder);
+      mutator.setValueCount(4);
+
+      // check that what we wrote is correct
+      final UnionVector.Accessor accessor = unionVector.getAccessor();
+      assertEquals(4, accessor.getValueCount());
+
+      assertEquals(false, accessor.isNull(0));
+      assertEquals(100, accessor.getObject(0));
+
+      assertEquals(true, accessor.isNull(1));
+
+      assertEquals(false, accessor.isNull(2));
+      assertEquals(100, accessor.getObject(2));
+
+      assertEquals(true, accessor.isNull(3));
+
+    } finally {
+      unionVector.clear();
+    }
+  }
+
+}