You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pr...@apache.org on 2017/12/21 05:19:31 UTC

[04/15] drill git commit: DRILL-5657: Size-aware vector writer structure

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriterIndex.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriterIndex.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriterIndex.java
new file mode 100644
index 0000000..7e225c9
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriterIndex.java
@@ -0,0 +1,76 @@
+/*
+ * 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.drill.exec.vector.accessor;
+
+/**
+ * A Drill record batch consists of a variety of vectors, including maps and lists.
+ * Each vector is written independently. A reader may skip some values in each row
+ * if no values appear for those columns.
+ * <p>
+ * This index provides a single view of the "current row" or "current array index"
+ * across a set of vectors. Each writer consults this index to determine:
+ * <ul>
+ * <li>The position to which to write a value.</li>
+ * <li>Whether the write position is beyond the "last write" position, which
+ * would require filling in any "missing" values.</li>
+ * </ul>
+ */
+
+public interface ColumnWriterIndex {
+
+  /**
+   * Index of the first entry for the current row
+   * @return index of the first entry for the current row
+   */
+
+  int rowStartIndex();
+
+  /**
+   * Current row or array index.
+   * @return row or array index
+   */
+
+  int vectorIndex();
+
+  /**
+   * Index for array elements that allows the caller to increment the
+   * index. For arrays, writing (or saving) one value automatically
+   * moves to the next value. Ignored for non-element indexes.
+   */
+
+  void nextElement();
+
+  /**
+   * When handling overflow, the index must be reset so that the current row
+   * starts at the start of the vector. Relative offsets must be preserved.
+   * (That is, if the current write position for an array is four greater than
+   * the start, then that offset must now be reset to four from the start of
+   * the vector.)
+   */
+
+  void rollover();
+
+  /**
+   * If this index represents a repeat level, return the index of the
+   * next higher repeat level.
+   *
+   * @return the outer repeat level index, if any
+   */
+
+  ColumnWriterIndex outerIndex();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectReader.java
new file mode 100644
index 0000000..9c53e58
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectReader.java
@@ -0,0 +1,60 @@
+/*
+ * 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.drill.exec.vector.accessor;
+
+/**
+ * Defines a reader to get values for value vectors using
+ * a simple, uniform interface modeled after a JSON object.
+ * Every column value is an object of one of three types:
+ * scalar, array or tuple. Methods exist to "cast" this object
+ * to the proper type. This model allows a very simple representation:
+ * tuples (rows, maps) consist of objects. Arrays are lists of
+ * objects.
+ * <p>
+ * {@see ObjectWriter>
+ */
+
+public interface ObjectReader {
+
+  /**
+   * The type of this reader.
+   *
+   * @return type of reader
+   */
+
+  ObjectType type();
+  ScalarReader scalar();
+  ScalarElementReader elements();
+  TupleReader tuple();
+  ArrayReader array();
+
+  /**
+   * Return the value of the underlying data as a Java object.
+   * Primarily for testing
+   * @return Java object that represents the underlying value
+   */
+
+  Object getObject();
+
+  /**
+   * Return the entire object as a string. Primarily for debugging.
+   * @return string representation of the object
+   */
+
+  String getAsString();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectType.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectType.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectType.java
new file mode 100644
index 0000000..e07ea75
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectType.java
@@ -0,0 +1,28 @@
+/*
+ * 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.drill.exec.vector.accessor;
+
+/**
+ * Type of writer. Follows the JSON-style model, with the
+ * most abstract object types being a scalar (primitive),
+ * tuple (map or row) or an array (repeated type.)
+ */
+
+public enum ObjectType {
+  SCALAR, TUPLE, ARRAY
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java
new file mode 100644
index 0000000..a49b0d8
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java
@@ -0,0 +1,101 @@
+/*
+ * 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.drill.exec.vector.accessor;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
+import org.apache.drill.exec.vector.accessor.TupleWriter.TupleWriterListener;
+
+/**
+ * Represents a column within a tuple. A column can be an array, a scalar or a
+ * tuple. Each has an associated column metadata (schema) and a writer. The
+ * writer is one of three kinds, depending on the kind of the column. If the
+ * column is a map, then the column also has an associated tuple loader to
+ * define and write to the tuple.
+ * <p>
+ * This interface defines a writer to set values for value vectors using a
+ * simple, uniform interface modeled after a JSON object. Every column value is
+ * an object of one of three types: scalar, array or tuple. Methods exist to
+ * "cast" this object to the proper type. This model allows a very simple
+ * representation: tuples (rows, maps) consist of objects. Arrays are lists of
+ * objects.
+ * <p>
+ * Every column resides at an index, is defined by a schema, is backed by a
+ * value vector, and and is written to by a writer. Each column also tracks the
+ * schema version in which it was added to detect schema evolution. Each column
+ * has an optional overflow vector that holds overflow record values when a
+ * batch becomes full.
+ * <p>
+ * {@see ObjectReader}
+ */
+
+public interface ObjectWriter {
+
+  /**
+   * Returns the schema of the column associated with this writer.
+   *
+   * @return schema for this writer's column
+   */
+
+  ColumnMetadata schema();
+
+  /**
+   * Bind a listener to the underlying scalar column, or array of scalar
+   * columns. Not valid if the underlying writer is a map or array of maps.
+   *
+   * @param listener
+   *          the column listener to bind
+   */
+
+  void bindListener(ColumnWriterListener listener);
+
+  /**
+   * Bind a listener to the underlying map or map array column. Not valid if the
+   * underlying writer is a scalar or scalar array.
+   *
+   * @param listener
+   *          the tuple listener to bind
+   */
+
+  void bindListener(TupleWriterListener listener);
+
+  /**
+   * Return the object (structure) type of this writer.
+   *
+   * @return type indicating if this is a scalar, tuple or array
+   */
+
+  ObjectType type();
+
+  ScalarWriter scalar();
+
+  TupleWriter tuple();
+
+  ArrayWriter array();
+
+  /**
+   * For debugging, set the object to the proper form of Java object as defined
+   * by the underlying writer type.
+   *
+   * @param value
+   *          Java object value to write
+   * @throws VectorOverflowException
+   */
+
+  void set(Object value);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarElementReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarElementReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarElementReader.java
new file mode 100644
index 0000000..d1f31a8
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarElementReader.java
@@ -0,0 +1,65 @@
+/*
+ * 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.drill.exec.vector.accessor;
+
+import java.math.BigDecimal;
+
+import org.joda.time.Period;
+
+/**
+ * Interface to access the values of an array column. In general, each
+ * vector implements just one of the get methods. Check the vector type
+ * to know which method to use. Though, generally, when writing test
+ * code, the type is known to the test writer.
+ * <p>
+ * Arrays allow random access to the values within the array. The index
+ * passed to each method is the index into the array for the current
+ * row and column. (This means that arrays are three dimensional:
+ * the usual (row, column) dimensions plus an array index dimension:
+ * (row, column, array index).
+ * <p>
+ * Note that the <tt>isNull()</tt> method is provided for completeness,
+ * but no Drill array allows null values at present.
+ * <p>
+ * {@see ScalarWriter}
+ */
+
+public interface ScalarElementReader {
+  /**
+   * Describe the type of the value. This is a compression of the
+   * value vector type: it describes which method will return the
+   * vector value.
+   * @return the value type which indicates which get method
+   * is valid for the column
+   */
+
+  ValueType valueType();
+  int size();
+
+  boolean isNull(int index);
+  int getInt(int index);
+  long getLong(int index);
+  double getDouble(int index);
+  String getString(int index);
+  byte[] getBytes(int index);
+  BigDecimal getDecimal(int index);
+  Period getPeriod(int index);
+
+  Object getObject(int index);
+  String getAsString(int index);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarReader.java
new file mode 100644
index 0000000..e1c26bf
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarReader.java
@@ -0,0 +1,75 @@
+/*
+ * 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.drill.exec.vector.accessor;
+
+import java.math.BigDecimal;
+
+import org.joda.time.Period;
+
+/**
+ * Defines a reader to obtain values from value vectors using
+ * a simple, uniform interface. Vector values are mapped to
+ * their "natural" representations: the representation closest
+ * to the actual vector value. For date and time values, this
+ * generally means a numeric value. Applications can then map
+ * this value to Java objects as desired. Decimal types all
+ * map to BigDecimal as that is the only way in Java to
+ * represent large decimal values.
+ * <p>
+ * In general, a column maps to just one value. However, derived
+ * classes may choose to provide type conversions if convenient.
+ * An exception is thrown if a call is made to a method that
+ * is not supported by the column type.
+ * <p>
+ * Values of scalars are provided directly, using the get method
+ * for the target type. Maps and arrays are structured types and
+ * require another level of reader abstraction to access each value
+ * in the structure.
+ * <p>
+ * {@see ScalarWriter}
+ */
+
+public interface ScalarReader {
+  /**
+   * Describe the type of the value. This is a compression of the
+   * value vector type: it describes which method will return the
+   * vector value.
+   * @return the value type which indicates which get method
+   * is valid for the column
+   */
+
+  ValueType valueType();
+
+  /**
+   * Report if the column is null. Non-nullable columns always
+   * return <tt>false</tt>.
+   * @return true if the column value is null, false if the
+   * value is set
+   */
+  boolean isNull();
+  int getInt();
+  long getLong();
+  double getDouble();
+  String getString();
+  byte[] getBytes();
+  BigDecimal getDecimal();
+  Period getPeriod();
+
+  Object getObject();
+  String getAsString();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarWriter.java
index 5cbe80a..776dc9c 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarWriter.java
@@ -22,16 +22,81 @@ import java.math.BigDecimal;
 import org.joda.time.Period;
 
 /**
- * Methods common to the {@link ColumnWriter} and
- * {@link ArrayWriter} interfaces.
+ * Represents a scalar value: a required column, a nullable column,
+ * or one element within an array of scalars.
+ * <p>
+ * Vector values are mapped to
+ * their "natural" representations: the representation closest
+ * to the actual vector value. For date and time values, this
+ * generally means a numeric value. Applications can then map
+ * this value to Java objects as desired. Decimal types all
+ * map to BigDecimal as that is the only way in Java to
+ * represent large decimal values.
+ * <p>
+ * In general, a column maps to just one value. However, derived
+ * classes may choose to provide type conversions if convenient.
+ * An exception is thrown if a call is made to a method that
+ * is not supported by the column type.
+ * <p>
+ * {@see ScalarReader}
+ * {@see ScalarElementReader}
  */
 
 public interface ScalarWriter {
+
+  /**
+   * Listener (callback) for vector overflow events. To be optionally
+   * implemented and bound by the client code of the writer. If no
+   * listener is bound, and a vector overflows, then an exception is
+   * thrown.
+   */
+
+  public interface ColumnWriterListener {
+
+    /**
+     * Alert the listener that a vector has overflowed. Upon return,
+     * all writers must have a new set of buffers available, ready
+     * to accept the in-flight value that triggered the overflow.
+     *
+     * @param writer the writer that triggered the overflow
+     */
+
+    void overflowed(ScalarWriter writer);
+
+    /**
+     * A writer wants to expand its vector. Allows the listener to
+     * either allow the growth, or trigger and overflow to limit
+     * batch size.
+     *
+     * @param writer the writer that wishes to grow its vector
+     * @param delta the amount by which the vector is to grow
+     * @return true if the vector can be grown, false if the writer
+     * should instead trigger an overflow by calling
+     * <tt>overflowed()</tt>
+     */
+
+    boolean canExpand(ScalarWriter writer, int delta);
+  }
+
+  void bindListener(ColumnWriterListener listener);
+
+  /**
+   * Describe the type of the value. This is a compression of the
+   * value vector type: it describes which method will return the
+   * vector value.
+   * @return the value type which indicates which get method
+   * is valid for the column
+   */
+
+  ValueType valueType();
+  void setNull();
   void setInt(int value);
   void setLong(long value);
   void setDouble(double value);
   void setString(String value);
-  void setBytes(byte[] value);
+  void setBytes(byte[] value, int len);
   void setDecimal(BigDecimal value);
   void setPeriod(Period value);
+
+  void setObject(Object value);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleAccessor.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleAccessor.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleAccessor.java
deleted file mode 100644
index ea9b869..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleAccessor.java
+++ /dev/null
@@ -1,71 +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.drill.exec.vector.accessor;
-
-import org.apache.drill.exec.record.MaterializedField;
-
-/**
- * Provides access to a "tuple". In Drill, both rows and maps are
- * tuples: both are an ordered collection of values, defined by a
- * schema. Each tuple has a schema that defines the column ordering
- * for indexed access. Each tuple also provides methods to get column
- * accessors by name or index.
- */
-
-public interface TupleAccessor {
-
-  /**
-   * Flattened view of the schema as needed for row-based access of scalar
-   * members. The scalar view presents scalar fields: those that can be set
-   * or retrieved. A separate map view presents map vectors. The scalar
-   * view is the one used by row set readers and writers. Column indexes
-   * are into the flattened view, with maps removed and map members flattened
-   * into the top-level name space with compound names.
-   */
-
-  public interface TupleSchema {
-    /**
-     * Return a column schema given an indexed into the flattened row structure.
-     *
-     * @param index index of the row in the flattened structure
-     * @return schema of the column
-     */
-
-    MaterializedField column(int index);
-
-    /**
-     * Returns {@code MaterializedField} instance from schema using the name specified in param.
-     *
-     * @param name name of the column in the schema
-     * @return {@code MaterializedField} instance
-     */
-    MaterializedField column(String name);
-
-    /**
-     * Returns index of the column in the schema with name specified in param.
-     *
-     * @param name name of the column in the schema
-     * @return index of the column in the schema
-     */
-    int columnIndex(String name);
-
-    int count();
-  }
-
-  TupleSchema schema();
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java
index acca767..908d6a0 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java
@@ -17,24 +17,38 @@
  */
 package org.apache.drill.exec.vector.accessor;
 
+import org.apache.drill.exec.record.TupleMetadata;
+
 /**
  * Interface for reading from tuples (rows or maps). Provides
  * a column reader for each column that can be obtained either
  * by name or column index (as defined in the tuple schema.)
  * Also provides two generic methods to get the value as a
  * Java object or as a string.
+ * <p>
+ * {@see TupleWriter}
  */
 
-public interface TupleReader extends TupleAccessor {
-  ColumnReader column(int colIndex);
+public interface TupleReader {
+  TupleMetadata schema();
+  int columnCount();
+
+  ObjectReader column(int colIndex);
+  ObjectReader column(String colName);
+
+  // Convenience methods
+
+  ObjectType type(int colIndex);
+  ObjectType type(String colName);
+  ScalarReader scalar(int colIndex);
+  ScalarReader scalar(String colName);
+  TupleReader tuple(int colIndex);
+  TupleReader tuple(String colName);
+  ArrayReader array(int colIndex);
+  ArrayReader array(String colName);
+  ScalarElementReader elements(int colIndex);
+  ScalarElementReader elements(String colName);
 
-  /**
-   * Returns column reader for the column with name specified in param.
-   *
-   * @param colName name of the column in the schema
-   * @return column reader
-   */
-  ColumnReader column(String colName);
-  Object get(int colIndex);
-  String getAsString(int colIndex);
+  Object getObject();
+  String getAsString();
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java
index 563734e..056c9b3 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java
@@ -17,25 +17,153 @@
  */
 package org.apache.drill.exec.vector.accessor;
 
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+
 /**
- * Interface for writing to rows via a column writer.
- * Column writers can be obtained by name or index. Column
- * indexes are defined by the tuple schema. Also provides
- * a convenience method to set the column value from a Java
- * object. The caller is responsible for providing the
- * correct object type for each column. (The object type
- * must match the column accessor type.)
+ * Writer for a tuple. A tuple is composed of columns with a fixed order and
+ * unique names: either can be used to reference columns. Columns are scalar
+ * (simple values), tuples (i.e. maps), or arrays (of scalars, tuples or
+ * arrays.) The row itself is just the top-level (anonymous) tuple. Generally,
+ * implementers of this interface provide additional services on the
+ * implementation of the top-level tuple (often called a "row writer.") Columns
+ * are accessible via the associated column writer by name or index. Column
+ * indexes are defined by the tuple schema.
+ * <p>
+ * Consumers of this interface can define the schema up front, or can define the
+ * schema as the write progresses. To avoid redundant checks to see if a column
+ * is already defined, consumers can simply ask for a column by name. The
+ * <tt>column()</tt> (and related) methods will throw an (unchecked)
+ * {@link UndefinedColumnException} exception if the column is undefined. The
+ * consumer can catch the exception, define the column, and fetch the column
+ * writer again. New columns may be added via this interface at any time; the
+ * new column takes the next available index.
+ * <p>
+ * Also provides a convenience method to set the column value from a Java
+ * object. The caller is responsible for providing the correct object type for
+ * each column. (The object type must match the column accessor type.)
+ * <p>
+ * Convenience methods allow getting a column as a scalar, tuple or array. These
+ * methods throw an exception if the column is not of the requested type.
+ *
+ * @see {@link SingleMapWriter}, the class which this class replaces
  */
 
-public interface TupleWriter extends TupleAccessor {
-  ColumnWriter column(int colIndex);
+public interface TupleWriter {
+
+  /**
+   * Listener (callback) to handle requests to add a new column to a tuple (row
+   * or map). Implemented and bound by the client code that creates or uses the
+   * tuple writer. If no listener is bound, then an attempt to add a column
+   * throws an exception.
+   */
+
+  public interface TupleWriterListener {
+    ObjectWriter addColumn(TupleWriter tuple, ColumnMetadata column);
+
+    ObjectWriter addColumn(TupleWriter tuple, MaterializedField field);
+  }
+
+  /**
+   * Unchecked exception thrown when attempting to access a column writer by
+   * name for an undefined columns. Clients that use a fixed schema can simply
+   * omit catch blocks for the exception since it is unchecked and won't be
+   * thrown if the schema can't evolve. Clients that can discover new columns
+   * should catch the exception and define the column (using an implementation
+   * that allows dynamic schema definition.)
+   */
+
+  @SuppressWarnings("serial")
+  public static class UndefinedColumnException extends RuntimeException {
+    public UndefinedColumnException(String colName) {
+      super("Undefined column: " + colName);
+    }
+  }
+
+  void bindListener(TupleWriterListener listener);
 
   /**
-   * Returns column writer for the column with name specified in param.
+   * Add a column to the tuple (row or map) that backs this writer. Support for
+   * this operation depends on whether the client code has registered a listener
+   * to implement the addition. Throws an exception if no listener is
+   * implemented, or if the add request is otherwise invalid (duplicate name,
+   * etc.)
    *
-   * @param colName name of the column in the schema
-   * @return column writer
+   * @param column
+   *          the metadata for the column to add
+   * @return the index of the newly added column which can be used to access the
+   *         newly added writer
    */
-  ColumnWriter column(String colName);
+
+  int addColumn(ColumnMetadata column);
+
+  int addColumn(MaterializedField schema);
+
+  TupleMetadata schema();
+
+  int size();
+
+  // Return the column as a generic object
+
+  ObjectWriter column(int colIndex);
+
+  ObjectWriter column(String colName);
+
+  // Convenience methods
+
+  ScalarWriter scalar(int colIndex);
+
+  ScalarWriter scalar(String colName);
+
+  TupleWriter tuple(int colIndex);
+
+  TupleWriter tuple(String colName);
+
+  ArrayWriter array(int colIndex);
+
+  ArrayWriter array(String colName);
+
+  ObjectType type(int colIndex);
+
+  ObjectType type(String colName);
+
+  /**
+   * Write a value to the given column, automatically calling the proper
+   * <tt>set<i>Type</i></tt> method for the data. While this method is
+   * convenient for testing, it incurs quite a bit of type-checking overhead and
+   * is not suitable for production code.
+   *
+   * @param colIndex
+   *          the index of the column to set
+   * @param value
+   *          the value to set. The type of the object must be compatible with
+   *          the type of the target column
+   */
+
   void set(int colIndex, Object value);
+
+  /**
+   * Write a row or map of values, given by Java objects. Object type must match
+   * expected column type.
+   * <p>
+   * Note that a single-column tuple is ambiguous if that column is an array. To
+   * avoid ambiguity, use <tt>set(0, value)</tt> in this case.
+   *
+   * @param values
+   *          variable-length argument list of column values
+   * @return true if the row was written, false if any column caused vector
+   *         overflow.
+   */
+
+  void setTuple(Object... values);
+
+  /**
+   * Set the tuple from an array of objects. Primarily for use in test tools.
+   *
+   * @param value
+   *          the object to set, which must be a generic <tt>Object</tt> array
+   */
+
+  void setObject(Object value);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ValueType.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ValueType.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ValueType.java
new file mode 100644
index 0000000..e6687dc
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ValueType.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.drill.exec.vector.accessor;
+
+/**
+ * Represents the primitive types supported to read and write data
+ * from value vectors. Vectors support many data widths. For simplicity
+ * (and because of no difference in performance), the get/set methods
+ * use a reduced set of types. In general, each reader and writer
+ * supports just one type. Though some may provide more than one
+ * (such as access to bytes for a <tt>STRING</tt> value.)
+ */
+
+public enum ValueType {
+  INTEGER, LONG, DOUBLE, STRING, BYTES, DECIMAL, PERIOD
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayReader.java
deleted file mode 100644
index deea7f8..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayReader.java
+++ /dev/null
@@ -1,128 +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.drill.exec.vector.accessor.impl;
-
-import java.math.BigDecimal;
-
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.accessor.ArrayReader;
-import org.apache.drill.exec.vector.accessor.TupleReader;
-import org.apache.drill.exec.vector.accessor.impl.AbstractColumnReader.VectorAccessor;
-import org.joda.time.Period;
-
-/**
- * Reader for an array-valued column. This reader provides access to specific
- * array members via an array index. This is an abstract base class;
- * subclasses are generated for each repeated value vector type.
- */
-
-public abstract class AbstractArrayReader extends AbstractColumnAccessor implements ArrayReader {
-
-  /**
-   * Column reader that provides access to an array column by returning a
-   * separate reader specifically for that array. That is, reading a column
-   * is a two-part process:<pre><code>
-   * tupleReader.column("arrayCol").array().getInt(2);</code></pre>
-   * This pattern is used to avoid overloading the column reader with
-   * both scalar and array access. Also, this pattern mimics the way
-   * that nested tuples (Drill maps) are handled.
-   */
-
-  public static class ArrayColumnReader extends AbstractColumnReader {
-
-    private final AbstractArrayReader arrayReader;
-
-    public ArrayColumnReader(AbstractArrayReader arrayReader) {
-      this.arrayReader = arrayReader;
-    }
-
-    @Override
-    public ValueType valueType() {
-       return ValueType.ARRAY;
-    }
-
-    @Override
-    public void bind(RowIndex rowIndex, ValueVector vector) {
-      arrayReader.bind(rowIndex, vector);
-      vectorIndex = rowIndex;
-    }
-
-    @Override
-    public ArrayReader array() {
-      return arrayReader;
-    }
-  }
-
-  protected VectorAccessor vectorAccessor;
-
-  public void bind(RowIndex rowIndex, MaterializedField field, VectorAccessor va) {
-    bind(rowIndex);
-    vectorAccessor = va;
-  }
-
-  @Override
-  public boolean isNull(int index) {
-    return false;
-  }
-
-  @Override
-  public int getInt(int index) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public long getLong(int index) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public double getDouble(int index) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public String getString(int index) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public byte[] getBytes(int index) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public BigDecimal getDecimal(int index) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Period getPeriod(int index) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public TupleReader map(int index) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ArrayReader array(int index) {
-    throw new UnsupportedOperationException();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayWriter.java
deleted file mode 100644
index d1d1263..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayWriter.java
+++ /dev/null
@@ -1,127 +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.drill.exec.vector.accessor.impl;
-
-import java.math.BigDecimal;
-
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.accessor.ArrayWriter;
-import org.apache.drill.exec.vector.complex.BaseRepeatedValueVector;
-import org.joda.time.Period;
-
-/**
- * Writer for an array-valued column. This writer appends values: once a value
- * is written, it cannot be changed. As a result, writer methods have no item index;
- * each set advances the array to the next position. This is an abstract base class;
- * subclasses are generated for each repeated value vector type.
- */
-
-public abstract class AbstractArrayWriter extends AbstractColumnAccessor implements ArrayWriter {
-
-  /**
-   * Column writer that provides access to an array column by returning a
-   * separate writer specifically for that array. That is, writing an array
-   * is a two-part process:<pre><code>
-   * tupleWriter.column("arrayCol").array().setInt(2);</code></pre>
-   * This pattern is used to avoid overloading the column reader with
-   * both scalar and array access. Also, this pattern mimics the way
-   * that nested tuples (Drill maps) are handled.
-   */
-
-  public static class ArrayColumnWriter extends AbstractColumnWriter {
-
-    private final AbstractArrayWriter arrayWriter;
-
-    public ArrayColumnWriter(AbstractArrayWriter arrayWriter) {
-      this.arrayWriter = arrayWriter;
-    }
-
-    @Override
-    public ValueType valueType() {
-      return ValueType.ARRAY;
-    }
-
-    @Override
-    public void bind(RowIndex rowIndex, ValueVector vector) {
-      arrayWriter.bind(rowIndex, vector);
-      vectorIndex = rowIndex;
-    }
-
-    @Override
-    public ArrayWriter array() {
-      return arrayWriter;
-    }
-
-    /**
-     * Arrays require a start step for each row, regardless of
-     * whether any values are written for that row.
-     */
-
-    public void start() {
-      arrayWriter.mutator().startNewValue(vectorIndex.index());
-    }
-  }
-
-  protected abstract BaseRepeatedValueVector.BaseRepeatedMutator mutator();
-
-  @Override
-  public int size() {
-    return mutator().getInnerValueCountAt(vectorIndex.index());
-  }
-
-  @Override
-  public boolean valid() {
-    // Not implemented yet
-    return true;
-  }
-
-  @Override
-  public void setInt(int value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setLong(long value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setDouble(double value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setString(String value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setBytes(byte[] value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setDecimal(BigDecimal value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setPeriod(Period value) {
-    throw new UnsupportedOperationException();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnAccessor.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnAccessor.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnAccessor.java
deleted file mode 100644
index 5b751c5..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnAccessor.java
+++ /dev/null
@@ -1,43 +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.drill.exec.vector.accessor.impl;
-
-import org.apache.drill.exec.vector.ValueVector;
-
-/**
- * Abstract base class for column readers and writers that
- * implements the mechanism for binding accessors to a row
- * index. The row index is implicit: index a row, then
- * column accessors pull out columns from that row.
- */
-
-public abstract class AbstractColumnAccessor {
-
-  public interface RowIndex {
-    int batch();
-    int index();
-  }
-
-  protected RowIndex vectorIndex;
-
-  protected void bind(RowIndex rowIndex) {
-    this.vectorIndex = rowIndex;
-  }
-
-  public abstract void bind(RowIndex rowIndex, ValueVector vector);
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnReader.java
deleted file mode 100644
index b88b08b..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnReader.java
+++ /dev/null
@@ -1,126 +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.drill.exec.vector.accessor.impl;
-
-import java.math.BigDecimal;
-
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.accessor.ArrayReader;
-import org.apache.drill.exec.vector.accessor.ColumnReader;
-import org.apache.drill.exec.vector.accessor.TupleReader;
-import org.joda.time.Period;
-
-/**
- * Column reader implementation that acts as the basis for the
- * generated, vector-specific implementations. All set methods
- * throw an exception; subclasses simply override the supported
- * method(s).
- */
-
-public abstract class AbstractColumnReader extends AbstractColumnAccessor implements ColumnReader {
-
-  public interface VectorAccessor {
-    ValueVector vector();
-  }
-
-  protected VectorAccessor vectorAccessor;
-
-  public void bind(RowIndex rowIndex, MaterializedField field, VectorAccessor va) {
-    bind(rowIndex);
-    vectorAccessor = va;
-  }
-
-  @Override
-  public Object getObject() {
-    switch (valueType()) {
-    case ARRAY:
-      // TODO: build an array. Just a bit tedious...
-      throw new UnsupportedOperationException();
-    case BYTES:
-      return getBytes();
-    case DECIMAL:
-      return getDecimal();
-    case DOUBLE:
-      return getDouble();
-    case INTEGER:
-      return getInt();
-    case LONG:
-      return getLong();
-    case MAP:
-      // TODO: build an array. Just a bit tedious...
-      throw new UnsupportedOperationException();
-    case PERIOD:
-      return getPeriod();
-    case STRING:
-      return getString();
-    default:
-      throw new IllegalStateException("Unexpected type: " + valueType());
-    }
-  }
-
-  @Override
-  public boolean isNull() {
-    return false;
-  }
-
-  @Override
-  public int getInt() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public long getLong() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public double getDouble() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public String getString() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public byte[] getBytes() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public BigDecimal getDecimal() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Period getPeriod() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public TupleReader map() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ArrayReader array() {
-    throw new UnsupportedOperationException();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnWriter.java
deleted file mode 100644
index 5071e03..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnWriter.java
+++ /dev/null
@@ -1,87 +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.drill.exec.vector.accessor.impl;
-
-import java.math.BigDecimal;
-
-import org.apache.drill.exec.vector.accessor.ArrayWriter;
-import org.apache.drill.exec.vector.accessor.ColumnWriter;
-import org.apache.drill.exec.vector.accessor.TupleWriter;
-import org.joda.time.Period;
-
-/**
- * Column writer implementation that acts as the basis for the
- * generated, vector-specific implementations. All set methods
- * throw an exception; subclasses simply override the supported
- * method(s).
- */
-
-public abstract class AbstractColumnWriter extends AbstractColumnAccessor implements ColumnWriter {
-
-  public void start() { }
-
-  @Override
-  public void setNull() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setInt(int value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setLong(long value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setDouble(double value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setString(String value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setBytes(byte[] value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setDecimal(BigDecimal value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setPeriod(Period value) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public TupleWriter map() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ArrayWriter array() {
-    throw new UnsupportedOperationException();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractTupleAccessor.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractTupleAccessor.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractTupleAccessor.java
deleted file mode 100644
index 98ea6ac..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractTupleAccessor.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.drill.exec.vector.accessor.impl;
-
-import org.apache.drill.exec.vector.accessor.TupleAccessor;
-
-/**
- * Common base class for tuple readers and writers.
- */
-
-public abstract class AbstractTupleAccessor implements TupleAccessor {
-
-  protected final TupleSchema schema;
-
-  public AbstractTupleAccessor(TupleSchema schema) {
-    this.schema = schema;
-  }
-
-  @Override
-  public TupleSchema schema() {
-    return schema;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AccessorUtilities.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AccessorUtilities.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AccessorUtilities.java
new file mode 100644
index 0000000..4bf8f98
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AccessorUtilities.java
@@ -0,0 +1,53 @@
+/*
+ * 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.drill.exec.vector.accessor.impl;
+
+public class AccessorUtilities {
+
+  private AccessorUtilities() { }
+
+  public static int sv4Batch(int sv4Index) {
+    return sv4Index >>> 16;
+  }
+
+  public static int sv4Index(int sv4Index) {
+    return sv4Index & 0xFFFF;
+  }
+
+  public static String bytesToString(byte[] value) {
+    StringBuilder buf = new StringBuilder()
+        .append("[");
+    int len = Math.min(value.length, 20);
+    for (int i = 0; i < len;  i++) {
+      if (i > 0) {
+        buf.append(", ");
+      }
+      String str = Integer.toHexString(value[i] & 0xFF);
+      if (str.length() < 2) {
+        buf.append("0");
+      }
+      buf.append(str);
+    }
+    if (value.length > len) {
+      buf.append("...");
+    }
+    buf.append("]");
+    return buf.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/ColumnAccessorFactory.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/ColumnAccessorFactory.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/ColumnAccessorFactory.java
deleted file mode 100644
index 019d3be..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/ColumnAccessorFactory.java
+++ /dev/null
@@ -1,122 +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.drill.exec.vector.accessor.impl;
-
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.vector.accessor.ColumnAccessors;
-import org.apache.drill.exec.vector.accessor.impl.AbstractArrayReader.ArrayColumnReader;
-import org.apache.drill.exec.vector.accessor.impl.AbstractArrayWriter.ArrayColumnWriter;
-
-/**
- * Gather generated accessor classes into a set of class
- * tables to allow rapid run-time creation of accessors.
- * The caller is responsible for binding the accessor to
- * a vector and a row index.
- */
-
-public class ColumnAccessorFactory {
-
-  private static Class<? extends AbstractColumnWriter> columnWriters[][] = buildColumnWriters();
-  private static Class<? extends AbstractColumnReader> columnReaders[][] = buildColumnReaders();
-  private static Class<? extends AbstractArrayWriter> arrayWriters[] = buildArrayWriters();
-  private static Class<? extends AbstractArrayReader> arrayReaders[] = buildArrayReaders();
-
-  @SuppressWarnings("unchecked")
-  private static Class<? extends AbstractColumnWriter>[][] buildColumnWriters() {
-    int typeCount = MinorType.values().length;
-    int modeCount = DataMode.values().length;
-    Class<? extends AbstractColumnWriter> writers[][] = new Class[typeCount][];
-    for (int i = 0; i < typeCount; i++) {
-      writers[i] = new Class[modeCount];
-    }
-
-    ColumnAccessors.defineWriters(writers);
-    return writers;
-  }
-
-  @SuppressWarnings("unchecked")
-  private static Class<? extends AbstractColumnReader>[][] buildColumnReaders() {
-    int typeCount = MinorType.values().length;
-    int modeCount = DataMode.values().length;
-    Class<? extends AbstractColumnReader> readers[][] = new Class[typeCount][];
-    for (int i = 0; i < typeCount; i++) {
-      readers[i] = new Class[modeCount];
-    }
-
-    ColumnAccessors.defineReaders(readers);
-    return readers;
-  }
-
-  @SuppressWarnings("unchecked")
-  private static Class<? extends AbstractArrayWriter>[] buildArrayWriters() {
-    int typeCount = MinorType.values().length;
-    Class<? extends AbstractArrayWriter> writers[] = new Class[typeCount];
-    ColumnAccessors.defineArrayWriters(writers);
-    return writers;
-  }
-
-  @SuppressWarnings("unchecked")
-  private static Class<? extends AbstractArrayReader>[] buildArrayReaders() {
-    int typeCount = MinorType.values().length;
-    Class<? extends AbstractArrayReader> readers[] = new Class[typeCount];
-    ColumnAccessors.defineArrayReaders(readers);
-    return readers;
-  }
-
-  public static AbstractColumnWriter newWriter(MajorType type) {
-    try {
-      if (type.getMode() == DataMode.REPEATED) {
-        Class<? extends AbstractArrayWriter> writerClass = arrayWriters[type.getMinorType().ordinal()];
-        if (writerClass == null) {
-          throw new UnsupportedOperationException();
-        }
-        return new ArrayColumnWriter(writerClass.newInstance());
-      } else {
-        Class<? extends AbstractColumnWriter> writerClass = columnWriters[type.getMinorType().ordinal()][type.getMode().ordinal()];
-        if (writerClass == null) {
-          throw new UnsupportedOperationException();
-        }
-        return writerClass.newInstance();
-      }
-    } catch (InstantiationException | IllegalAccessException e) {
-      throw new IllegalStateException(e);
-    }
-  }
-
-  public static AbstractColumnReader newReader(MajorType type) {
-    try {
-      if (type.getMode() == DataMode.REPEATED) {
-        Class<? extends AbstractArrayReader> readerClass = arrayReaders[type.getMinorType().ordinal()];
-        if (readerClass == null) {
-          throw new UnsupportedOperationException();
-        }
-        return new ArrayColumnReader(readerClass.newInstance());
-      } else {
-        Class<? extends AbstractColumnReader> readerClass = columnReaders[type.getMinorType().ordinal()][type.getMode().ordinal()];
-        if (readerClass == null) {
-          throw new UnsupportedOperationException();
-        }
-        return readerClass.newInstance();
-      }
-    } catch (InstantiationException | IllegalAccessException e) {
-      throw new IllegalStateException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalFormatter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalFormatter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalFormatter.java
new file mode 100644
index 0000000..35bd445
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalFormatter.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.drill.exec.vector.accessor.impl;
+
+/**
+ * Interface for dumping object state in a hierarchical fashion during
+ * debugging.
+ */
+
+public interface HierarchicalFormatter {
+
+  HierarchicalFormatter startObject(Object obj);
+  void extend();
+  HierarchicalFormatter attribute(String label);
+  HierarchicalFormatter attribute(String label, Object value);
+  HierarchicalFormatter attributeIdentity(String label, Object value);
+  HierarchicalFormatter attributeArray(String label);
+  HierarchicalFormatter element(int index, Object value);
+  HierarchicalFormatter elementIdentity(int index, Object value);
+  HierarchicalFormatter element(int index);
+  HierarchicalFormatter endArray();
+  HierarchicalFormatter endObject();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalPrinter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalPrinter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalPrinter.java
new file mode 100644
index 0000000..0c37d23
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalPrinter.java
@@ -0,0 +1,238 @@
+/*
+ * 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.drill.exec.vector.accessor.impl;
+
+import java.io.PrintStream;
+import java.util.ArrayDeque;
+import java.util.Deque;
+
+/**
+ * Prints a complex object structure in a quasi-JSON format for use
+ * in debugging. Generally only used in an ad-hoc fashion during
+ * debugging sessions; never in production.
+ */
+
+public class HierarchicalPrinter implements HierarchicalFormatter {
+
+  public enum State { OBJECT, OBJECT_ATTRIB, ARRAY, OBJECT_ELEMENT }
+
+  private static class ObjState {
+    private State state = State.OBJECT;
+    private String attrib;
+    private int index = -1;
+    private int extensions;
+
+    public ObjState(int extensions) {
+      this.extensions = extensions;
+    }
+  }
+
+  private final PrintStream out;
+  private Deque<ObjState> stack = new ArrayDeque<ObjState>();
+  private int pendingExtensions = 0;
+  private ObjState curObject;
+  private int level;
+
+  public HierarchicalPrinter() {
+    out = System.out;
+  }
+
+  @Override
+  public void extend() {
+    pendingExtensions++;
+  }
+
+  @Override
+  public HierarchicalFormatter startObject(Object obj) {
+    if (curObject != null) {
+      stack.push(curObject);
+      switch (curObject.state) {
+      case OBJECT_ATTRIB:
+        startAttrib(curObject.attrib);
+        curObject.attrib = null;
+        curObject.state = State.OBJECT;
+        break;
+      case OBJECT:
+        startAttrib("missing-attrib");
+        curObject.state = State.OBJECT;
+        break;
+      case OBJECT_ELEMENT:
+        startElement(curObject.index);
+        curObject.state = State.ARRAY;
+        curObject.index = -1;
+        break;
+      default:
+        assert false;
+      }
+    }
+
+    printObjIdentity(obj);
+    out.println(" {");
+    level++;
+    curObject = new ObjState(pendingExtensions);
+    pendingExtensions = 0;
+    return this;
+  }
+
+  private void printObjIdentity(Object value) {
+    out.print(value.getClass().getSimpleName());
+    out.print( " (");
+    out.print(System.identityHashCode(value) % 1000);
+    out.print(")");
+  }
+
+  private void startElement(int index) {
+    indent();
+    out.print("[");
+    out.print(index);
+    out.print("] = ");
+  }
+
+  private void startAttrib(String label) {
+    indent();
+    out.print(label);
+    out.print(" = ");
+  }
+
+  @Override
+  public HierarchicalFormatter attribute(String label) {
+    curObject.attrib = label;
+    curObject.state = State.OBJECT_ATTRIB;
+    return this;
+  }
+
+  @Override
+  public HierarchicalFormatter attribute(String label, Object value) {
+    attribPrefix();
+    startAttrib(label);
+    printValue(value);
+    out.println();
+    return this;
+  }
+
+  private void attribPrefix() {
+    switch (curObject.state) {
+    case OBJECT_ATTRIB:
+      startAttrib(curObject.attrib);
+      out.println("<Unknown> {}");
+      break;
+    case OBJECT:
+      break;
+    default:
+      assert false;
+    }
+  }
+
+  @Override
+  public HierarchicalFormatter attributeIdentity(String label, Object obj) {
+    attribPrefix();
+    startAttrib(label);
+    objIdentity(obj);
+    out.println();
+    return this;
+  }
+
+  private void objIdentity(Object obj) {
+    if (obj == null) {
+      out.print("null");
+    } else {
+      printObjIdentity(obj);
+    }
+  }
+
+  private void printValue(Object value) {
+    if (value == null) {
+      out.print("null");
+    } else if (value instanceof String) {
+      out.print("\"");
+      out.print(value);
+      out.print("\"");
+    } else {
+      out.print(value.toString());
+    }
+  }
+
+  @Override
+  public HierarchicalFormatter endObject() {
+    if (level == 0) {
+      out.println( "} // Mismatch!");
+      return this;
+    }
+    if (curObject.extensions == 0) {
+      level--;
+      indent();
+      out.println("}");
+      if (level == 0) {
+        curObject = null;
+      } else {
+        curObject = stack.pop();
+      }
+    } else {
+      curObject.extensions--;
+    }
+    return this;
+  }
+
+  private void indent() {
+    for (int i = 0; i < level; i++) {
+      out.print("  ");
+    }
+  }
+
+  @Override
+  public HierarchicalFormatter attributeArray(String label) {
+    startAttrib(label);
+    out.println("[");
+    level++;
+    curObject.state = State.ARRAY;
+    return this;
+  }
+
+  @Override
+  public HierarchicalFormatter element(int index, Object value) {
+    startElement(index);
+    printValue(value);
+    out.println();
+    return this;
+  }
+
+  @Override
+  public HierarchicalFormatter element(int index) {
+    curObject.index = index;
+    curObject.state = State.OBJECT_ELEMENT;
+    return this;
+  }
+
+  @Override
+  public HierarchicalFormatter elementIdentity(int index, Object obj) {
+    startElement(index);
+    objIdentity(obj);
+    out.println();
+    return this;
+  }
+
+  @Override
+  public HierarchicalFormatter endArray() {
+    level--;
+    indent();
+    out.println("]");
+    curObject.state = State.OBJECT;
+    return this;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleReaderImpl.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleReaderImpl.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleReaderImpl.java
deleted file mode 100644
index 97a6e3c..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleReaderImpl.java
+++ /dev/null
@@ -1,151 +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.drill.exec.vector.accessor.impl;
-
-import org.apache.drill.exec.vector.accessor.ArrayReader;
-import org.apache.drill.exec.vector.accessor.ColumnReader;
-import org.apache.drill.exec.vector.accessor.TupleReader;
-
-/**
- * Reader for a tuple (a row or a map.) Provides access to each
- * column using either a name or a numeric index.
- */
-
-public class TupleReaderImpl extends AbstractTupleAccessor implements TupleReader {
-
-  private final AbstractColumnReader readers[];
-
-  public TupleReaderImpl(TupleSchema schema, AbstractColumnReader readers[]) {
-    super(schema);
-    this.readers = readers;
-  }
-
-  @Override
-  public ColumnReader column(int colIndex) {
-    return readers[colIndex];
-  }
-
-  @Override
-  public ColumnReader column(String colName) {
-    int index = schema.columnIndex(colName);
-    if (index == -1) {
-      return null; }
-    return readers[index];
-  }
-
-  @Override
-  public Object get(int colIndex) {
-    ColumnReader colReader = column(colIndex);
-    if (colReader.isNull()) {
-      return null; }
-    switch (colReader.valueType()) {
-    case BYTES:
-      return colReader.getBytes();
-    case DOUBLE:
-      return colReader.getDouble();
-    case INTEGER:
-      return colReader.getInt();
-    case LONG:
-      return colReader.getLong();
-    case STRING:
-      return colReader.getString();
-    default:
-      throw new IllegalArgumentException("Unsupported type " + colReader.valueType());
-    }
-  }
-
-  @Override
-  public String getAsString(int colIndex) {
-    ColumnReader colReader = column(colIndex);
-    if (colReader.isNull()) {
-      return "null";
-    }
-    switch (colReader.valueType()) {
-    case BYTES:
-      return bytesToString(colReader.getBytes());
-    case DOUBLE:
-      return Double.toString(colReader.getDouble());
-    case INTEGER:
-      return Integer.toString(colReader.getInt());
-    case LONG:
-      return Long.toString(colReader.getLong());
-    case STRING:
-      return "\"" + colReader.getString() + "\"";
-    case DECIMAL:
-      return colReader.getDecimal().toPlainString();
-    case ARRAY:
-      return getArrayAsString(colReader.array());
-    default:
-      throw new IllegalArgumentException("Unsupported type " + colReader.valueType());
-    }
-  }
-
-  private String bytesToString(byte[] value) {
-    StringBuilder buf = new StringBuilder()
-        .append("[");
-    int len = Math.min(value.length, 20);
-    for (int i = 0; i < len;  i++) {
-      if (i > 0) {
-        buf.append(", ");
-      }
-      buf.append((int) value[i]);
-    }
-    if (value.length > len) {
-      buf.append("...");
-    }
-    buf.append("]");
-    return buf.toString();
-  }
-
-  private String getArrayAsString(ArrayReader array) {
-    StringBuilder buf = new StringBuilder();
-    buf.append("[");
-    for (int i = 0; i < array.size(); i++) {
-      if (i > 0) {
-        buf.append( ", " );
-      }
-      switch (array.valueType()) {
-      case BYTES:
-        buf.append(bytesToString(array.getBytes(i)));
-        break;
-      case DOUBLE:
-        buf.append(Double.toString(array.getDouble(i)));
-        break;
-      case INTEGER:
-        buf.append(Integer.toString(array.getInt(i)));
-        break;
-      case LONG:
-        buf.append(Long.toString(array.getLong(i)));
-        break;
-      case STRING:
-        buf.append("\"" + array.getString(i) + "\"");
-        break;
-      case DECIMAL:
-        buf.append(array.getDecimal(i).toPlainString());
-        break;
-      case MAP:
-      case ARRAY:
-        throw new UnsupportedOperationException("Unsupported type " + array.valueType());
-      default:
-        throw new IllegalArgumentException("Unexpected type " + array.valueType());
-      }
-    }
-    buf.append("]");
-    return buf.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleWriterImpl.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleWriterImpl.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleWriterImpl.java
deleted file mode 100644
index 015b099..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleWriterImpl.java
+++ /dev/null
@@ -1,162 +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.drill.exec.vector.accessor.impl;
-
-import java.math.BigDecimal;
-
-import org.apache.drill.exec.vector.accessor.AccessorUtilities;
-import org.apache.drill.exec.vector.accessor.ArrayWriter;
-import org.apache.drill.exec.vector.accessor.ColumnAccessor.ValueType;
-import org.apache.drill.exec.vector.accessor.ColumnWriter;
-import org.apache.drill.exec.vector.accessor.TupleWriter;
-import org.joda.time.Period;
-
-/**
- * Implementation for a writer for a tuple (a row or a map.) Provides access to each
- * column using either a name or a numeric index.
- */
-
-public class TupleWriterImpl extends AbstractTupleAccessor implements TupleWriter {
-
-  private final AbstractColumnWriter writers[];
-
-  public TupleWriterImpl(TupleSchema schema, AbstractColumnWriter writers[]) {
-    super(schema);
-    this.writers = writers;
-  }
-
-  public void start() {
-    for (int i = 0; i < writers.length;  i++) {
-      writers[i].start();
-    }
-  }
-
-  @Override
-  public ColumnWriter column(int colIndex) {
-    return writers[colIndex];
-  }
-
-  @Override
-  public ColumnWriter column(String colName) {
-    int index = schema.columnIndex(colName);
-    if (index == -1) {
-      return null; }
-    return writers[index];
-  }
-
-  @Override
-  public void set(int colIndex, Object value) {
-    ColumnWriter colWriter = column(colIndex);
-    if (value == null) {
-      // Arrays have no null concept, just an empty array.
-      if (colWriter.valueType() != ValueType.ARRAY) {
-        colWriter.setNull();
-      }
-    } else if (value instanceof Integer) {
-      colWriter.setInt((Integer) value);
-    } else if (value instanceof Long) {
-      colWriter.setLong((Long) value);
-    } else if (value instanceof String) {
-      colWriter.setString((String) value);
-    } else if (value instanceof BigDecimal) {
-      colWriter.setDecimal((BigDecimal) value);
-    } else if (value instanceof Period) {
-      colWriter.setPeriod((Period) value);
-    } else if (value instanceof byte[]) {
-      colWriter.setBytes((byte[]) value);
-    } else if (value instanceof Byte) {
-      colWriter.setInt((Byte) value);
-    } else if (value instanceof Short) {
-      colWriter.setInt((Short) value);
-    } else if (value instanceof Double) {
-      colWriter.setDouble((Double) value);
-    } else if (value instanceof Float) {
-      colWriter.setDouble((Float) value);
-    } else if (value.getClass().getName().startsWith("[")) {
-      setArray(colIndex, value);
-    } else {
-      throw new IllegalArgumentException("Unsupported type " +
-                value.getClass().getSimpleName() + " for column " + colIndex);
-    }
-  }
-
-  public void setArray(int colIndex, Object value) {
-    if (value == null) {
-      // Assume null means a 0-element array since Drill does
-      // not support null for the whole array.
-
-      return;
-    }
-    String objClass = value.getClass().getName();
-    if (!objClass.startsWith("[")) {
-      throw new IllegalArgumentException("Argument is not an array");
-    }
-
-    ColumnWriter colWriter = column(colIndex);
-    if (colWriter.valueType() != ValueType.ARRAY) {
-      throw new IllegalArgumentException("Column is not an array");
-    }
-
-    ArrayWriter arrayWriter = colWriter.array();
-
-    // Figure out type
-
-    char second = objClass.charAt( 1 );
-    switch ( second ) {
-    case  'B':
-      AccessorUtilities.setByteArray(arrayWriter, (byte[]) value );
-      break;
-    case  'S':
-      AccessorUtilities.setShortArray(arrayWriter, (short[]) value );
-      break;
-    case  'I':
-      AccessorUtilities.setIntArray(arrayWriter, (int[]) value );
-      break;
-    case  'J':
-      AccessorUtilities.setLongArray(arrayWriter, (long[]) value );
-      break;
-    case  'F':
-      AccessorUtilities.setFloatArray(arrayWriter, (float[]) value );
-      break;
-    case  'D':
-      AccessorUtilities.setDoubleArray(arrayWriter, (double[]) value );
-      break;
-    case  'Z':
-      AccessorUtilities.setBooleanArray(arrayWriter, (boolean[]) value );
-      break;
-    case 'L':
-     int posn = objClass.indexOf(';');
-
-      // If the array is of type Object, then we have no type info.
-
-      String memberClassName = objClass.substring( 2, posn );
-      if (memberClassName.equals(String.class.getName())) {
-        AccessorUtilities.setStringArray(arrayWriter, (String[]) value );
-      } else if (memberClassName.equals(Period.class.getName())) {
-        AccessorUtilities.setPeriodArray(arrayWriter, (Period[]) value );
-      } else if (memberClassName.equals(BigDecimal.class.getName())) {
-        AccessorUtilities.setBigDecimalArray(arrayWriter, (BigDecimal[]) value );
-      } else {
-        throw new IllegalArgumentException( "Unknown Java array type: " + memberClassName );
-      }
-      break;
-    default:
-      throw new IllegalArgumentException( "Unknown Java array type: " + second );
-    }
-  }
-}