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/07/03 19:49:36 UTC

[3/3] drill git commit: DRILL-5518: Test framework enhancements

DRILL-5518: Test framework enhancements

* Create a SubOperatorTest base class to do routine setup and shutdown.
* Additional methods to simplify creating complex schemas with field
widths.
* Define a test workspace with plugin-specific options (as for the CSV
storage plugin)
* When verifying row sets, add methods to verify and release just the
"actual" batch in addition to the existing method for verify and free
both the actual and expected batches.
* Allow reading of row set values as object for generic comparisons.
* "Column builder" within schema builder to simplify building a single
MatrializedField for tests.
* Misc. code cleanup.

closes #851


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

Branch: refs/heads/master
Commit: 63e243378f3be125f1e8bfb52c74b8211c87bfc3
Parents: 92c9304
Author: Paul Rogers <pr...@maprtech.com>
Authored: Tue May 16 15:55:41 2017 -0700
Committer: Paul Rogers <pr...@maprtech.com>
Committed: Mon Jul 3 11:41:20 2017 -0700

----------------------------------------------------------------------
 .../exec/ops/AbstractOperatorExecContext.java   |  17 +--
 .../drill/exec/record/VectorContainer.java      |   2 -
 .../exec/cache/TestBatchSerialization.java      |   2 +-
 .../impl/xsort/managed/SortTestUtilities.java   |   2 +-
 .../physical/impl/xsort/managed/TestSorter.java |  13 +--
 .../drill/exec/record/TestVectorContainer.java  |   2 +-
 .../exec/store/easy/text/compliant/TestCsv.java |  10 +-
 .../org/apache/drill/test/ClientFixture.java    |   6 ++
 .../java/org/apache/drill/test/ClusterTest.java |   1 -
 .../org/apache/drill/test/OperatorFixture.java  |   9 +-
 .../org/apache/drill/test/SubOperatorTest.java  |  36 +++++++
 .../apache/drill/test/rowSet/RowSetBuilder.java |  33 ++++++
 .../drill/test/rowSet/RowSetComparison.java     |  18 +++-
 .../apache/drill/test/rowSet/SchemaBuilder.java |  98 +++++++++++++++--
 .../drill/test/rowSet/test/RowSetTest.java      | 108 +++++++++++++------
 .../exec/vector/accessor/ColumnReader.java      |   1 +
 .../accessor/impl/AbstractColumnReader.java     |  28 +++++
 .../vector/accessor/impl/TupleReaderImpl.java   |  73 ++++++++++---
 18 files changed, 367 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/main/java/org/apache/drill/exec/ops/AbstractOperatorExecContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/AbstractOperatorExecContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/AbstractOperatorExecContext.java
index a517fdf..ebef55c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/AbstractOperatorExecContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/AbstractOperatorExecContext.java
@@ -36,15 +36,16 @@ public class AbstractOperatorExecContext implements OperatorExecContext {
   protected final ExecutionControls executionControls;
   protected final PhysicalOperator popConfig;
   protected final BufferManager manager;
-  protected final OperatorStatReceiver statsWriter;
+  protected OperatorStatReceiver statsWriter;
 
   public AbstractOperatorExecContext(BufferAllocator allocator, PhysicalOperator popConfig,
                                      ExecutionControls executionControls,
                                      OperatorStatReceiver stats) {
     this.allocator = allocator;
     this.popConfig = popConfig;
-    manager = new BufferManagerImpl(allocator);
+    this.manager = new BufferManagerImpl(allocator);
     statsWriter = stats;
+
     this.executionControls = executionControls;
   }
 
@@ -64,10 +65,9 @@ public class AbstractOperatorExecContext implements OperatorExecContext {
   }
 
   @Override
-  public ExecutionControls getExecutionControls() { return executionControls; }
-
-  @Override
-  public OperatorStatReceiver getStatsWriter() { return statsWriter; }
+  public ExecutionControls getExecutionControls() {
+    return executionControls;
+  }
 
   @Override
   public BufferAllocator getAllocator() {
@@ -87,4 +87,9 @@ public class AbstractOperatorExecContext implements OperatorExecContext {
       }
     }
   }
