You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by hg...@apache.org on 2015/05/12 02:42:30 UTC

[1/2] drill git commit: DRILL-3027: Add convenience methods to test builder for creating nested baseline values

Repository: drill
Updated Branches:
  refs/heads/master a7359f36b -> 27b4aae20


DRILL-3027: Add convenience methods to test builder for creating nested baseline values


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

Branch: refs/heads/master
Commit: c8769edc1edbc743b8859148dba92259149df6f8
Parents: a7359f3
Author: Hanifi Gunes <hg...@maprtech.com>
Authored: Mon May 11 15:09:28 2015 -0700
Committer: Hanifi Gunes <hg...@maprtech.com>
Committed: Mon May 11 16:11:03 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/drill/BaseTestQuery.java    | 34 --------------
 .../test/java/org/apache/drill/TestBuilder.java | 49 ++++++++++++++++++--
 .../org/apache/drill/TestFrameworkTest.java     | 21 ++++-----
 .../vector/complex/writer/TestJsonReader.java   | 14 +++---
 4 files changed, 61 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/c8769edc/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
index db1ed34..83134b3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
@@ -246,40 +246,6 @@ public class BaseTestQuery extends ExecTest {
     }
   }
 
-  private Object wrapStringInHadoopTextObject(Object o) {
-    if (o instanceof String) {
-      o = new Text((String)o);
-    }
-    return o;
-  }
-
-  // convenience method for making a list
-  protected JsonStringArrayList list(Object... values) {
-    JsonStringArrayList ret = new JsonStringArrayList<>();
-    for (int i = 0; i < values.length; i++) {
-      values[i] = wrapStringInHadoopTextObject(values[i]);
-    }
-    ret.addAll(Arrays.asList(values));
-    return ret;
-  }
-
-  protected JsonStringHashMap map(Object... keysAndValues) {
-    JsonStringHashMap ret = new JsonStringHashMap();
-    final String errorMsg = "Must provide a list of keys and values to construct a map, expects" +
-          "an even number or arguments, alternating strings for key names and objects for values.";
-    if (keysAndValues.length % 2 != 0) {
-      throw new RuntimeException(errorMsg);
-    }
-    for (int i = 0; i < keysAndValues.length - 1; i += 2) {
-      if ( ! (keysAndValues[i] instanceof String) )  {
-        throw new RuntimeException(errorMsg);
-      }
-      keysAndValues[i + 1] = wrapStringInHadoopTextObject(keysAndValues[i + 1]);
-      ret.put(keysAndValues[i], keysAndValues[i + 1]);
-    }
-    return ret;
-  }
-
   @AfterClass
   public static void resetDrillbitCount() {
     // some test classes assume this value to be 1 and will fail if run along other tests that increase it

http://git-wip-us.apache.org/repos/asf/drill/blob/c8769edc/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java
index 13f958c..666ecda 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java
@@ -19,6 +19,7 @@ package org.apache.drill;
 
 import com.google.common.base.Joiner;
 
+import com.google.common.base.Preconditions;
 import org.antlr.runtime.ANTLRStringStream;
 import org.antlr.runtime.CommonTokenStream;
 import org.antlr.runtime.RecognitionException;
@@ -29,6 +30,9 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.util.JsonStringArrayList;
+import org.apache.drill.exec.util.JsonStringHashMap;
+import org.apache.hadoop.io.Text;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -366,8 +370,8 @@ public class TestBuilder {
     private TypeProtos.MajorType[] baselineTypes;
 
     CSVTestBuilder(String baselineFile, BufferAllocator allocator, String query, UserBitShared.QueryType queryType, Boolean ordered,
-                     boolean approximateEquality, Map<SchemaPath, TypeProtos.MajorType> baselineTypeMap,
-                     String baselineOptionSettingQueries, String testOptionSettingQueries, boolean highPerformanceComparison) {
+                   boolean approximateEquality, Map<SchemaPath, TypeProtos.MajorType> baselineTypeMap,
+                   String baselineOptionSettingQueries, String testOptionSettingQueries, boolean highPerformanceComparison) {
       super(allocator, query, queryType, ordered, approximateEquality, baselineTypeMap, baselineOptionSettingQueries, testOptionSettingQueries,
           highPerformanceComparison);
       this.baselineFilePath = baselineFile;
@@ -434,7 +438,7 @@ public class TestBuilder {
         // set default cast size for varchar, the cast function will take the lesser of this passed value and the
         // length of the incoming data when choosing the length for the outgoing data
         if (majorType.getMinorType() == TypeProtos.MinorType.VARCHAR ||
-          majorType.getMinorType() == TypeProtos.MinorType.VARBINARY) {
+            majorType.getMinorType() == TypeProtos.MinorType.VARBINARY) {
           precision = "(65000)";
         }
         aliasedExpectedColumns[i] = "cast(" + aliasedExpectedColumns[i] + " as " +
@@ -456,8 +460,8 @@ public class TestBuilder {
     private String baselineFilePath;
 
     JSONTestBuilder(String baselineFile, BufferAllocator allocator, String query, UserBitShared.QueryType queryType, Boolean ordered,
-                   boolean approximateEquality, Map<SchemaPath, TypeProtos.MajorType> baselineTypeMap,
-                   String baselineOptionSettingQueries, String testOptionSettingQueries, boolean highPerformanceComparison) {
+                    boolean approximateEquality, Map<SchemaPath, TypeProtos.MajorType> baselineTypeMap,
+                    String baselineOptionSettingQueries, String testOptionSettingQueries, boolean highPerformanceComparison) {
       super(allocator, query, queryType, ordered, approximateEquality, baselineTypeMap, baselineOptionSettingQueries, testOptionSettingQueries,
           highPerformanceComparison);
       this.baselineFilePath = baselineFile;
@@ -505,4 +509,39 @@ public class TestBuilder {
     }
 
   }
+
+  /**
+   * Convenience method to create a {@link JsonStringArrayList list} from the given values.
+   */
+  public static JsonStringArrayList listOf(Object... values) {
+    final JsonStringArrayList list = new JsonStringArrayList<>();
+    for (Object value:values) {
+      if (value instanceof CharSequence) {
+        list.add(new Text(value.toString()));
+      } else {
+        list.add(value);
+      }
+    }
+    return list;
+  }
+
+  /**
+   * Convenience method to create a {@link JsonStringHashMap<String, Object> map} instance with the given key value sequence.
+   *
+   * Key value sequence consists of key - value pairs such that a key precedes its value. For instance:
+   *
+   * mapOf("name", "Adam", "age", 41) corresponds to {"name": "Adam", "age": 41} in JSON.
+   */
+  public static JsonStringHashMap<String, Object> mapOf(Object... keyValueSequence) {
+    Preconditions.checkArgument(keyValueSequence.length%2==0, "Length of key value sequence must be even");
+    final JsonStringHashMap<String, Object> map = new JsonStringHashMap<>();
+    for (int i=0; i<keyValueSequence.length; i+=2) {
+      Object value = keyValueSequence[i+1];
+      if (value instanceof CharSequence) {
+        value = new Text(value.toString());
+      }
+      map.put(String.class.cast(keyValueSequence[i]), value);
+    }
+    return map;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/c8769edc/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java b/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java
index c6e81a9..deeb7cb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java
@@ -17,15 +17,12 @@
  ******************************************************************************/
 package org.apache.drill;
 
+import static org.apache.drill.TestBuilder.listOf;
+import static org.apache.drill.TestBuilder.mapOf;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.types.MinorType;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
-import org.apache.drill.exec.util.JsonStringArrayList;
-import org.apache.drill.exec.util.JsonStringHashMap;
-import org.apache.hadoop.io.Text;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.math.BigDecimal;
@@ -146,14 +143,14 @@ public class TestFrameworkTest extends BaseTestQuery{
         .baselineColumns("integer", "float", "x", "z", "l", "rl")
         .baselineValues(2010l,
                         17.4,
-                        map("y", "kevin",
+                        mapOf("y", "kevin",
                             "z", "paul"),
-                        list(map("orange", "yellow",
-                                 "pink", "red"),
-                             map("pink", "purple")),
-                        list(4l, 2l),
-                        list(list(2l, 1l),
-                             list(4l, 6l)))
+                        listOf(mapOf("orange", "yellow",
+                                "pink", "red"),
+                            mapOf("pink", "purple")),
+                        listOf(4l, 2l),
+                        listOf(listOf(2l, 1l),
+                            listOf(4l, 6l)))
         .build().run();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/c8769edc/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
index dfa89ca..912a5f0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.vector.complex.writer;
 
+import static org.apache.drill.TestBuilder.listOf;
+import static org.apache.drill.TestBuilder.mapOf;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -67,24 +69,24 @@ public class TestJsonReader extends BaseTestQuery {
         .sqlQuery("select flatten(config) as flat from cp.`/store/json/null_list.json`")
         .ordered()
         .baselineColumns("flat")
-        .baselineValues(list())
-        .baselineValues(list(testVal))
+        .baselineValues(listOf())
+        .baselineValues(listOf(testVal))
         .go();
 
     testBuilder()
         .sqlQuery("select flatten(config) as flat from cp.`/store/json/null_list_v2.json`")
         .ordered()
         .baselineColumns("flat")
-        .baselineValues(map("repeated_varchar", list()))
-        .baselineValues(map("repeated_varchar", list(testVal)))
+        .baselineValues(mapOf("repeated_varchar", listOf()))
+        .baselineValues(mapOf("repeated_varchar", listOf(testVal)))
         .go();
 
     testBuilder()
         .sqlQuery("select flatten(config) as flat from cp.`/store/json/null_list_v3.json`")
         .ordered()
         .baselineColumns("flat")
-        .baselineValues(map("repeated_map", list(map("repeated_varchar", list()))))
-        .baselineValues(map("repeated_map", list(map("repeated_varchar", list(testVal)))))
+        .baselineValues(mapOf("repeated_map", listOf(mapOf("repeated_varchar", listOf()))))
+        .baselineValues(mapOf("repeated_map", listOf(mapOf("repeated_varchar", listOf(testVal)))))
         .go();
   }
 


[2/2] drill git commit: DRILL-2776: add extensive tests for empty population coverage

Posted by hg...@apache.org.
DRILL-2776: add extensive tests for empty population coverage


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

Branch: refs/heads/master
Commit: 27b4aae2008a8125411eb0ec23a07cad8cea096e
Parents: c8769ed
Author: Hanifi Gunes <hg...@maprtech.com>
Authored: Mon May 11 16:13:52 2015 -0700
Committer: Hanifi Gunes <hg...@maprtech.com>
Committed: Mon May 11 17:20:05 2015 -0700

----------------------------------------------------------------------
 .../vector/complex/TestEmptyPopulation.java     | 314 +++++++++++++++++++
 .../exec/vector/complex/TestEmptyPopulator.java |  86 -----
 .../vector/complex/map-empty-between.json       |   3 +
 .../vector/complex/map-empty-first.json         |   3 +
 .../vector/complex/map-empty-last.json          |   3 +
 .../multi-repeated-list-empty-between.json      |   3 +
 .../multi-repeated-list-empty-first.json        |   3 +
 .../complex/multi-repeated-list-empty-last.json |   3 +
 .../multi-repeated-list-multi-empty.json        |   3 +
 .../complex/repeated-list-empty-between.json    |   3 +
 .../complex/repeated-list-empty-first.json      |   3 +
 .../complex/repeated-list-empty-last.json       |   3 +
 .../complex/repeated-map-empty-between.json     |   3 +
 .../complex/repeated-map-empty-first.json       |   3 +
 .../vector/complex/repeated-map-empty-last.json |   3 +
 .../complex/repeated-scalar-empty-between.json  |   3 +
 .../complex/repeated-scalar-empty-first.json    |   3 +
 .../complex/repeated-scalar-empty-last.json     |   3 +
 18 files changed, 362 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulation.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulation.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulation.java
new file mode 100644
index 0000000..06a73e2
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulation.java
@@ -0,0 +1,314 @@
+/**
+ * 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.complex;
+
+import static org.apache.drill.TestBuilder.listOf;
+import static org.apache.drill.TestBuilder.mapOf;
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestEmptyPopulation extends BaseTestQuery {
+
+  private UInt4Vector offsets;
+  private UInt4Vector.Accessor accessor;
+  private UInt4Vector.Mutator mutator;
+  private EmptyValuePopulator populator;
+  private BufferAllocator allocator = new TopLevelAllocator();
+
+
+  @Before
+  public void initialize() {
+    offsets = new UInt4Vector(null, allocator);
+    offsets.allocateNewSafe();
+    accessor = offsets.getAccessor();
+    mutator = offsets.getMutator();
+    mutator.set(0, 0);
+    mutator.setValueCount(1);
+    Assert.assertTrue("offsets must have one value", accessor.getValueCount() == 1);
+    populator = new EmptyValuePopulator(offsets);
+  }
+
+  @Test(expected = IndexOutOfBoundsException.class)
+  public void testNegativeValuesThrowException() {
+    populator.populate(-1);
+  }
+
+  @Test
+  public void testZeroHasNoEffect() {
+    populator.populate(0);
+    Assert.assertTrue("offset must have one value", accessor.getValueCount() == 1);
+  }
+
+  @Test
+  public void testEmptyPopulationWorks() {
+    populator.populate(1);
+    Assert.assertEquals("offset must have valid size", 2, accessor.getValueCount());
+    Assert.assertEquals("value must match", 0, accessor.get(1));
+
+    mutator.set(1, 10);
+    populator.populate(2);
+    Assert.assertEquals("offset must have valid size", 3, accessor.getValueCount());
+    Assert.assertEquals("value must match", 10, accessor.get(1));
+
+    mutator.set(2, 20);
+    populator.populate(5);
+    Assert.assertEquals("offset must have valid size", 6, accessor.getValueCount());
+    for (int i=2; i<=5;i++) {
+      Assert.assertEquals(String.format("value at index[%s] must match", i), 20, accessor.get(i));
+    }
+
+    populator.populate(0);
+    Assert.assertEquals("offset must have valid size", 1, accessor.getValueCount());
+    Assert.assertEquals("value must match", 0, accessor.get(0));
+  }
+
+  @Test
+  public void testRepeatedScalarEmptyFirst() throws Exception {
+    final String query = "select * from cp.`vector/complex/repeated-scalar-empty-first.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf())
+        .baselineValues(listOf(1L))
+        .baselineValues(listOf(2L))
+        .go();
+  }
+
+  @Test
+  public void testRepeatedScalarEmptyLast() throws Exception {
+    final String query = "select * from cp.`vector/complex/repeated-scalar-empty-last.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf(1L))
+        .baselineValues(listOf(2L))
+        .baselineValues(listOf())
+        .go();
+  }
+
+  @Test
+  public void testRepeatedScalarEmptyInBetween() throws Exception {
+    final String query = "select * from cp.`vector/complex/repeated-scalar-empty-between.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf(1L))
+        .baselineValues(listOf())
+        .baselineValues(listOf(2L))
+        .go();
+  }
+
+  @Test
+  public void testRepeatedListEmptyFirst() throws Exception {
+    final String query = "select * from cp.`vector/complex/repeated-list-empty-first.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf())
+        .baselineValues(listOf(listOf(1L)))
+        .baselineValues(listOf(listOf(2L)))
+        .go();
+  }
+
+  @Test
+  public void testRepeatedListEmptyLast() throws Exception {
+    final String query = "select * from cp.`vector/complex/repeated-list-empty-last.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf(listOf(1L)))
+        .baselineValues(listOf(listOf(2L)))
+        .baselineValues(listOf())
+        .go();
+  }
+
+  @Test
+  public void testRepeatedListEmptyBetween() throws Exception {
+    final String query = "select * from cp.`vector/complex/repeated-list-empty-between.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf(listOf(1L)))
+        .baselineValues(listOf())
+        .baselineValues(listOf(listOf(2L)))
+        .go();
+  }
+
+
+  @Test
+  public void testRepeatedMapEmptyFirst() throws Exception {
+    final String query = "select * from cp.`vector/complex/repeated-map-empty-first.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf())
+        .baselineValues(listOf(mapOf("b", 1L)))
+        .baselineValues(listOf(mapOf("b", 2L)))
+        .go();
+  }
+
+  @Test
+  public void testRepeatedMapEmptyLast() throws Exception {
+    final String query = "select * from cp.`vector/complex/repeated-map-empty-last.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf(mapOf("b", 1L)))
+        .baselineValues(listOf(mapOf("b", 2L)))
+        .baselineValues(listOf())
+        .go();
+  }
+
+  @Test
+  public void testRepeatedMapEmptyBetween() throws Exception {
+    final String query = "select * from cp.`vector/complex/repeated-map-empty-between.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf(mapOf("b", 1L)))
+        .baselineValues(listOf())
+        .baselineValues(listOf(mapOf("b", 2L)))
+        .go();
+  }
+
+  @Test
+  public void testMapEmptyFirst() throws Exception {
+    final String query = "select * from cp.`vector/complex/map-empty-first.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a", "c")
+        .baselineValues(mapOf(), 1L)
+        .baselineValues(mapOf("b", 1L), null)
+        .baselineValues(mapOf("b", 2L), null)
+        .go();
+  }
+
+  @Test
+  public void testMapEmptyLast() throws Exception {
+    final String query = "select * from cp.`vector/complex/map-empty-last.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a", "c")
+        .baselineValues(mapOf("b", 1L), null)
+        .baselineValues(mapOf("b", 2L), null)
+        .baselineValues(mapOf(), 1L)
+        .go();
+  }
+
+  @Test
+  public void testMapEmptyBetween() throws Exception {
+    final String query = "select * from cp.`vector/complex/map-empty-between.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a", "c")
+        .baselineValues(mapOf("b", 1L), null)
+        .baselineValues(mapOf(), 1L)
+        .baselineValues(mapOf("b", 2L), null)
+        .go();
+  }
+
+  @Test
+  public void testMultiLevelRepeatedListEmptyFirst() throws Exception {
+    final String query = "select * from cp.`vector/complex/multi-repeated-list-empty-first.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf())
+        .baselineValues(listOf(listOf(listOf(1L), listOf(3L)), listOf(listOf(5L, 7L))))
+        .baselineValues(listOf(listOf(listOf(2L), listOf(4L))))
+        .go();
+  }
+
+  @Test
+  public void testMultiLevelRepeatedListEmptyLast() throws Exception {
+    final String query = "select * from cp.`vector/complex/multi-repeated-list-empty-last.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf(listOf(listOf(1L), listOf(3L)), listOf(listOf(5L, 7L))))
+        .baselineValues(listOf(listOf(listOf(2L), listOf(4L))))
+        .baselineValues(listOf())
+        .go();
+  }
+
+  @Test
+  public void testMultiLevelRepeatedListEmptyBetween() throws Exception {
+    final String query = "select * from cp.`vector/complex/multi-repeated-list-empty-between.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf(listOf(listOf(1L), listOf(3L)), listOf(listOf(5L, 7L))))
+        .baselineValues(listOf())
+        .baselineValues(listOf(listOf(listOf(2L), listOf(4L))))
+        .go();
+  }
+
+
+  @Test
+  public void testMultiLevelRepeatedListWithMultipleEmpties() throws Exception {
+    final String query = "select * from cp.`vector/complex/multi-repeated-list-multi-empty.json`";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a")
+        .baselineValues(listOf())
+        .baselineValues(listOf(listOf(listOf(1L), listOf(3L)), listOf(listOf())))
+        .baselineValues(listOf(listOf(listOf(2L), listOf()), listOf()))
+        .go();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulator.java
deleted file mode 100644
index 23cc316..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/TestEmptyPopulator.java
+++ /dev/null
@@ -1,86 +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.complex;
-
-import org.apache.drill.exec.ExecTest;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.memory.TopLevelAllocator;
-import org.apache.drill.exec.vector.UInt4Vector;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.runners.MockitoJUnitRunner;
-
-@RunWith(MockitoJUnitRunner.class)
-public class TestEmptyPopulator extends ExecTest {
-  private static final int BUF_SIZE = 10000;
-
-  private UInt4Vector offsets;
-  private UInt4Vector.Accessor accessor;
-  private UInt4Vector.Mutator mutator;
-  private EmptyValuePopulator populator;
-  private BufferAllocator allocator = new TopLevelAllocator();
-
-
-  @Before
-  public void initialize() {
-    offsets = new UInt4Vector(null, allocator);
-    offsets.allocateNewSafe();
-    accessor = offsets.getAccessor();
-    mutator = offsets.getMutator();
-    mutator.set(0, 0);
-    mutator.setValueCount(1);
-    Assert.assertTrue("offsets must have one value", accessor.getValueCount() == 1);
-    populator = new EmptyValuePopulator(offsets);
-  }
-
-  @Test(expected = IndexOutOfBoundsException.class)
-  public void testNegativeValuesThrowException() {
-    populator.populate(-1);
-  }
-
-  @Test
-  public void testZeroHasNoEffect() {
-    populator.populate(0);
-    Assert.assertTrue("offset must have one value", accessor.getValueCount() == 1);
-  }
-
-  @Test
-  public void testEmptyPopulationWorks() {
-    populator.populate(1);
-    Assert.assertEquals("offset must have valid size", 2, accessor.getValueCount());
-    Assert.assertEquals("value must match", 0, accessor.get(1));
-
-    mutator.set(1, 10);
-    populator.populate(2);
-    Assert.assertEquals("offset must have valid size", 3, accessor.getValueCount());
-    Assert.assertEquals("value must match", 10, accessor.get(1));
-
-    mutator.set(2, 20);
-    populator.populate(5);
-    Assert.assertEquals("offset must have valid size", 6, accessor.getValueCount());
-    for (int i=2; i<=5;i++) {
-      Assert.assertEquals(String.format("value at index[%s] must match", i), 20, accessor.get(i));
-    }
-
-    populator.populate(0);
-    Assert.assertEquals("offset must have valid size", 1, accessor.getValueCount());
-    Assert.assertEquals("value must match", 0, accessor.get(0));
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/map-empty-between.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/map-empty-between.json b/exec/java-exec/src/test/resources/vector/complex/map-empty-between.json
new file mode 100644
index 0000000..eb206d9
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/map-empty-between.json
@@ -0,0 +1,3 @@
+{"a":{b:1}}
+{"c": 1}
+{"a":{b:2}}

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/map-empty-first.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/map-empty-first.json b/exec/java-exec/src/test/resources/vector/complex/map-empty-first.json
new file mode 100644
index 0000000..ab3009d
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/map-empty-first.json
@@ -0,0 +1,3 @@
+{"c": 1}
+{"a":{b:1}}
+{"a":{b:2}}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/map-empty-last.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/map-empty-last.json b/exec/java-exec/src/test/resources/vector/complex/map-empty-last.json
new file mode 100644
index 0000000..7b74271
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/map-empty-last.json
@@ -0,0 +1,3 @@
+{"a":{b:1}}
+{"a":{b:2}}
+{"c": 1}

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-between.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-between.json b/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-between.json
new file mode 100644
index 0000000..0c007a9
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-between.json
@@ -0,0 +1,3 @@
+{"a":[[[1], [3]], [[5, 7]]]}
+{"a":[]}
+{"a":[[[2], [4]]]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-first.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-first.json b/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-first.json
new file mode 100644
index 0000000..469bb57
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-first.json
@@ -0,0 +1,3 @@
+{"a":[]}
+{"a":[[[1], [3]], [[5, 7]]]}
+{"a":[[[2], [4]]]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-last.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-last.json b/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-last.json
new file mode 100644
index 0000000..73bf266
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-empty-last.json
@@ -0,0 +1,3 @@
+{"a":[[[1], [3]], [[5, 7]]]}
+{"a":[[[2], [4]]]}
+{"a":[]}

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-multi-empty.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-multi-empty.json b/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-multi-empty.json
new file mode 100644
index 0000000..9b911de
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/multi-repeated-list-multi-empty.json
@@ -0,0 +1,3 @@
+{"a":[]}
+{"a":[[[1], [3]], [[]]]}
+{"a":[[[2], []], []]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-between.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-between.json b/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-between.json
new file mode 100644
index 0000000..4136783
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-between.json
@@ -0,0 +1,3 @@
+{"a":[[1]]}
+{"a":[]}
+{"a":[[2]]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-first.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-first.json b/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-first.json
new file mode 100644
index 0000000..6ac00b7
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-first.json
@@ -0,0 +1,3 @@
+{"a":[]}
+{"a":[[1]]}
+{"a":[[2]]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-last.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-last.json b/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-last.json
new file mode 100644
index 0000000..8b0c3d5
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/repeated-list-empty-last.json
@@ -0,0 +1,3 @@
+{"a":[[1]]}
+{"a":[[2]]}
+{"a":[]}

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-between.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-between.json b/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-between.json
new file mode 100644
index 0000000..1aa626a
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-between.json
@@ -0,0 +1,3 @@
+{"a":[{b:1}]}
+{"a":[]}
+{"a":[{b:2}]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-first.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-first.json b/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-first.json
new file mode 100644
index 0000000..7c2301c
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-first.json
@@ -0,0 +1,3 @@
+{"a":[]}
+{"a":[{b:1}]}
+{"a":[{b:2}]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-last.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-last.json b/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-last.json
new file mode 100644
index 0000000..e9816d9
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/repeated-map-empty-last.json
@@ -0,0 +1,3 @@
+{"a":[{b:1}]}
+{"a":[{b:2}]}
+{"a":[]}

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-between.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-between.json b/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-between.json
new file mode 100644
index 0000000..e303329
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-between.json
@@ -0,0 +1,3 @@
+{"a":[1]}
+{"a":[]}
+{"a":[2]}

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-first.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-first.json b/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-first.json
new file mode 100644
index 0000000..2121221
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-first.json
@@ -0,0 +1,3 @@
+{"a":[]}
+{"a":[1]}
+{"a":[2]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/27b4aae2/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-last.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-last.json b/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-last.json
new file mode 100644
index 0000000..926b370
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/repeated-scalar-empty-last.json
@@ -0,0 +1,3 @@
+{"a":[1]}
+{"a":[2]}
+{"a":[]}