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/08/18 23:30:35 UTC

[2/5] arrow git commit: ARROW-259: Use Flatbuffer Field type instead of MaterializedField

http://git-wip-us.apache.org/repos/asf/arrow/blob/e7e399db/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleListReaderImpl.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleListReaderImpl.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleListReaderImpl.java
index f16f628..b8f5865 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleListReaderImpl.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleListReaderImpl.java
@@ -24,14 +24,11 @@ import org.apache.arrow.vector.complex.AbstractContainerVector;
 import org.apache.arrow.vector.complex.reader.FieldReader;
 import org.apache.arrow.vector.complex.writer.BaseWriter.ListWriter;
 import org.apache.arrow.vector.complex.writer.BaseWriter.MapWriter;
-import org.apache.arrow.vector.types.Types;
-import org.apache.arrow.vector.types.Types.MajorType;
 import org.apache.arrow.vector.types.Types.MinorType;
 
 @SuppressWarnings("unused")
 public class SingleListReaderImpl extends AbstractFieldReader{
 
-  private static final MajorType TYPE = Types.optional(MinorType.LIST);
   private final String name;
   private final AbstractContainerVector container;
   private FieldReader reader;
@@ -43,12 +40,6 @@ public class SingleListReaderImpl extends AbstractFieldReader{
   }
 
   @Override
-  public MajorType getType() {
-    return TYPE;
-  }
-
-
-  @Override
   public void setPosition(int index) {
     super.setPosition(index);
     if (reader != null) {
@@ -71,6 +62,11 @@ public class SingleListReaderImpl extends AbstractFieldReader{
   }
 
   @Override
+  public MinorType getMinorType() {
+    return MinorType.LIST;
+  }
+
+  @Override
   public boolean isSet() {
     return false;
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/e7e399db/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java
index 84b9980..1c43240 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java
@@ -27,9 +27,9 @@ import org.apache.arrow.vector.ValueVector;
 import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.complex.reader.FieldReader;
 import org.apache.arrow.vector.complex.writer.BaseWriter.MapWriter;
-import org.apache.arrow.vector.types.Types.MajorType;
 
 import com.google.common.collect.Maps;
+import org.apache.arrow.vector.types.Types.MinorType;
 
 @SuppressWarnings("unused")
 public class SingleMapReaderImpl extends AbstractFieldReader{
@@ -77,13 +77,13 @@ public class SingleMapReaderImpl extends AbstractFieldReader{
   }
 
   @Override
-  public boolean isSet() {
-    return true;
+  public MinorType getMinorType() {
+    return MinorType.MAP;
   }
 
   @Override
-  public MajorType getType(){
-    return vector.getField().getType();
+  public boolean isSet() {
+    return true;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/arrow/blob/e7e399db/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java
index 9b54d02..39cf004 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java
@@ -25,8 +25,6 @@ import org.apache.arrow.vector.complex.reader.FieldReader;
 import org.apache.arrow.vector.complex.writer.BaseWriter.ListWriter;
 import org.apache.arrow.vector.complex.writer.FieldWriter;
 import org.apache.arrow.vector.holders.UnionHolder;
-import org.apache.arrow.vector.types.Types.DataMode;
-import org.apache.arrow.vector.types.Types.MajorType;
 import org.apache.arrow.vector.types.Types.MinorType;
 
 public class UnionListReader extends AbstractFieldReader {
@@ -46,12 +44,6 @@ public class UnionListReader extends AbstractFieldReader {
     return true;
   }
 
-  MajorType type = new MajorType(MinorType.LIST, DataMode.OPTIONAL);
-
-  public MajorType getType() {
-    return type;
-  }
-
   private int currentOffset;
   private int maxOffset;
 
@@ -73,6 +65,11 @@ public class UnionListReader extends AbstractFieldReader {
   }
 
   @Override
+  public MinorType getMinorType() {
+    return MinorType.LIST;
+  }
+
+  @Override
   public void read(int index, UnionHolder holder) {
     setPosition(idx());
     for (int i = -1; i < index; i++) {
@@ -83,6 +80,12 @@ public class UnionListReader extends AbstractFieldReader {
   }
 
   @Override
+  public int size() {
+    int size = maxOffset - currentOffset - 1;
+    return size < 0 ? 0 : size;
+  }
+
+  @Override
   public boolean next() {
     if (currentOffset + 1 < maxOffset) {
       data.getReader().setPosition(++currentOffset);

http://git-wip-us.apache.org/repos/asf/arrow/blob/e7e399db/java/vector/src/main/java/org/apache/arrow/vector/holders/ObjectHolder.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/holders/ObjectHolder.java b/java/vector/src/main/java/org/apache/arrow/vector/holders/ObjectHolder.java
deleted file mode 100644
index 5a5fe03..0000000
--- a/java/vector/src/main/java/org/apache/arrow/vector/holders/ObjectHolder.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * 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.holders;
-
-import org.apache.arrow.vector.types.Types;
-
-/*
- * Holder class for the vector ObjectVector. This holder internally stores a
- * reference to an object. The ObjectVector maintains an array of these objects.
- * This holder can be used only as workspace variables in aggregate functions.
- * Using this holder should be avoided and we should stick to native holder types.
- */
-@Deprecated
-public class ObjectHolder implements ValueHolder {
-  public static final Types.MajorType TYPE = Types.required(Types.MinorType.GENERIC_OBJECT);
-
-  public Types.MajorType getType() {
-    return TYPE;
-  }
-
-  public Object obj;
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/e7e399db/java/vector/src/main/java/org/apache/arrow/vector/holders/UnionHolder.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/holders/UnionHolder.java b/java/vector/src/main/java/org/apache/arrow/vector/holders/UnionHolder.java
index b868a62..b1b695e 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/holders/UnionHolder.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/holders/UnionHolder.java
@@ -18,17 +18,14 @@
 package org.apache.arrow.vector.holders;
 
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.types.Types.DataMode;
-import org.apache.arrow.vector.types.Types.MajorType;
 import org.apache.arrow.vector.types.Types.MinorType;
 
 public class UnionHolder implements ValueHolder {
-  public static final MajorType TYPE = new MajorType(MinorType.UNION, DataMode.OPTIONAL);
   public FieldReader reader;
   public int isSet;
 
-  public MajorType getType() {
-    return reader.getType();
+  public MinorType getMinorType() {
+    return reader.getMinorType();
   }
 
   public boolean isSet() {

http://git-wip-us.apache.org/repos/asf/arrow/blob/e7e399db/java/vector/src/main/java/org/apache/arrow/vector/types/MaterializedField.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/types/MaterializedField.java b/java/vector/src/main/java/org/apache/arrow/vector/types/MaterializedField.java
deleted file mode 100644
index c73098b..0000000
--- a/java/vector/src/main/java/org/apache/arrow/vector/types/MaterializedField.java
+++ /dev/null
@@ -1,217 +0,0 @@
-/**
- * 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.types;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.LinkedHashSet;
-import java.util.Objects;
-
-import org.apache.arrow.vector.types.Types.DataMode;
-import org.apache.arrow.vector.types.Types.MajorType;
-import org.apache.arrow.vector.util.BasicTypeHelper;
-
-
-public class MaterializedField {
-  private final String name;
-  private final MajorType type;
-  // use an ordered set as existing code relies on order (e,g. parquet writer)
-  private final LinkedHashSet<MaterializedField> children;
-
-  MaterializedField(String name, MajorType type, LinkedHashSet<MaterializedField> children) {
-    this.name = name;
-    this.type = type;
-    this.children = children;
-  }
-
-  public Collection<MaterializedField> getChildren() {
-    return new ArrayList<>(children);
-  }
-
-  public MaterializedField newWithChild(MaterializedField child) {
-    MaterializedField newField = clone();
-    newField.addChild(child);
-    return newField;
-  }
-
-  public void addChild(MaterializedField field){
-    children.add(field);
-  }
-
-  public MaterializedField clone() {
-    return withPathAndType(name, getType());
-  }
-
-  public MaterializedField withType(MajorType type) {
-    return withPathAndType(name, type);
-  }
-
-  public MaterializedField withPath(String name) {
-    return withPathAndType(name, getType());
-  }
-
-  public MaterializedField withPathAndType(String name, final MajorType type) {
-    final LinkedHashSet<MaterializedField> newChildren = new LinkedHashSet<>(children.size());
-    for (final MaterializedField child:children) {
-      newChildren.add(child.clone());
-    }
-    return new MaterializedField(name, type, newChildren);
-  }
-
-//  public String getLastName(){
-//    PathSegment seg = key.path.getRootSegment();
-//    while (seg.getChild() != null) {
-//      seg = seg.getChild();
-//    }
-//    return seg.getNameSegment().getPath();
-//  }
-
-
-  // TODO: rewrite without as direct match rather than conversion then match.
-//  public boolean matches(SerializedField booleanfield){
-//    MaterializedField f = create(field);
-//    return f.equals(this);
-//  }
-
-  public static MaterializedField create(String name, MajorType type){
-    return new MaterializedField(name, type, new LinkedHashSet<MaterializedField>());
-  }
-
-//  public String getName(){
-//    StringBuilder sb = new StringBuilder();
-//    boolean first = true;
-//    for(NamePart np : def.getNameList()){
-//      if(np.getType() == Type.ARRAY){
-//        sb.append("[]");
-//      }else{
-//        if(first){
-//          first = false;
-//        }else{
-//          sb.append(".");
-//        }
-//        sb.append('`');
-//        sb.append(np.getName());
-//        sb.append('`');
-//
-//      }
-//    }
-//    return sb.toString();
-//  }
-
-  public String getPath() {
-    return getName();
-  }
-
-  public String getLastName() {
-    return getName();
-  }
-
-  public String getName() {
-    return name;
-  }
-
-//  public int getWidth() {
-//    return type.getWidth();
-//  }
-
-  public MajorType getType() {
-    return type;
-  }
-
-  public int getScale() {
-      return type.getScale();
-  }
-  public int getPrecision() {
-      return type.getPrecision();
-  }
-  public boolean isNullable() {
-    return type.getMode() == DataMode.OPTIONAL;
-  }
-
-  public DataMode getDataMode() {
-    return type.getMode();
-  }
-
-  public MaterializedField getOtherNullableVersion(){
-    MajorType mt = type;
-    DataMode newDataMode = null;
-    switch (mt.getMode()){
-    case OPTIONAL:
-      newDataMode = DataMode.REQUIRED;
-      break;
-    case REQUIRED:
-      newDataMode = DataMode.OPTIONAL;
-      break;
-    default:
-      throw new UnsupportedOperationException();
-    }
-    return new MaterializedField(name, new MajorType(mt.getMinorType(), newDataMode, mt.getPrecision(), mt.getScale(), mt.getTimezone(), mt.getSubTypes()), children);
-  }
-
-  public Class<?> getValueClass() {
-    return BasicTypeHelper.getValueVectorClass(getType().getMinorType(), getDataMode());
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(this.name, this.type, this.children);
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    MaterializedField other = (MaterializedField) obj;
-    // DRILL-1872: Compute equals only on key. See also the comment
-    // in MapVector$MapTransferPair
-
-    return this.name.equalsIgnoreCase(other.name) &&
-            Objects.equals(this.type, other.type);
-  }
-
-
-  @Override
-  public String toString() {
-    final int maxLen = 10;
-    String childStr = children != null && !children.isEmpty() ? toString(children, maxLen) : "";
-    return name + "(" + type.getMinorType().name() + ":" + type.getMode().name() + ")" + childStr;
-  }
-
-
-  private String toString(Collection<?> collection, int maxLen) {
-    StringBuilder builder = new StringBuilder();
-    builder.append("[");
-    int i = 0;
-    for (Iterator<?> iterator = collection.iterator(); iterator.hasNext() && i < maxLen; i++) {
-      if (i > 0){
-        builder.append(", ");
-      }
-      builder.append(iterator.next());
-    }
-    builder.append("]");
-    return builder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/e7e399db/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 88999cb..5ea1456 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
@@ -17,150 +17,508 @@
  */
 package org.apache.arrow.vector.types;
 
-import java.util.ArrayList;
-import java.util.List;
+import org.apache.arrow.flatbuf.Type;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.NullableBigIntVector;
+import org.apache.arrow.vector.NullableBitVector;
+import org.apache.arrow.vector.NullableDateVector;
+import org.apache.arrow.vector.NullableDecimalVector;
+import org.apache.arrow.vector.NullableFloat4Vector;
+import org.apache.arrow.vector.NullableFloat8Vector;
+import org.apache.arrow.vector.NullableIntVector;
+import org.apache.arrow.vector.NullableIntervalDayVector;
+import org.apache.arrow.vector.NullableIntervalYearVector;
+import org.apache.arrow.vector.NullableSmallIntVector;
+import org.apache.arrow.vector.NullableTimeStampVector;
+import org.apache.arrow.vector.NullableTimeVector;
+import org.apache.arrow.vector.NullableTinyIntVector;
+import org.apache.arrow.vector.NullableUInt1Vector;
+import org.apache.arrow.vector.NullableUInt2Vector;
+import org.apache.arrow.vector.NullableUInt4Vector;
+import org.apache.arrow.vector.NullableUInt8Vector;
+import org.apache.arrow.vector.NullableVarBinaryVector;
+import org.apache.arrow.vector.NullableVarCharVector;
+import org.apache.arrow.vector.SmallIntVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.ZeroVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.UnionVector;
+import org.apache.arrow.vector.complex.impl.BigIntWriterImpl;
+import org.apache.arrow.vector.complex.impl.BitWriterImpl;
+import org.apache.arrow.vector.complex.impl.DateWriterImpl;
+import org.apache.arrow.vector.complex.impl.Float4WriterImpl;
+import org.apache.arrow.vector.complex.impl.Float8WriterImpl;
+import org.apache.arrow.vector.complex.impl.IntWriterImpl;
+import org.apache.arrow.vector.complex.impl.IntervalDayWriterImpl;
+import org.apache.arrow.vector.complex.impl.IntervalYearWriterImpl;
+import org.apache.arrow.vector.complex.impl.SingleMapWriter;
+import org.apache.arrow.vector.complex.impl.SmallIntWriterImpl;
+import org.apache.arrow.vector.complex.impl.TimeStampWriterImpl;
+import org.apache.arrow.vector.complex.impl.TimeWriterImpl;
+import org.apache.arrow.vector.complex.impl.TinyIntWriterImpl;
+import org.apache.arrow.vector.complex.impl.UInt1WriterImpl;
+import org.apache.arrow.vector.complex.impl.UInt2WriterImpl;
+import org.apache.arrow.vector.complex.impl.UInt4WriterImpl;
+import org.apache.arrow.vector.complex.impl.UInt8WriterImpl;
+import org.apache.arrow.vector.complex.impl.UnionListWriter;
+import org.apache.arrow.vector.complex.impl.UnionWriter;
+import org.apache.arrow.vector.complex.impl.VarBinaryWriterImpl;
+import org.apache.arrow.vector.complex.impl.VarCharWriterImpl;
+import org.apache.arrow.vector.complex.writer.FieldWriter;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.ArrowType.Binary;
+import org.apache.arrow.vector.types.pojo.ArrowType.Bool;
+import org.apache.arrow.vector.types.pojo.ArrowType.Date;
+import org.apache.arrow.vector.types.pojo.ArrowType.FloatingPoint;
+import org.apache.arrow.vector.types.pojo.ArrowType.Int;
+import org.apache.arrow.vector.types.pojo.ArrowType.IntervalDay;
+import org.apache.arrow.vector.types.pojo.ArrowType.IntervalYear;
+import org.apache.arrow.vector.types.pojo.ArrowType.List;
+import org.apache.arrow.vector.types.pojo.ArrowType.Null;
+import org.apache.arrow.vector.types.pojo.ArrowType.Time;
+import org.apache.arrow.vector.types.pojo.ArrowType.Timestamp;
+import org.apache.arrow.vector.types.pojo.ArrowType.Tuple;
+import org.apache.arrow.vector.types.pojo.ArrowType.Union;
+import org.apache.arrow.vector.types.pojo.ArrowType.Utf8;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.util.CallBack;
+
+import java.util.HashMap;
 import java.util.Map;
-import java.util.Objects;
 
 public class Types {
+
+  public static final Field NULL_FIELD = new Field("", true, Null.INSTANCE, null);
+  public static final Field TINYINT_FIELD = new Field("", true, new Int(8, true), null);
+  public static final Field SMALLINT_FIELD = new Field("", true, new Int(16, true), null);
+  public static final Field INT_FIELD = new Field("", true, new Int(32, true), null);
+  public static final Field BIGINT_FIELD = new Field("", true, new Int(64, true), null);
+  public static final Field UINT1_FIELD = new Field("", true, new Int(8, false), null);
+  public static final Field UINT2_FIELD = new Field("", true, new Int(16, false), null);
+  public static final Field UINT4_FIELD = new Field("", true, new Int(32, false), null);
+  public static final Field UINT8_FIELD = new Field("", true, new Int(64, false), null);
+  public static final Field DATE_FIELD = new Field("", true, Date.INSTANCE, null);
+  public static final Field TIME_FIELD = new Field("", true, Time.INSTANCE, null);
+  public static final Field TIMESTAMP_FIELD = new Field("", true, new Timestamp(""), null);
+  public static final Field INTERVALDAY_FIELD = new Field("", true, IntervalDay.INSTANCE, null);
+  public static final Field INTERVALYEAR_FIELD = new Field("", true, IntervalYear.INSTANCE, null);
+  public static final Field FLOAT4_FIELD = new Field("", true, new FloatingPoint(0), null);
+  public static final Field FLOAT8_FIELD = new Field("", true, new FloatingPoint(1), null);
+  public static final Field LIST_FIELD = new Field("", true, List.INSTANCE, null);
+  public static final Field VARCHAR_FIELD = new Field("", true, Utf8.INSTANCE, null);
+  public static final Field VARBINARY_FIELD = new Field("", true, Binary.INSTANCE, null);
+  public static final Field BIT_FIELD = new Field("", true, Bool.INSTANCE, null);
+
+
   public enum MinorType {
-    LATE,   //  late binding type
-    MAP,   //  an empty map column.  Useful for conceptual setup.  Children listed within here
-
-    TINYINT,   //  single byte signed integer
-    SMALLINT,   //  two byte signed integer
-    INT,   //  four byte signed integer
-    BIGINT,   //  eight byte signed integer
-    DECIMAL9,   //  a decimal supporting precision between 1 and 9
-    DECIMAL18,   //  a decimal supporting precision between 10 and 18
-    DECIMAL28SPARSE,   //  a decimal supporting precision between 19 and 28
-    DECIMAL38SPARSE,   //  a decimal supporting precision between 29 and 38
-    MONEY,   //  signed decimal with two digit precision
-    DATE,   //  days since 4713bc
-    TIME,   //  time in micros before or after 2000/1/1
-    TIMETZ,  //  time in micros before or after 2000/1/1 with timezone
-    TIMESTAMPTZ,   //  unix epoch time in millis
-    TIMESTAMP,   //  TBD
-    INTERVAL,   //  TBD
-    FLOAT4,   //  4 byte ieee 754
-    FLOAT8,   //  8 byte ieee 754
-    BIT,  //  single bit value (boolean)
-    FIXEDCHAR,  //  utf8 fixed length string, padded with spaces
-    FIXED16CHAR,
-    FIXEDBINARY,   //  fixed length binary, padded with 0 bytes
-    VARCHAR,   //  utf8 variable length string
-    VAR16CHAR, // utf16 variable length string
-    VARBINARY,   //  variable length binary
-    UINT1,  //  unsigned 1 byte integer
-    UINT2,  //  unsigned 2 byte integer
-    UINT4,   //  unsigned 4 byte integer
-    UINT8,   //  unsigned 8 byte integer
-    DECIMAL28DENSE, // dense decimal representation, supporting precision between 19 and 28
-    DECIMAL38DENSE, // dense decimal representation, supporting precision between 28 and 38
-    NULL, // a value of unknown type (e.g. a missing reference).
-    INTERVALYEAR, // Interval type specifying YEAR to MONTH
-    INTERVALDAY, // Interval type specifying DAY to SECONDS
-    LIST,
-    GENERIC_OBJECT,
-    UNION
-  }
+    NULL(Null.INSTANCE) {
+      @Override
+      public Field getField() {
+        return NULL_FIELD;
+      }
 
-  public enum DataMode {
-    REQUIRED,
-    OPTIONAL,
-    REPEATED
-  }
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return ZeroVector.INSTANCE;
+      }
 
-  public static class MajorType {
-    private MinorType minorType;
-    private DataMode mode;
-    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, 0, 0, 0, 0, null);
-    }
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return null;
+      }
+    },
+    MAP(Tuple.INSTANCE) {
+      @Override
+      public Field getField() {
+        throw new UnsupportedOperationException("Cannot get simple field for Map type");
+      }
 
-    public MajorType(MinorType minorType, DataMode mode, int precision, int scale) {
-      this(minorType, mode, precision, scale, 0, 0, null);
-    }
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+         return new MapVector(name, allocator, callBack);
+      }
 
-    public MajorType(MinorType minorType, DataMode mode, int precision, int scale, int timezone, List<MinorType> subTypes) {
-      this(minorType, mode, precision, scale, timezone, 0, subTypes);
-    }
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new SingleMapWriter((MapVector) vector);
+      }
+    },   //  an empty map column.  Useful for conceptual setup.  Children listed within here
 
-    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<>();
+    TINYINT(new Int(8, true)) {
+      @Override
+      public Field getField() {
+        return TINYINT_FIELD;
       }
-    }
 
-    public MinorType getMinorType() {
-      return minorType;
-    }
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableTinyIntVector(name, allocator);
+      }
 
-    public DataMode getMode() {
-      return mode;
-    }
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new TinyIntWriterImpl((NullableTinyIntVector) vector);
+      }
+    },   //  single byte signed integer
+    SMALLINT(new Int(16, true)) {
+      @Override
+      public Field getField() {
+        return SMALLINT_FIELD;
+      }
 
-    public int getPrecision() {
-      return precision;
-    }
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new SmallIntVector(name, allocator);
+      }
 
-    public int getScale() {
-      return scale;
-    }
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new SmallIntWriterImpl((NullableSmallIntVector) vector);
+      }
+    },   //  two byte signed integer
+    INT(new Int(32, true)) {
+      @Override
+      public Field getField() {
+        return INT_FIELD;
+      }
 
-    public int getTimezone() {
-      return timezone;
-    }
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableIntVector(name, allocator);
+      }
 
-    public List<MinorType> getSubTypes() {
-      return subTypes;
-    }
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new IntWriterImpl((NullableIntVector) vector);
+      }
+    },   //  four byte signed integer
+    BIGINT(new Int(64, true)) {
+      @Override
+      public Field getField() {
+        return BIGINT_FIELD;
+      }
 
-    public int getWidth() {
-      return width;
-    }
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableBigIntVector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new BigIntWriterImpl((NullableBigIntVector) vector);
+      }
+    },   //  eight byte signed integer
+    DATE(Date.INSTANCE) {
+      @Override
+      public Field getField() {
+        return DATE_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableDateVector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new DateWriterImpl((NullableDateVector) vector);
+      }
+    },   //  days since 4713bc
+    TIME(Time.INSTANCE) {
+      @Override
+      public Field getField() {
+        return TIME_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableTimeVector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new TimeWriterImpl((NullableTimeVector) vector);
+      }
+    },   //  time in micros before or after 2000/1/1
+    TIMESTAMP(new Timestamp("")) {
+      @Override
+      public Field getField() {
+        return TIMESTAMP_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableTimeStampVector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new TimeStampWriterImpl((NullableTimeStampVector) vector);
+      }
+    },
+    INTERVALDAY(IntervalDay.INSTANCE) {
+      @Override
+      public Field getField() {
+        return INTERVALDAY_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableIntervalDayVector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new IntervalDayWriterImpl((NullableIntervalDayVector) vector);
+      }
+    },
+    INTERVALYEAR(IntervalYear.INSTANCE) {
+      @Override
+      public Field getField() {
+        return INTERVALYEAR_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableIntervalDayVector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new IntervalYearWriterImpl((NullableIntervalYearVector) vector);
+      }
+    },
+    FLOAT4(new FloatingPoint(0)) {
+      @Override
+      public Field getField() {
+        return FLOAT4_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableFloat4Vector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new Float4WriterImpl((NullableFloat4Vector) vector);
+      }
+    },   //  4 byte ieee 754
+    FLOAT8(new FloatingPoint(1)) {
+      @Override
+      public Field getField() {
+        return FLOAT8_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableFloat8Vector(name, allocator);
+      }
 
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new Float8WriterImpl((NullableFloat8Vector) vector);
+      }
+    },   //  8 byte ieee 754
+    BIT(Bool.INSTANCE) {
+      @Override
+      public Field getField() {
+        return BIT_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableBitVector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new BitWriterImpl((NullableBitVector) vector);
+      }
+    },  //  single bit value (boolean)
+    VARCHAR(Utf8.INSTANCE) {
+      @Override
+      public Field getField() {
+        return VARCHAR_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableVarCharVector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new VarCharWriterImpl((NullableVarCharVector) vector);
+      }
+    },   //  utf8 variable length string
+    VARBINARY(Binary.INSTANCE) {
+      @Override
+      public Field getField() {
+        return VARBINARY_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableVarBinaryVector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new VarBinaryWriterImpl((NullableVarBinaryVector) vector);
+      }
+    },   //  variable length binary
+    DECIMAL(null) {
+      @Override
+      public ArrowType getType() {
+        throw new UnsupportedOperationException("Cannot get simple type for Decimal type");
+      }
+      @Override
+      public Field getField() {
+        throw new UnsupportedOperationException("Cannot get simple field for Decimal type");
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableDecimalVector(name, allocator, precisionScale[0], precisionScale[1]);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new VarBinaryWriterImpl((NullableVarBinaryVector) vector);
+      }
+    },   //  variable length binary
+    UINT1(new Int(8, false)) {
+      @Override
+      public Field getField() {
+        return UINT1_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableUInt1Vector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new UInt1WriterImpl((NullableUInt1Vector) vector);
+      }
+    },  //  unsigned 1 byte integer
+    UINT2(new Int(16, false)) {
+      @Override
+      public Field getField() {
+        return UINT2_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableUInt2Vector(name, allocator);
+      }
 
-    @Override
-    public boolean equals(Object other) {
-      if (other == null) {
-        return false;
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new UInt2WriterImpl((NullableUInt2Vector) vector);
       }
-      if (!(other instanceof MajorType)) {
-        return false;
+    },  //  unsigned 2 byte integer
+    UINT4(new Int(32, false)) {
+      @Override
+      public Field getField() {
+        return UINT8_FIELD;
       }
-      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);
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableUInt4Vector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new UInt4WriterImpl((NullableUInt4Vector) vector);
+      }
+    },   //  unsigned 4 byte integer
+    UINT8(new Int(64, false)) {
+      @Override
+      public Field getField() {
+        return UINT8_FIELD;
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new NullableUInt8Vector(name, allocator);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new UInt8WriterImpl((NullableUInt8Vector) vector);
+      }
+    },   //  unsigned 8 byte integer
+    LIST(List.INSTANCE) {
+      @Override
+      public Field getField() {
+        throw new UnsupportedOperationException("Cannot get simple field for List type");
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new ListVector(name, allocator, callBack);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new UnionListWriter((ListVector) vector);
+      }
+    },
+    UNION(Union.INSTANCE) {
+      @Override
+      public Field getField() {
+        throw new UnsupportedOperationException("Cannot get simple field for Union type");
+      }
+
+      @Override
+      public ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale) {
+        return new UnionVector(name, allocator, callBack);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new UnionWriter((UnionVector) vector);
+      }
+    };
+
+    private final ArrowType type;
+
+    MinorType(ArrowType type) {
+      this.type = type;
     }
 
-  }
+    public ArrowType getType() {
+      return type;
+    }
+
+    public abstract Field getField();
 
-  public static MajorType required(MinorType minorType) {
-    return new MajorType(minorType, DataMode.REQUIRED);
+    public abstract ValueVector getNewVector(String name, BufferAllocator allocator, CallBack callBack, int... precisionScale);
+
+    public abstract FieldWriter getNewFieldWriter(ValueVector vector);
   }
-  public static MajorType optional(MinorType minorType) {
-    return new MajorType(minorType, DataMode.OPTIONAL);
+
+  private static final Map<ArrowType,MinorType> ARROW_TYPE_MINOR_TYPE_MAP;
+
+  public static MinorType getMinorTypeForArrowType(ArrowType arrowType) {
+    if (arrowType.getTypeType() == Type.Decimal) {
+      return MinorType.DECIMAL;
+    }
+    return ARROW_TYPE_MINOR_TYPE_MAP.get(arrowType);
   }
-  public static MajorType repeated(MinorType minorType) {
-    return new MajorType(minorType, DataMode.REPEATED);
+
+  static {
+    ARROW_TYPE_MINOR_TYPE_MAP = new HashMap<>();
+    for (MinorType minorType : MinorType.values()) {
+      if (minorType != MinorType.DECIMAL) {
+        ARROW_TYPE_MINOR_TYPE_MAP.put(minorType.getType(), minorType);
+      }
+    }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/e7e399db/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Field.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Field.java b/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Field.java
new file mode 100644
index 0000000..49d0503
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Field.java
@@ -0,0 +1,105 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.types.pojo;
+
+
+import com.google.common.collect.ImmutableList;
+import com.google.flatbuffers.FlatBufferBuilder;
+
+import java.util.List;
+import java.util.Objects;
+
+import static org.apache.arrow.vector.types.pojo.ArrowType.getTypeForField;
+
+public class Field {
+  private final String name;
+  private final boolean nullable;
+  private final ArrowType type;
+  private final List<Field> children;
+
+  public Field(String name, boolean nullable, ArrowType type, List<Field> children) {
+    this.name = name;
+    this.nullable = nullable;
+    this.type = type;
+    if (children == null) {
+      this.children = ImmutableList.of();
+    } else {
+      this.children = children;
+    }
+  }
+
+  public static Field convertField(org.apache.arrow.flatbuf.Field field) {
+    String name = field.name();
+    boolean nullable = field.nullable();
+    ArrowType type = getTypeForField(field);
+    ImmutableList.Builder<Field> childrenBuilder = ImmutableList.builder();
+    for (int i = 0; i < field.childrenLength(); i++) {
+      childrenBuilder.add(convertField(field.children(i)));
+    }
+    List<Field> children = childrenBuilder.build();
+    return new Field(name, nullable, type, children);
+  }
+
+  public int getField(FlatBufferBuilder builder) {
+    int nameOffset = builder.createString(name);
+    int typeOffset = type.getType(builder);
+    int[] childrenData = new int[children.size()];
+    for (int i = 0; i < children.size(); i++) {
+      childrenData[i] = children.get(i).getField(builder);
+    }
+    int childrenOffset = org.apache.arrow.flatbuf.Field.createChildrenVector(builder, childrenData);
+    org.apache.arrow.flatbuf.Field.startField(builder);
+    org.apache.arrow.flatbuf.Field.addName(builder, nameOffset);
+    org.apache.arrow.flatbuf.Field.addNullable(builder, nullable);
+    org.apache.arrow.flatbuf.Field.addTypeType(builder, type.getTypeType());
+    org.apache.arrow.flatbuf.Field.addType(builder, typeOffset);
+    org.apache.arrow.flatbuf.Field.addChildren(builder, childrenOffset);
+    return org.apache.arrow.flatbuf.Field.endField(builder);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public boolean isNullable() {
+    return nullable;
+  }
+
+  public ArrowType getType() {
+    return type;
+  }
+
+  public List<Field> getChildren() {
+    return children;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (!(obj instanceof Field)) {
+      return false;
+    }
+    Field that = (Field) obj;
+    return Objects.equals(this.name, that.name) &&
+            Objects.equals(this.nullable, that.nullable) &&
+            Objects.equals(this.type, that.type) &&
+            (Objects.equals(this.children, that.children) ||
+                    (this.children == null && that.children.size() == 0) ||
+                    (this.children.size() == 0 && that.children == null));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/e7e399db/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Schema.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Schema.java b/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Schema.java
new file mode 100644
index 0000000..9e28941
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Schema.java
@@ -0,0 +1,74 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.types.pojo;
+
+
+import com.google.common.collect.ImmutableList;
+import com.google.flatbuffers.FlatBufferBuilder;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Objects;
+
+import static org.apache.arrow.vector.types.pojo.ArrowType.getTypeForField;
+import static org.apache.arrow.vector.types.pojo.Field.convertField;
+
+public class Schema {
+  private List<Field> fields;
+
+  public Schema(List<Field> fields) {
+    this.fields = ImmutableList.copyOf(fields);
+  }
+
+  public int getSchema(FlatBufferBuilder builder) {
+    int[] fieldOffsets = new int[fields.size()];
+    for (int i = 0; i < fields.size(); i++) {
+      fieldOffsets[i] = fields.get(i).getField(builder);
+    }
+    int fieldsOffset = org.apache.arrow.flatbuf.Schema.createFieldsVector(builder, fieldOffsets);
+    org.apache.arrow.flatbuf.Schema.startSchema(builder);
+    org.apache.arrow.flatbuf.Schema.addFields(builder, fieldsOffset);
+    return org.apache.arrow.flatbuf.Schema.endSchema(builder);
+  }
+
+  public List<Field> getFields() {
+    return fields;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(fields);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (!(obj instanceof Schema)) {
+      return false;
+    }
+    return Objects.equals(this.fields, ((Schema) obj).fields);
+  }
+
+  public static Schema convertSchema(org.apache.arrow.flatbuf.Schema schema) {
+    ImmutableList.Builder<Field> childrenBuilder = ImmutableList.builder();
+    for (int i = 0; i < schema.fieldsLength(); i++) {
+      childrenBuilder.add(convertField(schema.fields(i)));
+    }
+    List<Field> fields = childrenBuilder.build();
+    return new Schema(fields);
+  }
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/e7e399db/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java b/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java
index b6dd13a..68b9fb2 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java
@@ -180,54 +180,4 @@ public class ByteFunctionHelpers {
     return lLen > rLen ? 1 : -1;
   }
 
-  /*
-   * Following are helper functions to interact with sparse decimal represented in a byte array.
-   */
-
-  // Get the integer ignore the sign
-  public static int getInteger(byte[] b, int index) {
-    return getInteger(b, index, true);
-  }
-  // Get the integer, ignore the sign
-  public static int getInteger(byte[] b, int index, boolean ignoreSign) {
-    int startIndex = index * DecimalUtility.INTEGER_SIZE;
-
-    if (index == 0 && ignoreSign == true) {
-      return (b[startIndex + 3] & 0xFF) |
-             (b[startIndex + 2] & 0xFF) << 8 |
-             (b[startIndex + 1] & 0xFF) << 16 |
-             (b[startIndex] & 0x7F) << 24;
-    }
-
-    return ((b[startIndex + 3] & 0xFF) |
-        (b[startIndex + 2] & 0xFF) << 8 |
-        (b[startIndex + 1] & 0xFF) << 16 |
-        (b[startIndex] & 0xFF) << 24);
-
-  }
-
-  // Set integer in the byte array
-  public static void setInteger(byte[] b, int index, int value) {
-    int startIndex = index * DecimalUtility.INTEGER_SIZE;
-    b[startIndex] = (byte) ((value >> 24) & 0xFF);
-    b[startIndex + 1] = (byte) ((value >> 16) & 0xFF);
-    b[startIndex + 2] = (byte) ((value >> 8) & 0xFF);
-    b[startIndex + 3] = (byte) ((value) & 0xFF);
-  }
-
-  // Set the sign in a sparse decimal representation
-  public static void setSign(byte[] b, boolean sign) {
-    int value = getInteger(b, 0);
-    if (sign == true) {
-      setInteger(b, 0, value | 0x80000000);
-    } else {
-      setInteger(b, 0, value & 0x7FFFFFFF);
-    }
-  }
-
-  // Get the sign
-  public static boolean getSign(byte[] b) {
-    return ((getInteger(b, 0, false) & 0x80000000) != 0);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/e7e399db/java/vector/src/main/java/org/apache/arrow/vector/util/CoreDecimalUtility.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/CoreDecimalUtility.java b/java/vector/src/main/java/org/apache/arrow/vector/util/CoreDecimalUtility.java
deleted file mode 100644
index 1eb2c13..0000000
--- a/java/vector/src/main/java/org/apache/arrow/vector/util/CoreDecimalUtility.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * 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.util;
-
-import java.math.BigDecimal;
-
-import org.apache.arrow.vector.types.Types;
-
-public class CoreDecimalUtility {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CoreDecimalUtility.class);
-
-  public static long getDecimal18FromBigDecimal(BigDecimal input, int scale, int precision) {
-    // Truncate or pad to set the input to the correct scale
-    input = input.setScale(scale, BigDecimal.ROUND_HALF_UP);
-
-    return (input.unscaledValue().longValue());
-  }
-
-  public static int getMaxPrecision(Types.MinorType decimalType) {
-    if (decimalType == Types.MinorType.DECIMAL9) {
-      return 9;
-    } else if (decimalType == Types.MinorType.DECIMAL18) {
-      return 18;
-    } else if (decimalType == Types.MinorType.DECIMAL28SPARSE) {
-      return 28;
-    } else if (decimalType == Types.MinorType.DECIMAL38SPARSE) {
-      return 38;
-    }
-    return 0;
-  }
-
-  /*
-   * Function returns the Minor decimal type given the precision
-   */
-  public static Types.MinorType getDecimalDataType(int precision) {
-    if (precision <= 9) {
-      return Types.MinorType.DECIMAL9;
-    } else if (precision <= 18) {
-      return Types.MinorType.DECIMAL18;
-    } else if (precision <= 28) {
-      return Types.MinorType.DECIMAL28SPARSE;
-    } else {
-      return Types.MinorType.DECIMAL38SPARSE;
-    }
-  }
-
-  /*
-   * Given a precision it provides the max precision of that decimal data type;
-   * For eg: given the precision 12, we would use DECIMAL18 to store the data
-   * which has a max precision range of 18 digits
-   */
-  public static int getPrecisionRange(int precision) {
-    return getMaxPrecision(getDecimalDataType(precision));
-  }
-  public static int getDecimal9FromBigDecimal(BigDecimal input, int scale, int precision) {
-    // Truncate/ or pad to set the input to the correct scale
-    input = input.setScale(scale, BigDecimal.ROUND_HALF_UP);
-
-    return (input.unscaledValue().intValue());
-  }
-
-  /*
-   * Helper function to detect if the given data type is Decimal
-   */
-  public static boolean isDecimalType(Types.MajorType type) {
-    return isDecimalType(type.getMinorType());
-  }
-
-  public static boolean isDecimalType(Types.MinorType minorType) {
-    if (minorType == Types.MinorType.DECIMAL9 || minorType == Types.MinorType.DECIMAL18 ||
-        minorType == Types.MinorType.DECIMAL28SPARSE || minorType == Types.MinorType.DECIMAL38SPARSE) {
-      return true;
-    }
-    return false;
-  }
-}