+
+  @Override
+  public OperatorStatReceiver getStatsWriter() {
+    return statsWriter;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
index 54a04bc..99353ea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
@@ -41,7 +41,6 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
 public class VectorContainer implements VectorAccessible {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorContainer.class);
 
   protected final List<VectorWrapper<?>> wrappers = Lists.newArrayList();
   private BatchSchema schema;
@@ -148,7 +147,6 @@ public class VectorContainer implements VectorAccessible {
         return (T) newVector;
       }
     } else {
-
       vector = TypeHelper.getNewVector(field, this.getAllocator(), callBack);
       add(vector);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
index 363c08c..05670c5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
@@ -137,7 +137,7 @@ public class TestBatchSerialization extends DrillTest {
 
     assertTrue(origSize >= result.size());
     new RowSetComparison(expected)
-      .verifyAndClear(result);
+      .verifyAndClearAll(result);
     outFile.delete();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
index 034da2c..1a4d4b2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
@@ -123,7 +123,7 @@ public class SortTestUtilities {
         assertTrue(merger.next());
         RowSet rowSet = new DirectRowSet(fixture.allocator(), dest);
         new RowSetComparison(expectedSet)
-              .verifyAndClear(rowSet);
+              .verifyAndClearAll(rowSet);
       }
       assertFalse(merger.next());
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
index dd371d7..9da8968 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
@@ -85,7 +85,7 @@ public class TestSorter extends DrillTest {
     sorter.sortBatch(rowSet.container(), rowSet.getSv2());
 
     new RowSetComparison(expected)
-        .verifyAndClear(rowSet);
+        .verifyAndClearAll(rowSet);
     sorter.close();
   }
 
@@ -219,16 +219,12 @@ public class TestSorter extends DrillTest {
       DataItem expected[] = Arrays.copyOf(data, data.length);
       doSort(expected);
       RowSet expectedRows = makeDataSet(actual.allocator(), actual.schema().batch(), expected);
-//      System.out.println("Expected:");
-//      expectedRows.print();
-//      System.out.println("Actual:");
-//      actual.print();
       doVerify(expected, expectedRows, actual);
     }
 
     protected void doVerify(DataItem[] expected, RowSet expectedRows, RowSet actual) {
       new RowSetComparison(expectedRows)
-            .verifyAndClear(actual);
+            .verifyAndClearAll(actual);
     }
 
     protected abstract void doSort(DataItem[] expected);
@@ -300,7 +296,7 @@ public class TestSorter extends DrillTest {
             .offset(offset)
             .span(nullCount)
             .withMask(true, false)
-            .verifyAndClear(actual);
+            .verifyAndClearAll(actual);
     }
   }
 
@@ -370,7 +366,7 @@ public class TestSorter extends DrillTest {
         int mo = rand.nextInt(12);
         int yr = rand.nextInt(10);
         Period period = makePeriod(yr, mo, day, hr, min, sec, ms);
-         builder.add(period);
+        builder.add(period);
       }
       return builder.build();
     }
