You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ja...@apache.org on 2016/02/17 13:39:38 UTC

[03/17] arrow git commit: ARROW-1: Initial Arrow Code Commit

http://git-wip-us.apache.org/repos/asf/arrow/blob/fa5f0299/java/vector/src/main/java/org/apache/arrow/vector/complex/reader/FieldReader.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/reader/FieldReader.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/reader/FieldReader.java
new file mode 100644
index 0000000..c4eb3dc
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/reader/FieldReader.java
@@ -0,0 +1,29 @@
+/**
+ * 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.complex.reader;
+
+import org.apache.arrow.vector.complex.reader.BaseReader.ListReader;
+import org.apache.arrow.vector.complex.reader.BaseReader.MapReader;
+import org.apache.arrow.vector.complex.reader.BaseReader.RepeatedListReader;
+import org.apache.arrow.vector.complex.reader.BaseReader.RepeatedMapReader;
+import org.apache.arrow.vector.complex.reader.BaseReader.ScalarReader;
+
+
+
+public interface FieldReader extends MapReader, ListReader, ScalarReader, RepeatedMapReader, RepeatedListReader {
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/arrow/blob/fa5f0299/java/vector/src/main/java/org/apache/arrow/vector/complex/writer/FieldWriter.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/writer/FieldWriter.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/writer/FieldWriter.java
new file mode 100644
index 0000000..ecffe0b
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/writer/FieldWriter.java
@@ -0,0 +1,27 @@
+/**
+ * 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.complex.writer;
+
+import org.apache.arrow.vector.complex.writer.BaseWriter.ListWriter;
+import org.apache.arrow.vector.complex.writer.BaseWriter.MapWriter;
+import org.apache.arrow.vector.complex.writer.BaseWriter.ScalarWriter;
+
+public interface FieldWriter extends MapWriter, ListWriter, ScalarWriter {
+  void allocate();
+  void clear();
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/fa5f0299/java/vector/src/main/java/org/apache/arrow/vector/holders/ComplexHolder.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/holders/ComplexHolder.java b/java/vector/src/main/java/org/apache/arrow/vector/holders/ComplexHolder.java
new file mode 100644
index 0000000..0f9310d
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/holders/ComplexHolder.java
@@ -0,0 +1,25 @@
+/**
+ * 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.complex.reader.FieldReader;
+
+public class ComplexHolder implements ValueHolder {
+  public FieldReader reader;
+  public int isSet;
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/fa5f0299/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
new file mode 100644
index 0000000..5a5fe03
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/holders/ObjectHolder.java
@@ -0,0 +1,38 @@
+/**
+ * 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/fa5f0299/java/vector/src/main/java/org/apache/arrow/vector/holders/RepeatedListHolder.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/holders/RepeatedListHolder.java b/java/vector/src/main/java/org/apache/arrow/vector/holders/RepeatedListHolder.java
new file mode 100644
index 0000000..83506cd
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/holders/RepeatedListHolder.java
@@ -0,0 +1,23 @@
+/**
+ * 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;
+
+public final class RepeatedListHolder implements ValueHolder{
+  public int start;
+  public int end;
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/fa5f0299/java/vector/src/main/java/org/apache/arrow/vector/holders/RepeatedMapHolder.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/holders/RepeatedMapHolder.java b/java/vector/src/main/java/org/apache/arrow/vector/holders/RepeatedMapHolder.java
new file mode 100644
index 0000000..85d782b
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/holders/RepeatedMapHolder.java
@@ -0,0 +1,23 @@
+/**
+ * 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;
+
+public final class RepeatedMapHolder implements ValueHolder{
+  public int start;
+  public int end;
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/fa5f0299/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
new file mode 100644
index 0000000..b868a62
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/holders/UnionHolder.java
@@ -0,0 +1,37 @@
+/**
+ * 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.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 boolean isSet() {
+    return isSet == 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/fa5f0299/java/vector/src/main/java/org/apache/arrow/vector/holders/ValueHolder.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/holders/ValueHolder.java b/java/vector/src/main/java/org/apache/arrow/vector/holders/ValueHolder.java
new file mode 100644
index 0000000..88cbcd4
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/holders/ValueHolder.java
@@ -0,0 +1,31 @@
+/**
+ * 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;
+
+/**
+ * Wrapper object for an individual value in Drill.
+ *
+ * ValueHolders are designed to be mutable wrapper objects for defining clean
+ * APIs that access data in Drill. For performance, object creation is avoided
+ * at all costs throughout execution. For this reason, ValueHolders are
+ * disallowed from implementing any methods, this allows for them to be
+ * replaced by their java primitive inner members during optimization of
+ * run-time generated code.
+ */
+public interface ValueHolder {
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/fa5f0299/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
new file mode 100644
index 0000000..c73098b
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/types/MaterializedField.java
@@ -0,0 +1,217 @@
+/**
+ * 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/fa5f0299/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
new file mode 100644
index 0000000..cef892c
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/types/Types.java
@@ -0,0 +1,132 @@
+/**
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class Types {
+  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
+  }
+
+  public enum DataMode {
+    REQUIRED,
+    OPTIONAL,
+    REPEATED
+  }
+
+  public static class MajorType {
+    private MinorType minorType;
+    private DataMode mode;
+    private Integer precision;
+    private Integer scale;
+    private Integer timezone;
+    private List<MinorType> subTypes;
+
+    public MajorType(MinorType minorType, DataMode mode) {
+      this(minorType, mode, null, null, null, null);
+    }
+
+    public MajorType(MinorType minorType, DataMode mode, Integer precision, Integer scale) {
+      this(minorType, mode, precision, scale, null, null);
+    }
+
+    public MajorType(MinorType minorType, DataMode mode, Integer precision, Integer scale, Integer timezone, List<MinorType> subTypes) {
+      this.minorType = minorType;
+      this.mode = mode;
+      this.precision = precision;
+      this.scale = scale;
+      this.timezone = timezone;
+      this.subTypes = subTypes;
+    }
+
+    public MinorType getMinorType() {
+      return minorType;
+    }
+
+    public DataMode getMode() {
+      return mode;
+    }
+
+    public Integer getPrecision() {
+      return precision;
+    }
+
+    public Integer getScale() {
+      return scale;
+    }
+
+    public Integer getTimezone() {
+      return timezone;
+    }
+
+    public List<MinorType> getSubTypes() {
+      return subTypes;
+    }
+  }
+
+  public static MajorType required(MinorType minorType) {
+    return new MajorType(minorType, DataMode.REQUIRED);
+  }
+  public static MajorType optional(MinorType minorType) {
+    return new MajorType(minorType, DataMode.OPTIONAL);
+  }
+  public static MajorType repeated(MinorType minorType) {
+    return new MajorType(minorType, DataMode.REPEATED);
+  }
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/fa5f0299/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
new file mode 100644
index 0000000..2bdfd70
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/ByteFunctionHelpers.java
@@ -0,0 +1,233 @@
+/*******************************************************************************
+
+ * 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 io.netty.buffer.ArrowBuf;
+import io.netty.util.internal.PlatformDependent;
+
+import org.apache.arrow.memory.BoundsChecking;
+
+import com.google.common.primitives.UnsignedLongs;
+
+public class ByteFunctionHelpers {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ByteFunctionHelpers.class);
+
+  /**
+   * Helper function to check for equality of bytes in two DrillBuffers
+   *
+   * @param left Left DrillBuf for comparison
+   * @param lStart start offset in the buffer
+   * @param lEnd end offset in the buffer
+   * @param right Right DrillBuf for comparison
+   * @param rStart start offset in the buffer
+   * @param rEnd end offset in the buffer
+   * @return 1 if left input is greater, -1 if left input is smaller, 0 otherwise
+   */
+  public static final int equal(final ArrowBuf left, int lStart, int lEnd, final ArrowBuf right, int rStart, int rEnd){
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
+      left.checkBytes(lStart, lEnd);
+      right.checkBytes(rStart, rEnd);
+    }
+    return memEqual(left.memoryAddress(), lStart, lEnd, right.memoryAddress(), rStart, rEnd);
+  }
+
+  private static final int memEqual(final long laddr, int lStart, int lEnd, final long raddr, int rStart,
+      final int rEnd) {
+
+    int n = lEnd - lStart;
+    if (n == rEnd - rStart) {
+      long lPos = laddr + lStart;
+      long rPos = raddr + rStart;
+
+      while (n > 7) {
+        long leftLong = PlatformDependent.getLong(lPos);
+        long rightLong = PlatformDependent.getLong(rPos);
+        if (leftLong != rightLong) {
+          return 0;
+        }
+        lPos += 8;
+        rPos += 8;
+        n -= 8;
+      }
+      while (n-- != 0) {
+        byte leftByte = PlatformDependent.getByte(lPos);
+        byte rightByte = PlatformDependent.getByte(rPos);
+        if (leftByte != rightByte) {
+          return 0;
+        }
+        lPos++;
+        rPos++;
+      }
+      return 1;
+    } else {
+      return 0;
+    }
+  }
+
+  /**
+   * Helper function to compare a set of bytes in two DrillBuffers.
+   *
+   * Function will check data before completing in the case that
+   *
+   * @param left Left DrillBuf to compare
+   * @param lStart start offset in the buffer
+   * @param lEnd end offset in the buffer
+   * @param right Right DrillBuf to compare
+   * @param rStart start offset in the buffer
+   * @param rEnd end offset in the buffer
+   * @return 1 if left input is greater, -1 if left input is smaller, 0 otherwise
+   */
+  public static final int compare(final ArrowBuf left, int lStart, int lEnd, final ArrowBuf right, int rStart, int rEnd){
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
+      left.checkBytes(lStart, lEnd);
+      right.checkBytes(rStart, rEnd);
+    }
+    return memcmp(left.memoryAddress(), lStart, lEnd, right.memoryAddress(), rStart, rEnd);
+  }
+
+  private static final int memcmp(final long laddr, int lStart, int lEnd, final long raddr, int rStart, final int rEnd) {
+    int lLen = lEnd - lStart;
+    int rLen = rEnd - rStart;
+    int n = Math.min(rLen, lLen);
+    long lPos = laddr + lStart;
+    long rPos = raddr + rStart;
+
+    while (n > 7) {
+      long leftLong = PlatformDependent.getLong(lPos);
+      long rightLong = PlatformDependent.getLong(rPos);
+      if (leftLong != rightLong) {
+        return UnsignedLongs.compare(Long.reverseBytes(leftLong), Long.reverseBytes(rightLong));
+      }
+      lPos += 8;
+      rPos += 8;
+      n -= 8;
+    }
+
+    while (n-- != 0) {
+      byte leftByte = PlatformDependent.getByte(lPos);
+      byte rightByte = PlatformDependent.getByte(rPos);
+      if (leftByte != rightByte) {
+        return ((leftByte & 0xFF) - (rightByte & 0xFF)) > 0 ? 1 : -1;
+      }
+      lPos++;
+      rPos++;
+    }
+
+    if (lLen == rLen) {
+      return 0;
+    }
+
+    return lLen > rLen ? 1 : -1;
+
+  }
+
+  /**
+   * Helper function to compare a set of bytes in DrillBuf to a ByteArray.
+   *
+   * @param left Left DrillBuf for comparison purposes
+   * @param lStart start offset in the buffer
+   * @param lEnd end offset in the buffer
+   * @param right second input to be compared
+   * @param rStart start offset in the byte array
+   * @param rEnd end offset in the byte array
+   * @return 1 if left input is greater, -1 if left input is smaller, 0 otherwise
+   */
+  public static final int compare(final ArrowBuf left, int lStart, int lEnd, final byte[] right, int rStart, final int rEnd) {
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
+      left.checkBytes(lStart, lEnd);
+    }
+    return memcmp(left.memoryAddress(), lStart, lEnd, right, rStart, rEnd);
+  }
+
+
+  private static final int memcmp(final long laddr, int lStart, int lEnd, final byte[] right, int rStart, final int rEnd) {
+    int lLen = lEnd - lStart;
+    int rLen = rEnd - rStart;
+    int n = Math.min(rLen, lLen);
+    long lPos = laddr + lStart;
+    int rPos = rStart;
+
+    while (n-- != 0) {
+      byte leftByte = PlatformDependent.getByte(lPos);
+      byte rightByte = right[rPos];
+      if (leftByte != rightByte) {
+        return ((leftByte & 0xFF) - (rightByte & 0xFF)) > 0 ? 1 : -1;
+      }
+      lPos++;
+      rPos++;
+    }
+
+    if (lLen == rLen) {
+      return 0;
+    }
+
+    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/fa5f0299/java/vector/src/main/java/org/apache/arrow/vector/util/CallBack.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/CallBack.java b/java/vector/src/main/java/org/apache/arrow/vector/util/CallBack.java
new file mode 100644
index 0000000..2498342
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/CallBack.java
@@ -0,0 +1,23 @@
+/**
+ * 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;
+
+
+public interface CallBack {
+  public void doWork();
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/fa5f0299/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
new file mode 100644
index 0000000..1eb2c13
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/CoreDecimalUtility.java
@@ -0,0 +1,91 @@
+/**
+ * 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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/fa5f0299/java/vector/src/main/java/org/apache/arrow/vector/util/DateUtility.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/DateUtility.java b/java/vector/src/main/java/org/apache/arrow/vector/util/DateUtility.java
new file mode 100644
index 0000000..f4fc173
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/DateUtility.java
@@ -0,0 +1,682 @@
+/**
+ * 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 org.joda.time.Period;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.DateTimeFormatterBuilder;
+import org.joda.time.format.DateTimeParser;
+
+import com.carrotsearch.hppc.ObjectIntHashMap;
+
+// Utility class for Date, DateTime, TimeStamp, Interval data types
+public class DateUtility {
+
+
+    /* We have a hashmap that stores the timezone as the key and an index as the value
+     * While storing the timezone in value vectors, holders we only use this index. As we
+     * reconstruct the timestamp, we use this index to index through the array timezoneList
+     * and get the corresponding timezone and pass it to joda-time
+     */
+  public static ObjectIntHashMap<String> timezoneMap = new ObjectIntHashMap<String>();
+
+    public static String[] timezoneList =  {"Africa/Abidjan",
+                                            "Africa/Accra",
+                                            "Africa/Addis_Ababa",
+                                            "Africa/Algiers",
+                                            "Africa/Asmara",
+                                            "Africa/Asmera",
+                                            "Africa/Bamako",
+                                            "Africa/Bangui",
+                                            "Africa/Banjul",
+                                            "Africa/Bissau",
+                                            "Africa/Blantyre",
+                                            "Africa/Brazzaville",
+                                            "Africa/Bujumbura",
+                                            "Africa/Cairo",
+                                            "Africa/Casablanca",
+                                            "Africa/Ceuta",
+                                            "Africa/Conakry",
+                                            "Africa/Dakar",
+                                            "Africa/Dar_es_Salaam",
+                                            "Africa/Djibouti",
+                                            "Africa/Douala",
+                                            "Africa/El_Aaiun",
+                                            "Africa/Freetown",
+                                            "Africa/Gaborone",
+                                            "Africa/Harare",
+                                            "Africa/Johannesburg",
+                                            "Africa/Juba",
+                                            "Africa/Kampala",
+                                            "Africa/Khartoum",
+                                            "Africa/Kigali",
+                                            "Africa/Kinshasa",
+                                            "Africa/Lagos",
+                                            "Africa/Libreville",
+                                            "Africa/Lome",
+                                            "Africa/Luanda",
+                                            "Africa/Lubumbashi",
+                                            "Africa/Lusaka",
+                                            "Africa/Malabo",
+                                            "Africa/Maputo",
+                                            "Africa/Maseru",
+                                            "Africa/Mbabane",
+                                            "Africa/Mogadishu",
+                                            "Africa/Monrovia",
+                                            "Africa/Nairobi",
+                                            "Africa/Ndjamena",
+                                            "Africa/Niamey",
+                                            "Africa/Nouakchott",
+                                            "Africa/Ouagadougou",
+                                            "Africa/Porto-Novo",
+                                            "Africa/Sao_Tome",
+                                            "Africa/Timbuktu",
+                                            "Africa/Tripoli",
+                                            "Africa/Tunis",
+                                            "Africa/Windhoek",
+                                            "America/Adak",
+                                            "America/Anchorage",
+                                            "America/Anguilla",
+                                            "America/Antigua",
+                                            "America/Araguaina",
+                                            "America/Argentina/Buenos_Aires",
+                                            "America/Argentina/Catamarca",
+                                            "America/Argentina/ComodRivadavia",
+                                            "America/Argentina/Cordoba",
+                                            "America/Argentina/Jujuy",
+                                            "America/Argentina/La_Rioja",
+                                            "America/Argentina/Mendoza",
+                                            "America/Argentina/Rio_Gallegos",
+                                            "America/Argentina/Salta",
+                                            "America/Argentina/San_Juan",
+                                            "America/Argentina/San_Luis",
+                                            "America/Argentina/Tucuman",
+                                            "America/Argentina/Ushuaia",
+                                            "America/Aruba",
+                                            "America/Asuncion",
+                                            "America/Atikokan",
+                                            "America/Atka",
+                                            "America/Bahia",
+                                            "America/Bahia_Banderas",
+                                            "America/Barbados",
+                                            "America/Belem",
+                                            "America/Belize",
+                                            "America/Blanc-Sablon",
+                                            "America/Boa_Vista",
+                                            "America/Bogota",
+                                            "America/Boise",
+                                            "America/Buenos_Aires",
+                                            "America/Cambridge_Bay",
+                                            "America/Campo_Grande",
+                                            "America/Cancun",
+                                            "America/Caracas",
+                                            "America/Catamarca",
+                                            "America/Cayenne",
+                                            "America/Cayman",
+                                            "America/Chicago",
+                                            "America/Chihuahua",
+                                            "America/Coral_Harbour",
+                                            "America/Cordoba",
+                                            "America/Costa_Rica",
+                                            "America/Cuiaba",
+                                            "America/Curacao",
+                                            "America/Danmarkshavn",
+                                            "America/Dawson",
+                                            "America/Dawson_Creek",
+                                            "America/Denver",
+                                            "America/Detroit",
+                                            "America/Dominica",
+                                            "America/Edmonton",
+                                            "America/Eirunepe",
+                                            "America/El_Salvador",
+                                            "America/Ensenada",
+                                            "America/Fort_Wayne",
+                                            "America/Fortaleza",
+                                            "America/Glace_Bay",
+                                            "America/Godthab",
+                                            "America/Goose_Bay",
+                                            "America/Grand_Turk",
+                                            "America/Grenada",
+                                            "America/Guadeloupe",
+                                            "America/Guatemala",
+                                            "America/Guayaquil",
+                                            "America/Guyana",
+                                            "America/Halifax",
+                                            "America/Havana",
+                                            "America/Hermosillo",
+                                            "America/Indiana/Indianapolis",
+                                            "America/Indiana/Knox",
+                                            "America/Indiana/Marengo",
+                                            "America/Indiana/Petersburg",
+                                            "America/Indiana/Tell_City",
+                                            "America/Indiana/Vevay",
+                                            "America/Indiana/Vincennes",
+                                            "America/Indiana/Winamac",
+                                            "America/Indianapolis",
+                                            "America/Inuvik",
+                                            "America/Iqaluit",
+                                            "America/Jamaica",
+                                            "America/Jujuy",
+                                            "America/Juneau",
+                                            "America/Kentucky/Louisville",
+                                            "America/Kentucky/Monticello",
+                                            "America/Knox_IN",
+                                            "America/Kralendijk",
+                                            "America/La_Paz",
+                                            "America/Lima",
+                                            "America/Los_Angeles",
+                                            "America/Louisville",
+                                            "America/Lower_Princes",
+                                            "America/Maceio",
+                                            "America/Managua",
+                                            "America/Manaus",
+                                            "America/Marigot",
+                                            "America/Martinique",
+                                            "America/Matamoros",
+                                            "America/Mazatlan",
+                                            "America/Mendoza",
+                                            "America/Menominee",
+                                            "America/Merida",
+                                            "America/Metlakatla",
+                                            "America/Mexico_City",
+                                            "America/Miquelon",
+                                            "America/Moncton",
+                                            "America/Monterrey",
+                                            "America/Montevideo",
+                                            "America/Montreal",
+                                            "America/Montserrat",
+                                            "America/Nassau",
+                                            "America/New_York",
+                                            "America/Nipigon",
+                                            "America/Nome",
+                                            "America/Noronha",
+                                            "America/North_Dakota/Beulah",
+                                            "America/North_Dakota/Center",
+                                            "America/North_Dakota/New_Salem",
+                                            "America/Ojinaga",
+                                            "America/Panama",
+                                            "America/Pangnirtung",
+                                            "America/Paramaribo",
+                                            "America/Phoenix",
+                                            "America/Port-au-Prince",
+                                            "America/Port_of_Spain",
+                                            "America/Porto_Acre",
+                                            "America/Porto_Velho",
+                                            "America/Puerto_Rico",
+                                            "America/Rainy_River",
+                                            "America/Rankin_Inlet",
+                                            "America/Recife",
+                                            "America/Regina",
+                                            "America/Resolute",
+                                            "America/Rio_Branco",
+                                            "America/Rosario",
+                                            "America/Santa_Isabel",
+                                            "America/Santarem",
+                                            "America/Santiago",
+                                            "America/Santo_Domingo",
+                                            "America/Sao_Paulo",
+                                            "America/Scoresbysund",
+                                            "America/Shiprock",
+                                            "America/Sitka",
+                                            "America/St_Barthelemy",
+                                            "America/St_Johns",
+                                            "America/St_Kitts",
+                                            "America/St_Lucia",
+                                            "America/St_Thomas",
+                                            "America/St_Vincent",
+                                            "America/Swift_Current",
+                                            "America/Tegucigalpa",
+                                            "America/Thule",
+                                            "America/Thunder_Bay",
+                                            "America/Tijuana",
+                                            "America/Toronto",
+                                            "America/Tortola",
+                                            "America/Vancouver",
+                                            "America/Virgin",
+                                            "America/Whitehorse",
+                                            "America/Winnipeg",
+                                            "America/Yakutat",
+                                            "America/Yellowknife",
+                                            "Antarctica/Casey",
+                                            "Antarctica/Davis",
+                                            "Antarctica/DumontDUrville",
+                                            "Antarctica/Macquarie",
+                                            "Antarctica/Mawson",
+                                            "Antarctica/McMurdo",
+                                            "Antarctica/Palmer",
+                                            "Antarctica/Rothera",
+                                            "Antarctica/South_Pole",
+                                            "Antarctica/Syowa",
+                                            "Antarctica/Vostok",
+                                            "Arctic/Longyearbyen",
+                                            "Asia/Aden",
+                                            "Asia/Almaty",
+                                            "Asia/Amman",
+                                            "Asia/Anadyr",
+                                            "Asia/Aqtau",
+                                            "Asia/Aqtobe",
+                                            "Asia/Ashgabat",
+                                            "Asia/Ashkhabad",
+                                            "Asia/Baghdad",
+                                            "Asia/Bahrain",
+                                            "Asia/Baku",
+                                            "Asia/Bangkok",
+                                            "Asia/Beirut",
+                                            "Asia/Bishkek",
+                                            "Asia/Brunei",
+                                            "Asia/Calcutta",
+                                            "Asia/Choibalsan",
+                                            "Asia/Chongqing",
+                                            "Asia/Chungking",
+                                            "Asia/Colombo",
+                                            "Asia/Dacca",
+                                            "Asia/Damascus",
+                                            "Asia/Dhaka",
+                                            "Asia/Dili",
+                                            "Asia/Dubai",
+                                            "Asia/Dushanbe",
+                                            "Asia/Gaza",
+                                            "Asia/Harbin",
+                                            "Asia/Hebron",
+                                            "Asia/Ho_Chi_Minh",
+                                            "Asia/Hong_Kong",
+                                            "Asia/Hovd",
+                                            "Asia/Irkutsk",
+                                            "Asia/Istanbul",
+                                            "Asia/Jakarta",
+                                            "Asia/Jayapura",
+                                            "Asia/Jerusalem",
+                                            "Asia/Kabul",
+                                            "Asia/Kamchatka",
+                                            "Asia/Karachi",
+                                            "Asia/Kashgar",
+                                            "Asia/Kathmandu",
+                                            "Asia/Katmandu",
+                                            "Asia/Kolkata",
+                                            "Asia/Krasnoyarsk",
+                                            "Asia/Kuala_Lumpur",
+                                            "Asia/Kuching",
+                                            "Asia/Kuwait",
+                                            "Asia/Macao",
+                                            "Asia/Macau",
+                                            "Asia/Magadan",
+                                            "Asia/Makassar",
+                                            "Asia/Manila",
+                                            "Asia/Muscat",
+                                            "Asia/Nicosia",
+                                            "Asia/Novokuznetsk",
+                                            "Asia/Novosibirsk",
+                                            "Asia/Omsk",
+                                            "Asia/Oral",
+                                            "Asia/Phnom_Penh",
+                                            "Asia/Pontianak",
+                                            "Asia/Pyongyang",
+                                            "Asia/Qatar",
+                                            "Asia/Qyzylorda",
+                                            "Asia/Rangoon",
+                                            "Asia/Riyadh",
+                                            "Asia/Saigon",
+                                            "Asia/Sakhalin",
+                                            "Asia/Samarkand",
+                                            "Asia/Seoul",
+                                            "Asia/Shanghai",
+                                            "Asia/Singapore",
+                                            "Asia/Taipei",
+                                            "Asia/Tashkent",
+                                            "Asia/Tbilisi",
+                                            "Asia/Tehran",
+                                            "Asia/Tel_Aviv",
+                                            "Asia/Thimbu",
+                                            "Asia/Thimphu",
+                                            "Asia/Tokyo",
+                                            "Asia/Ujung_Pandang",
+                                            "Asia/Ulaanbaatar",
+                                            "Asia/Ulan_Bator",
+                                            "Asia/Urumqi",
+                                            "Asia/Vientiane",
+                                            "Asia/Vladivostok",
+                                            "Asia/Yakutsk",
+                                            "Asia/Yekaterinburg",
+                                            "Asia/Yerevan",
+                                            "Atlantic/Azores",
+                                            "Atlantic/Bermuda",
+                                            "Atlantic/Canary",
+                                            "Atlantic/Cape_Verde",
+                                            "Atlantic/Faeroe",
+                                            "Atlantic/Faroe",
+                                            "Atlantic/Jan_Mayen",
+                                            "Atlantic/Madeira",
+                                            "Atlantic/Reykjavik",
+                                            "Atlantic/South_Georgia",
+                                            "Atlantic/St_Helena",
+                                            "Atlantic/Stanley",
+                                            "Australia/ACT",
+                                            "Australia/Adelaide",
+                                            "Australia/Brisbane",
+                                            "Australia/Broken_Hill",
+                                            "Australia/Canberra",
+                                            "Australia/Currie",
+                                            "Australia/Darwin",
+                                            "Australia/Eucla",
+                                            "Australia/Hobart",
+                                            "Australia/LHI",
+                                            "Australia/Lindeman",
+                                            "Australia/Lord_Howe",
+                                            "Australia/Melbourne",
+                                            "Australia/NSW",
+                                            "Australia/North",
+                                            "Australia/Perth",
+                                            "Australia/Queensland",
+                                            "Australia/South",
+                                            "Australia/Sydney",
+                                            "Australia/Tasmania",
+                                            "Australia/Victoria",
+                                            "Australia/West",
+                                            "Australia/Yancowinna",
+                                            "Brazil/Acre",
+                                            "Brazil/DeNoronha",
+                                            "Brazil/East",
+                                            "Brazil/West",
+                                            "CET",
+                                            "CST6CDT",
+                                            "Canada/Atlantic",
+                                            "Canada/Central",
+                                            "Canada/East-Saskatchewan",
+                                            "Canada/Eastern",
+                                            "Canada/Mountain",
+                                            "Canada/Newfoundland",
+                                            "Canada/Pacific",
+                                            "Canada/Saskatchewan",
+                                            "Canada/Yukon",
+                                            "Chile/Continental",
+                                            "Chile/EasterIsland",
+                                            "Cuba",
+                                            "EET",
+                                            "EST",
+                                            "EST5EDT",
+                                            "Egypt",
+                                            "Eire",
+                                            "Etc/GMT",
+                                            "Etc/GMT+0",
+                                            "Etc/GMT+1",
+                                            "Etc/GMT+10",
+                                            "Etc/GMT+11",
+                                            "Etc/GMT+12",
+                                            "Etc/GMT+2",
+                                            "Etc/GMT+3",
+                                            "Etc/GMT+4",
+                                            "Etc/GMT+5",
+                                            "Etc/GMT+6",
+                                            "Etc/GMT+7",
+                                            "Etc/GMT+8",
+                                            "Etc/GMT+9",
+                                            "Etc/GMT-0",
+                                            "Etc/GMT-1",
+                                            "Etc/GMT-10",
+                                            "Etc/GMT-11",
+                                            "Etc/GMT-12",
+                                            "Etc/GMT-13",
+                                            "Etc/GMT-14",
+                                            "Etc/GMT-2",
+                                            "Etc/GMT-3",
+                                            "Etc/GMT-4",
+                                            "Etc/GMT-5",
+                                            "Etc/GMT-6",
+                                            "Etc/GMT-7",
+                                            "Etc/GMT-8",
+                                            "Etc/GMT-9",
+                                            "Etc/GMT0",
+                                            "Etc/Greenwich",
+                                            "Etc/UCT",
+                                            "Etc/UTC",
+                                            "Etc/Universal",
+                                            "Etc/Zulu",
+                                            "Europe/Amsterdam",
+                                            "Europe/Andorra",
+                                            "Europe/Athens",
+                                            "Europe/Belfast",
+                                            "Europe/Belgrade",
+                                            "Europe/Berlin",
+                                            "Europe/Bratislava",
+                                            "Europe/Brussels",
+                                            "Europe/Bucharest",
+                                            "Europe/Budapest",
+                                            "Europe/Chisinau",
+                                            "Europe/Copenhagen",
+                                            "Europe/Dublin",
+                                            "Europe/Gibraltar",
+                                            "Europe/Guernsey",
+                                            "Europe/Helsinki",
+                                            "Europe/Isle_of_Man",
+                                            "Europe/Istanbul",
+                                            "Europe/Jersey",
+                                            "Europe/Kaliningrad",
+                                            "Europe/Kiev",
+                                            "Europe/Lisbon",
+                                            "Europe/Ljubljana",
+                                            "Europe/London",
+                                            "Europe/Luxembourg",
+                                            "Europe/Madrid",
+                                            "Europe/Malta",
+                                            "Europe/Mariehamn",
+                                            "Europe/Minsk",
+                                            "Europe/Monaco",
+                                            "Europe/Moscow",
+                                            "Europe/Nicosia",
+                                            "Europe/Oslo",
+                                            "Europe/Paris",
+                                            "Europe/Podgorica",
+                                            "Europe/Prague",
+                                            "Europe/Riga",
+                                            "Europe/Rome",
+                                            "Europe/Samara",
+                                            "Europe/San_Marino",
+                                            "Europe/Sarajevo",
+                                            "Europe/Simferopol",
+                                            "Europe/Skopje",
+                                            "Europe/Sofia",
+                                            "Europe/Stockholm",
+                                            "Europe/Tallinn",
+                                            "Europe/Tirane",
+                                            "Europe/Tiraspol",
+                                            "Europe/Uzhgorod",
+                                            "Europe/Vaduz",
+                                            "Europe/Vatican",
+                                            "Europe/Vienna",
+                                            "Europe/Vilnius",
+                                            "Europe/Volgograd",
+                                            "Europe/Warsaw",
+                                            "Europe/Zagreb",
+                                            "Europe/Zaporozhye",
+                                            "Europe/Zurich",
+                                            "GB",
+                                            "GB-Eire",
+                                            "GMT",
+                                            "GMT+0",
+                                            "GMT-0",
+                                            "GMT0",
+                                            "Greenwich",
+                                            "HST",
+                                            "Hongkong",
+                                            "Iceland",
+                                            "Indian/Antananarivo",
+                                            "Indian/Chagos",
+                                            "Indian/Christmas",
+                                            "Indian/Cocos",
+                                            "Indian/Comoro",
+                                            "Indian/Kerguelen",
+                                            "Indian/Mahe",
+                                            "Indian/Maldives",
+                                            "Indian/Mauritius",
+                                            "Indian/Mayotte",
+                                            "Indian/Reunion",
+                                            "Iran",
+                                            "Israel",
+                                            "Jamaica",
+                                            "Japan",
+                                            "Kwajalein",
+                                            "Libya",
+                                            "MET",
+                                            "MST",
+                                            "MST7MDT",
+                                            "Mexico/BajaNorte",
+                                            "Mexico/BajaSur",
+                                            "Mexico/General",
+                                            "NZ",
+                                            "NZ-CHAT",
+                                            "Navajo",
+                                            "PRC",
+                                            "PST8PDT",
+                                            "Pacific/Apia",
+                                            "Pacific/Auckland",
+                                            "Pacific/Chatham",
+                                            "Pacific/Chuuk",
+                                            "Pacific/Easter",
+                                            "Pacific/Efate",
+                                            "Pacific/Enderbury",
+                                            "Pacific/Fakaofo",
+                                            "Pacific/Fiji",
+                                            "Pacific/Funafuti",
+                                            "Pacific/Galapagos",
+                                            "Pacific/Gambier",
+                                            "Pacific/Guadalcanal",
+                                            "Pacific/Guam",
+                                            "Pacific/Honolulu",
+                                            "Pacific/Johnston",
+                                            "Pacific/Kiritimati",
+                                            "Pacific/Kosrae",
+                                            "Pacific/Kwajalein",
+                                            "Pacific/Majuro",
+                                            "Pacific/Marquesas",
+                                            "Pacific/Midway",
+                                            "Pacific/Nauru",
+                                            "Pacific/Niue",
+                                            "Pacific/Norfolk",
+                                            "Pacific/Noumea",
+                                            "Pacific/Pago_Pago",
+                                            "Pacific/Palau",
+                                            "Pacific/Pitcairn",
+                                            "Pacific/Pohnpei",
+                                            "Pacific/Ponape",
+                                            "Pacific/Port_Moresby",
+                                            "Pacific/Rarotonga",
+                                            "Pacific/Saipan",
+                                            "Pacific/Samoa",
+                                            "Pacific/Tahiti",
+                                            "Pacific/Tarawa",
+                                            "Pacific/Tongatapu",
+                                            "Pacific/Truk",
+                                            "Pacific/Wake",
+                                            "Pacific/Wallis",
+                                            "Pacific/Yap",
+                                            "Poland",
+                                            "Portugal",
+                                            "ROC",
+                                            "ROK",
+                                            "Singapore",
+                                            "Turkey",
+                                            "UCT",
+                                            "US/Alaska",
+                                            "US/Aleutian",
+                                            "US/Arizona",
+                                            "US/Central",
+                                            "US/East-Indiana",
+                                            "US/Eastern",
+                                            "US/Hawaii",
+                                            "US/Indiana-Starke",
+                                            "US/Michigan",
+                                            "US/Mountain",
+                                            "US/Pacific",
+                                            "US/Pacific-New",
+                                            "US/Samoa",
+                                            "UTC",
+                                            "Universal",
+                                            "W-SU",
+                                            "WET",
+                                            "Zulu"};
+
+    static {
+      for (int i = 0; i < timezoneList.length; i++) {
+        timezoneMap.put(timezoneList[i], i);
+      }
+    }
+
+    public static final DateTimeFormatter formatDate        = DateTimeFormat.forPattern("yyyy-MM-dd");
+    public static final DateTimeFormatter formatTimeStamp    = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS");
+    public static final DateTimeFormatter formatTimeStampTZ = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS ZZZ");
+    public static final DateTimeFormatter formatTime        = DateTimeFormat.forPattern("HH:mm:ss.SSS");
+
+    public static DateTimeFormatter dateTimeTZFormat = null;
+    public static DateTimeFormatter timeFormat = null;
+
+    public static final int yearsToMonths = 12;
+    public static final int hoursToMillis = 60 * 60 * 1000;
+    public static final int minutesToMillis = 60 * 1000;
+    public static final int secondsToMillis = 1000;
+    public static final int monthToStandardDays = 30;
+    public static final long monthsToMillis = 2592000000L; // 30 * 24 * 60 * 60 * 1000
+    public static final int daysToStandardMillis = 24 * 60 * 60 * 1000;
+
+
+  public static int getIndex(String timezone) {
+        return timezoneMap.get(timezone);
+    }
+
+    public static String getTimeZone(int index) {
+        return timezoneList[index];
+    }
+
+    // Function returns the date time formatter used to parse date strings
+    public static DateTimeFormatter getDateTimeFormatter() {
+
+        if (dateTimeTZFormat == null) {
+            DateTimeFormatter dateFormatter = DateTimeFormat.forPattern("yyyy-MM-dd");
+            DateTimeParser optionalTime = DateTimeFormat.forPattern(" HH:mm:ss").getParser();
+            DateTimeParser optionalSec = DateTimeFormat.forPattern(".SSS").getParser();
+            DateTimeParser optionalZone = DateTimeFormat.forPattern(" ZZZ").getParser();
+
+            dateTimeTZFormat = new DateTimeFormatterBuilder().append(dateFormatter).appendOptional(optionalTime).appendOptional(optionalSec).appendOptional(optionalZone).toFormatter();
+        }
+
+        return dateTimeTZFormat;
+    }
+
+    // Function returns time formatter used to parse time strings
+    public static DateTimeFormatter getTimeFormatter() {
+        if (timeFormat == null) {
+            DateTimeFormatter timeFormatter = DateTimeFormat.forPattern("HH:mm:ss");
+            DateTimeParser optionalSec = DateTimeFormat.forPattern(".SSS").getParser();
+            timeFormat = new DateTimeFormatterBuilder().append(timeFormatter).appendOptional(optionalSec).toFormatter();
+        }
+        return timeFormat;
+    }
+
+    public static int monthsFromPeriod(Period period){
+      return (period.getYears() * yearsToMonths) + period.getMonths();
+    }
+
+    public static int millisFromPeriod(final Period period){
+      return (period.getHours() * hoursToMillis) +
+      (period.getMinutes() * minutesToMillis) +
+      (period.getSeconds() * secondsToMillis) +
+      (period.getMillis());
+    }
+
+}