@@ -385,7 +381,6 @@ public class TestSorter extends DrillTest {
       long prevMs = 0;
       while (reader.next()) {
         Period period = reader.column(0).getPeriod().normalizedStandard();
-//        System.out.println(period);
         int years = period.getYears();
         assertTrue(prevYears <= years);
         if (prevYears != years) {

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java
index d7a59bf..930e2a8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java
@@ -108,7 +108,7 @@ public class TestVectorContainer extends DrillTest {
     // Merge containers via row set facade
 
     RowSet mergedRs = left.merge(right);
-    comparison.verifyAndClear(mergedRs);
+    comparison.verifyAndClearAll(mergedRs);
 
     // Add a selection vector. Merging is forbidden, in the present code,
     // for batches that have a selection vector.

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
index c18adc9..b7bc9fd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
@@ -98,13 +98,11 @@ public class TestCsv extends ClusterTest {
         .add("b", MinorType.VARCHAR)
         .add("c", MinorType.VARCHAR)
         .build();
-    assertEquals(expectedSchema, actual.batchSchema());
-
     RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
         .add("10", "foo", "bar")
         .build();
     new RowSetComparison(expected)
-      .verifyAndClear(actual);
+      .verifyAndClearAll(actual);
   }
 
   String invalidHeaders[] = {
@@ -126,13 +124,11 @@ public class TestCsv extends ClusterTest {
         .add("c_2", MinorType.VARCHAR)
         .add("c_2_2", MinorType.VARCHAR)
         .build();
-    assertEquals(expectedSchema, actual.batchSchema());
-
     RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
         .add("10", "foo", "bar", "fourth", "fifth", "sixth")
         .build();
     new RowSetComparison(expected)
-      .verifyAndClear(actual);
+      .verifyAndClearAll(actual);
   }
 
   // Test fix for DRILL-5590
@@ -154,7 +150,7 @@ public class TestCsv extends ClusterTest {
         .add("10", "foo", "bar")
         .build();
     new RowSetComparison(expected)
-      .verifyAndClear(actual);
+      .verifyAndClearAll(actual);
   }
 
   private String makeStatement(String fileName) {

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
index a63a287..5a06ec2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
@@ -27,12 +27,14 @@ import org.apache.drill.TestBuilder;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.testing.Controls;
 import org.apache.drill.exec.testing.ControlsInjectionUtil;
 import org.apache.drill.test.ClusterFixture.FixtureTestServices;
 import org.apache.drill.test.QueryBuilder.QuerySummary;
+import org.apache.drill.test.rowSet.RowSetBuilder;
 
 /**
  * Represents a Drill client. Provides many useful test-specific operations such
@@ -225,4 +227,8 @@ public class ClientFixture implements AutoCloseable {
     ControlsInjectionUtil.validateControlsString(controls);
     alterSession(ExecConstants.DRILLBIT_CONTROL_INJECTIONS, controls);
   }
+
+  public RowSetBuilder rowSetBuilder(BatchSchema schema) {
+    return new RowSetBuilder(allocator(), schema);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/test/ClusterTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterTest.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterTest.java
index 62beedd..e204fde 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterTest.java
@@ -118,5 +118,4 @@ public class ClusterTest extends DrillTest {
   public QueryBuilder queryBuilder( ) {
     return client.queryBuilder();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
index 2c72c3c..976812c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
@@ -32,10 +32,12 @@ import org.apache.drill.exec.expr.ClassGenerator;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.RootAllocatorFactory;
+import org.apache.drill.exec.ops.AbstractOperatorExecContext;
 import org.apache.drill.exec.ops.FragmentExecContext;
 import org.apache.drill.exec.ops.MetricDef;
 import org.apache.drill.exec.ops.OperExecContext;
 import org.apache.drill.exec.ops.OperExecContextImpl;
+import org.apache.drill.exec.ops.OperatorExecContext;
 import org.apache.drill.exec.ops.OperatorStatReceiver;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.record.BatchSchema;
@@ -50,7 +52,6 @@ import org.apache.drill.test.rowSet.HyperRowSetImpl;
 import org.apache.drill.test.rowSet.IndirectRowSet;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
-import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 import org.apache.drill.test.rowSet.RowSetBuilder;
 
 /**
@@ -222,7 +223,7 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
   }
 
   /**
-   * Implements a write-only version of the stats collector for use by opeators,
+   * Implements a write-only version of the stats collector for use by operators,
    * then provides simplified test-time accessors to get the stats values when
    * validating code in tests.
    */
@@ -328,4 +329,8 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
       throw new IllegalStateException( "Unexpected selection mode" );
     }
   }
+
+  public OperatorExecContext operatorContext(PhysicalOperator config) {
+    return new AbstractOperatorExecContext(allocator(), config, context.getExecutionControls(), stats);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/test/SubOperatorTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/SubOperatorTest.java b/exec/java-exec/src/test/java/org/apache/drill/test/SubOperatorTest.java
new file mode 100644
index 0000000..6bc2afc
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/SubOperatorTest.java
@@ -0,0 +1,36 @@
+/*
+ * 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.test;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class SubOperatorTest extends DrillTest {
+
+  protected static OperatorFixture fixture;
+
+  @BeforeClass
+  public static void classSetup() throws Exception {
+    fixture = OperatorFixture.standardFixture();
+  }
+
+  @AfterClass
+  public static void classTeardown() throws Exception {
+    fixture.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
index 74e9356..80e8ae4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
@@ -56,6 +56,8 @@ public final class RowSetBuilder {
    * <tt>add(10, new int[] {100, 200});</tt><br>
    * @param values column values in column index order
    * @return this builder
+   * @see {@link #addSingleCol(Object)} to create a row of a single column when
+   * the value to <tt>add()</tt> is ambiguous
    */
 
   public RowSetBuilder add(Object...values) {
@@ -64,6 +66,37 @@ public final class RowSetBuilder {
   }
 
   /**
+   * The {@link #add(Object...)} method uses Java variable-length arguments to
+   * pass a row of values. But, when the row consists of a single array, Java
+   * gets confused: is that an array for variable-arguments or is it the value
+   * of the first argument? This method clearly states that the single value
+   * (including an array) is meant to be the value of the first (and only)
+   * column.
+   * <p>
+   * Examples:<code><pre>
+   *     RowSetBuilder twoColsBuilder = ...
+   *     // Fine, second item is an array of strings for a repeated Varchar
+   *     // column.
+   *     twoColsBuilder.add("First", new String[] {"a", "b", "c"});
+   *     ...
+   *     RowSetBuilder oneColBuilder = ...
+   *     // Ambiguous: is this a varargs array of three items?
+   *     // That is how Java will perceive it.
+   *     oneColBuilder.add(new String[] {"a", "b", "c"});
+   *     // Unambiguous: this is a single column value for the
+   *     // repeated Varchar column.
+   *     oneColBuilder.addSingleCol(new String[] {"a", "b", "c"});
+   * </pre></code>
+   * @param value value of the first column, which may be an array for a
+   * repeated column
+   * @return this builder
+   */
+
+  public RowSetBuilder addSingleCol(Object value) {
+    return add(new Object[] { value });
+  }
+
+  /**
    * Build the row set with a selection vector 2. The SV2 is
    * initialized to have a 1:1 index to the rows: SV2 0 points
    * to row 1, SV2 position 1 points to row 1 and so on.

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java
index 3ba7471..ea50074 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java
@@ -138,7 +138,7 @@ public class RowSetComparison {
 
   /**
    * Convenience method to verify the actual results, then free memory
-   * for both the expected and actual result sets.
+   * for the actual result sets.
    * @param actual the actual results to verify
    */
 
@@ -146,6 +146,20 @@ public class RowSetComparison {
     try {
       verify(actual);
     } finally {
+      actual.clear();
+    }
+  }
+
+  /**
+   * Convenience method to verify the actual results, then free memory
+   * for both the expected and actual result sets.
+   * @param actual the actual results to verify
+   */
+
+  public void verifyAndClearAll(RowSet actual) {
+    try {
+      verify(actual);
+    } finally {
       expected.clear();
       actual.clear();
     }
@@ -158,7 +172,7 @@ public class RowSetComparison {
       }
       ColumnReader ec = er.column(i);
       ColumnReader ac = ar.column(i);
-      String label = er.index() + ":" + i;
+      String label = (er.index() + 1) + ":" + i;
       assertEquals(label, ec.valueType(), ac.valueType());
       if (ec.isNull()) {
         assertTrue(label + " - column not null", ac.isNull());

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
index b946ab9..39b0128 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
@@ -53,6 +53,47 @@ import org.apache.drill.exec.record.MaterializedField;
 public class SchemaBuilder {
 
   /**
+   * Build a column schema (AKA "materialized field") based on name and a
+   * variety of schema options. Every column needs a name and (minor) type,
+   * some may need a mode other than required, may need a width, may
+   * need scale and precision, and so on.
+   */
+
+  // TODO: Add map methods
+
+  public static class ColumnBuilder {
+    private final String name;
+    private final MajorType.Builder typeBuilder;
+
+    public ColumnBuilder(String name, MinorType type) {
+      this.name = name;
+      typeBuilder = MajorType.newBuilder()
+          .setMinorType(type)
+          .setMode(DataMode.REQUIRED);
+    }
+
+    public ColumnBuilder setMode(DataMode mode) {
+      typeBuilder.setMode(mode);
+      return this;
+    }
+
+    public ColumnBuilder setWidth(int width) {
+      typeBuilder.setPrecision(width);
+      return this;
+    }
+
+    public ColumnBuilder setScale(int scale, int precision) {
+      typeBuilder.setScale(scale);
+      typeBuilder.setPrecision(precision);
+      return this;
+    }
+
+    public MaterializedField build() {
+      return MaterializedField.create(name, typeBuilder.build());
+    }
+  }
+
+  /**
    * Internal structure for building a map. A map is just a schema,
    * but one that is part of a parent column.
    */
@@ -73,11 +114,7 @@ public class SchemaBuilder {
 
     @Override
     public SchemaBuilder buildMap() {
-      MaterializedField col = MaterializedField.create(memberName,
-          MajorType.newBuilder()
-            .setMinorType(MinorType.MAP)
-            .setMode(DataMode.REQUIRED)
-            .build());
+      MaterializedField col = columnSchema(memberName, MinorType.MAP, DataMode.REQUIRED);
       for (MaterializedField childCol : columns) {
         col.addChild(childCol);
       }
@@ -96,27 +133,68 @@ public class SchemaBuilder {
 
   public SchemaBuilder() { }
 
+  /**
+   * Create a new schema starting with the base schema. Allows appending
+   * additional columns to an additional schema.
+   */
+
+  public SchemaBuilder(BatchSchema baseSchema) {
+    for (MaterializedField field : baseSchema) {
+      add(field);
+    }
+  }
+
   public SchemaBuilder add(String pathName, MajorType type) {
-    MaterializedField col = MaterializedField.create(pathName, type);
+    return add(MaterializedField.create(pathName, type));
+  }
+
+  public SchemaBuilder add(MaterializedField col) {
     columns.add(col);
     return this;
   }
 
+  /**
+   * Create a column schema using the "basic three" properties of name, type and
+   * cardinality (AKA "data mode.") Use the {@link ColumnBuilder} for to set
+   * other schema attributes.
+   */
+
+  public static MaterializedField columnSchema(String pathName, MinorType type, DataMode mode) {
+    return MaterializedField.create(pathName,
+        MajorType.newBuilder()
+          .setMinorType(type)
+          .setMode(mode)
+          .build());
+  }
+
   public SchemaBuilder add(String pathName, MinorType type, DataMode mode) {
-    return add(pathName, MajorType.newBuilder()
-        .setMinorType(type)
-        .setMode(mode)
-        .build());
+    return add(columnSchema(pathName, type, mode));
   }
 
   public SchemaBuilder add(String pathName, MinorType type) {
     return add(pathName, type, DataMode.REQUIRED);
   }
 
+  public SchemaBuilder add(String pathName, MinorType type, int width) {
+    MaterializedField field = new SchemaBuilder.ColumnBuilder(pathName, type)
+        .setMode(DataMode.REQUIRED)
+        .setWidth(width)
+        .build();
+    return add(field);
+  }
+
   public SchemaBuilder addNullable(String pathName, MinorType type) {
     return add(pathName, type, DataMode.OPTIONAL);
   }
 
+  public SchemaBuilder addNullable(String pathName, MinorType type, int width) {
+    MaterializedField field = new SchemaBuilder.ColumnBuilder(pathName, type)
+        .setMode(DataMode.OPTIONAL)
+        .setWidth(width)
+        .build();
+    return add(field);
+  }
+
   public SchemaBuilder addArray(String pathName, MinorType type) {
     return add(pathName, type, DataMode.REPEATED);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
index 8d9179b..03417ff 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
@@ -30,7 +30,7 @@ import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.vector.accessor.ArrayReader;
 import org.apache.drill.exec.vector.accessor.ArrayWriter;
 import org.apache.drill.exec.vector.accessor.TupleAccessor.TupleSchema;
-import org.apache.drill.test.OperatorFixture;
+import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
 import org.apache.drill.test.rowSet.RowSet.RowSetReader;
 import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
@@ -40,25 +40,11 @@ import org.apache.drill.test.rowSet.RowSetSchema;
 import org.apache.drill.test.rowSet.RowSetSchema.FlattenedSchema;
 import org.apache.drill.test.rowSet.RowSetSchema.PhysicalSchema;
 import org.apache.drill.test.rowSet.SchemaBuilder;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.base.Splitter;
 
-public class RowSetTest {
-
-  private static OperatorFixture fixture;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    fixture = OperatorFixture.standardFixture();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    fixture.close();
-  }
+public class RowSetTest extends SubOperatorTest {
 
   /**
    * Test a simple physical schema with no maps.
@@ -105,6 +91,17 @@ public class RowSetTest {
     assertEquals("b", physical.column(2).field().getName());
   }
 
+  /**
+   * Validate that the actual column metadata is as expected by
+   * cross-checking: validate that the column at the index and
+   * the column at the column name are both correct.
+   *
+   * @param schema the schema for the row set
+   * @param index column index
+   * @param fullName expected column name
+   * @param type expected type
+   */
+
   public void crossCheck(TupleSchema schema, int index, String fullName, MinorType type) {
     String name = null;
     for (String part : Splitter.on(".").split(fullName)) {
@@ -116,6 +113,10 @@ public class RowSetTest {
     assertEquals(type, schema.column(index).getType().getMinorType());
   }
 
+  /**
+   * Verify that a nested map schema works as expected.
+   */
+
   @Test
   public void testMapSchema() {
     BatchSchema batchSchema = new SchemaBuilder()
@@ -185,17 +186,13 @@ public class RowSetTest {
     assertEquals("a.e.f", eSchema.column(0).fullName());
   }
 
-  @Test
-  public void testScalarReaderWriter() {
-    testTinyIntRW();
-    testSmallIntRW();
-    testIntRW();
-    testLongRW();
-    testFloatRW();
-    testDoubleRW();
-  }
+  /**
+   * Verify that simple scalar (non-repeated) column readers
+   * and writers work as expected. This is for tiny ints.
+   */
 
-  private void testTinyIntRW() {
+  @Test
+  public void testTinyIntRW() {
     BatchSchema batchSchema = new SchemaBuilder()
         .add("col", MinorType.TINYINT)
         .build();
@@ -204,18 +201,21 @@ public class RowSetTest {
         .add(Byte.MAX_VALUE)
         .add(Byte.MIN_VALUE)
         .build();
+    assertEquals(3, rs.rowCount());
     RowSetReader reader = rs.reader();
     assertTrue(reader.next());
     assertEquals(0, reader.column(0).getInt());
     assertTrue(reader.next());
     assertEquals(Byte.MAX_VALUE, reader.column(0).getInt());
+    assertEquals((int) Byte.MAX_VALUE, reader.column(0).getObject());
     assertTrue(reader.next());
     assertEquals(Byte.MIN_VALUE, reader.column(0).getInt());
     assertFalse(reader.next());
     rs.clear();
   }
 
-  private void testSmallIntRW() {
+  @Test
+  public void testSmallIntRW() {
     BatchSchema batchSchema = new SchemaBuilder()
         .add("col", MinorType.SMALLINT)
         .build();
@@ -229,13 +229,15 @@ public class RowSetTest {
     assertEquals(0, reader.column(0).getInt());
     assertTrue(reader.next());
     assertEquals(Short.MAX_VALUE, reader.column(0).getInt());
+    assertEquals((int) Short.MAX_VALUE, reader.column(0).getObject());
     assertTrue(reader.next());
     assertEquals(Short.MIN_VALUE, reader.column(0).getInt());
     assertFalse(reader.next());
     rs.clear();
   }
 
-  private void testIntRW() {
+  @Test
+  public void testIntRW() {
     BatchSchema batchSchema = new SchemaBuilder()
         .add("col", MinorType.INT)
         .build();
@@ -249,13 +251,15 @@ public class RowSetTest {
     assertEquals(0, reader.column(0).getInt());
     assertTrue(reader.next());
     assertEquals(Integer.MAX_VALUE, reader.column(0).getInt());
+    assertEquals(Integer.MAX_VALUE, reader.column(0).getObject());
     assertTrue(reader.next());
     assertEquals(Integer.MIN_VALUE, reader.column(0).getInt());
     assertFalse(reader.next());
     rs.clear();
   }
 
-  private void testLongRW() {
+  @Test
+  public void testLongRW() {
     BatchSchema batchSchema = new SchemaBuilder()
         .add("col", MinorType.BIGINT)
         .build();
@@ -269,13 +273,15 @@ public class RowSetTest {
     assertEquals(0, reader.column(0).getLong());
     assertTrue(reader.next());
     assertEquals(Long.MAX_VALUE, reader.column(0).getLong());
+    assertEquals(Long.MAX_VALUE, reader.column(0).getObject());
     assertTrue(reader.next());
     assertEquals(Long.MIN_VALUE, reader.column(0).getLong());
     assertFalse(reader.next());
     rs.clear();
   }
 
-  private void testFloatRW() {
+  @Test
+  public void testFloatRW() {
     BatchSchema batchSchema = new SchemaBuilder()
         .add("col", MinorType.FLOAT4)
         .build();
@@ -288,14 +294,16 @@ public class RowSetTest {
     assertTrue(reader.next());
     assertEquals(0, reader.column(0).getDouble(), 0.000001);
     assertTrue(reader.next());
-    assertEquals(Float.MAX_VALUE, reader.column(0).getDouble(), 0.000001);
+    assertEquals((double) Float.MAX_VALUE, reader.column(0).getDouble(), 0.000001);
+    assertEquals((double) Float.MAX_VALUE, (double) reader.column(0).getObject(), 0.000001);
     assertTrue(reader.next());
-    assertEquals(Float.MIN_VALUE, reader.column(0).getDouble(), 0.000001);
+    assertEquals((double) Float.MIN_VALUE, reader.column(0).getDouble(), 0.000001);
     assertFalse(reader.next());
     rs.clear();
   }
 
-  private void testDoubleRW() {
+  @Test
+  public void testDoubleRW() {
     BatchSchema batchSchema = new SchemaBuilder()
         .add("col", MinorType.FLOAT8)
         .build();
@@ -309,6 +317,7 @@ public class RowSetTest {
     assertEquals(0, reader.column(0).getDouble(), 0.000001);
     assertTrue(reader.next());
     assertEquals(Double.MAX_VALUE, reader.column(0).getDouble(), 0.000001);
+    assertEquals(Double.MAX_VALUE, (double) reader.column(0).getObject(), 0.000001);
     assertTrue(reader.next());
     assertEquals(Double.MIN_VALUE, reader.column(0).getDouble(), 0.000001);
     assertFalse(reader.next());
@@ -316,6 +325,30 @@ public class RowSetTest {
   }
 
   @Test
+  public void testStringRW() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", MinorType.VARCHAR)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .add("")
+        .add("abcd")
+        .build();
+    RowSetReader reader = rs.reader();
+    assertTrue(reader.next());
+    assertEquals("", reader.column(0).getString());
+    assertTrue(reader.next());
+    assertEquals("abcd", reader.column(0).getString());
+    assertEquals("abcd", reader.column(0).getObject());
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  /**
+   * Test writing to and reading from a row set with nested maps.
+   * Map fields are flattened into a logical schema.
+   */
+
+  @Test
   public void testMap() {
     BatchSchema batchSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
@@ -343,8 +376,13 @@ public class RowSetTest {
     rs.clear();
   }
 
+  /**
+   * Test an array of ints (as an example fixed-width type)
+   * at the top level of a schema.
+   */
+
   @Test
-  public void TestTopScalarArray() {
+  public void TestTopFixedWidthArray() {
     BatchSchema batchSchema = new SchemaBuilder()
         .add("c", MinorType.INT)
         .addArray("a", MinorType.INT)
@@ -394,7 +432,7 @@ public class RowSetTest {
       .build();
 
     new RowSetComparison(rs1)
-      .verifyAndClear(rs2);
+      .verifyAndClearAll(rs2);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReader.java
index 860a866..4932567 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReader.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReader.java
@@ -58,6 +58,7 @@ public interface ColumnReader extends ColumnAccessor {
   byte[] getBytes();
   BigDecimal getDecimal();
   Period getPeriod();
+  Object getObject();
   TupleReader map();
   ArrayReader array();
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/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
index 1ef2243..b88b08b 100644
--- 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
@@ -47,6 +47,34 @@ public abstract class AbstractColumnReader extends AbstractColumnAccessor implem
   }
 
   @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;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/63e24337/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
index 041023b..97a6e3c 100644
--- 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
@@ -17,6 +17,7 @@
  */
 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;
 
@@ -76,21 +77,7 @@ public class TupleReaderImpl extends AbstractTupleAccessor implements TupleReade
     }
     switch (colReader.valueType()) {
     case BYTES:
-      StringBuilder buf = new StringBuilder()
-          .append("[");
-      byte value[] = colReader.getBytes();
-      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();
+      return bytesToString(colReader.getBytes());
     case DOUBLE:
       return Double.toString(colReader.getDouble());
     case INTEGER:
@@ -101,8 +88,64 @@ public class TupleReaderImpl extends AbstractTupleAccessor implements TupleReade
       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();
+  }
 }