This is an automated email from the ASF dual-hosted git repository.

arina pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git


The following commit(s) were added to refs/heads/master by this push:
     new 8c37156  DRILL-7413: Test and fix scan operator vectors
8c37156 is described below

commit 8c37156a91ee531e4d1b672a41255cdeb11bdd57
Author: Paul Rogers <par0...@yahoo.com>
AuthorDate: Sun Oct 20 12:03:28 2019 -0700

    DRILL-7413: Test and fix scan operator vectors
    
    Enables vector validation tests for the ScanBatch and all
    EasyFormat plugins. Fixes a bug in scan batch that failed to set
    the record count in the output container.
    
    Fixes a number of formatting and other issues found while adding
    the tests.
---
 .../apache/drill/exec/physical/impl/ScanBatch.java |  28 +-
 .../physical/impl/validate/BatchValidator.java     |  62 +++-
 .../apache/drill/exec/record/VectorContainer.java  |   5 +
 .../exec/store/easy/json/JSONFormatPlugin.java     |  24 +-
 .../sequencefile/SequenceFileFormatPlugin.java     |   9 +-
 .../drill/exec/record/vector/TestDateTypes.java    | 376 ++++++++++-----------
 .../test/resources/parquet/alltypes_repeated.json  |   4 +-
 7 files changed, 278 insertions(+), 230 deletions(-)

diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index cdb36f4..79b40ee 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -17,9 +17,11 @@
  */
 package org.apache.drill.exec.physical.impl;
 
-import 
org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
-import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
-import io.netty.buffer.DrillBuf;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.SchemaPath;
@@ -54,10 +56,10 @@ import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.SchemaChangeCallBack;
 import org.apache.drill.exec.vector.ValueVector;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
+import 
org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import io.netty.buffer.DrillBuf;
 
 /**
  * Record batch used for a particular scan. Operators against one or more
@@ -155,6 +157,8 @@ public class ScanBatch implements CloseableRecordBatch {
     return context;
   }
 
+  public OperatorContext getOperatorContext() { return oContext; }
+
   @Override
   public BatchSchema getSchema() {
     return schema;
@@ -224,7 +228,8 @@ public class ScanBatch implements CloseableRecordBatch {
       logger.trace("currentReader.next return recordCount={}", recordCount);
       Preconditions.checkArgument(recordCount >= 0, "recordCount from 
RecordReader.next() should not be negative");
       boolean isNewSchema = mutator.isNewSchema();
-      populateImplicitVectorsAndSetCount();
+      populateImplicitVectors();
+      mutator.container.setValueCount(recordCount);
       oContext.getStats().batchReceived(0, recordCount, isNewSchema);
 
       boolean toContinueIter = true;
@@ -339,12 +344,8 @@ public class ScanBatch implements CloseableRecordBatch {
     }
   }
 
-  private void populateImplicitVectorsAndSetCount() {
+  private void populateImplicitVectors() {
     mutator.populateImplicitVectors(implicitValues, recordCount);
-    for (Map.Entry<String, ValueVector> entry: 
mutator.fieldVectorMap().entrySet()) {
-      logger.debug("set record count {} for vv {}", recordCount, 
entry.getKey());
-      entry.getValue().getMutator().setValueCount(recordCount);
-    }
   }
 
   @Override
@@ -602,6 +603,7 @@ public class ScanBatch implements CloseableRecordBatch {
   public VectorContainer getContainer() {
     return container;
   }
+
   /**
    * Verify list of implicit column values is valid input:
    *   - Either implicit column list is empty;
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
index 2753f55..7b9ff80 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
@@ -20,12 +20,16 @@ package org.apache.drill.exec.physical.impl.validate;
 import java.util.IdentityHashMap;
 import java.util.Map;
 
-import org.apache.drill.exec.physical.impl.protocol.OperatorRecordBatch;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.SimpleVectorWrapper;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.store.dfs.FormatPlugin;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
 import org.apache.drill.exec.vector.BitVector;
 import org.apache.drill.exec.vector.FixedWidthVector;
 import org.apache.drill.exec.vector.NullableVector;
@@ -33,6 +37,7 @@ import org.apache.drill.exec.vector.RepeatedBitVector;
 import org.apache.drill.exec.vector.UInt1Vector;
 import org.apache.drill.exec.vector.UInt4Vector;
 import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarBinaryVector;
 import org.apache.drill.exec.vector.VarCharVector;
 import org.apache.drill.exec.vector.VariableWidthVector;
 import org.apache.drill.exec.vector.complex.BaseRepeatedValueVector;
@@ -137,9 +142,9 @@ public class BatchValidator {
     }
   }
 
-  private enum CheckMode { COUNTS, ALL };
+  private enum CheckMode { COUNTS, ALL, NONE, SPECIAL };
 
-  private static final Map<Class<? extends RecordBatch>, CheckMode> checkRules 
= buildRules();
+  private static final Map<Class<?>, CheckMode> checkRules = buildRules();
 
   private final ErrorReporter errorReporter;
 
@@ -153,18 +158,44 @@ public class BatchValidator {
    * Over time, this table should include all operators, and thus become
    * unnecessary.
    */
-  private static Map<Class<? extends RecordBatch>, CheckMode> buildRules() {
-    Map<Class<? extends RecordBatch>, CheckMode> rules = new 
IdentityHashMap<>();
-    rules.put(OperatorRecordBatch.class, CheckMode.ALL);
+  private static Map<Class<?>, CheckMode> buildRules() {
+    final Map<Class<?>, CheckMode> rules = new IdentityHashMap<>();
+    // Operators
+    rules.put(ScanBatch.class, CheckMode.SPECIAL);
+    // Scan types
+    rules.put(EasySubScan.class, CheckMode.ALL);
     return rules;
   }
 
+  private static CheckMode lookup(Object subject) {
+    final CheckMode checkMode = checkRules.get(subject.getClass());
+    return checkMode == null ? CheckMode.NONE : checkMode;
+  }
+
+  private static CheckMode checkMode(RecordBatch batch) {
+    final CheckMode checkMode = lookup(batch);
+    if (checkMode != CheckMode.SPECIAL) {
+      return checkMode;
+    }
+    // For Scan, enable readers one-by-one.
+    final ScanBatch scan = (ScanBatch) batch;
+    final OperatorContext opContext = scan.getOperatorContext();
+    final PhysicalOperator opDefn = opContext.getOperatorDefn();
+    final CheckMode opCheckMode = lookup(opDefn);
+    if (opCheckMode != CheckMode.SPECIAL) {
+      return opCheckMode;
+    }
+    final EasySubScan easyScan = (EasySubScan) opDefn;
+    final FormatPlugin plugin = easyScan.getFormatPlugin();
+    return lookup(plugin);
+  }
+
   public static boolean validate(RecordBatch batch) {
-    CheckMode checkMode = checkRules.get(batch.getClass());
+    final CheckMode checkMode = checkMode(batch);
 
     // If no rule, don't check this batch.
 
-    if (checkMode == null) {
+    if (checkMode == CheckMode.NONE) {
 
       // As work proceeds, might want to log those batches not checked.
       // For now, there are too many.
@@ -312,6 +343,8 @@ public class BatchValidator {
 
     if (vector instanceof VarCharVector) {
       validateVarCharVector(name, (VarCharVector) vector);
+    } else if (vector instanceof VarBinaryVector) {
+      validateVarBinaryVector(name, (VarBinaryVector) vector);
     } else {
       logger.debug("Don't know how to validate vector: {}  of class {}",
           name, vector.getClass().getSimpleName());
@@ -331,6 +364,19 @@ public class BatchValidator {
     validateOffsetVector(name + "-offsets", vector.getOffsetVector(), false, 
valueCount, dataLength);
   }
 
+  private void validateVarBinaryVector(String name, VarBinaryVector vector) {
+    final int valueCount = vector.getAccessor().getValueCount();
+
+    // Disabled because a large number of operators
+    // set up offset vectors wrongly.
+    if (valueCount == 0) {
+      return;
+    }
+
+    final int dataLength = vector.getBuffer().writerIndex();
+    validateOffsetVector(name + "-offsets", vector.getOffsetVector(), false, 
valueCount, dataLength);
+  }
+
   private void validateRepeatedVector(String name, BaseRepeatedValueVector 
vector) {
     ValueVector dataVector = vector.getDataVector();
     int dataLength = dataVector.getAccessor().getValueCount();
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 c2553c2..4ee50f5 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
@@ -448,6 +448,11 @@ public class VectorContainer implements VectorAccessible {
     return true;
   }
 
+  public void setValueCount(int valueCount) {
+    VectorAccessibleUtilities.setValueCount(this, valueCount);
+    setRecordCount(valueCount);
+  }
+
   /**
    * Merge two batches to create a single, combined, batch. Vectors
    * appear in the order defined by {@link BatchSchema#merge(BatchSchema)}.
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
index fea350a..c812999 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
@@ -17,15 +17,11 @@
  */
 package org.apache.drill.exec.store.easy.json;
 
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.List;
 import java.util.Map;
+
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
@@ -51,9 +47,17 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.ObjectMapper;
 
 public class JSONFormatPlugin extends EasyFormatPlugin<JSONFormatConfig> {
-  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(JSONFormatPlugin.class);
+  private static final Logger logger = 
LoggerFactory.getLogger(JSONFormatPlugin.class);
   public static final String DEFAULT_NAME = "json";
 
   private static final boolean IS_COMPRESSIBLE = true;
@@ -77,11 +81,7 @@ public class JSONFormatPlugin extends 
EasyFormatPlugin<JSONFormatConfig> {
 
   @Override
   public boolean isStatisticsRecordWriter(FragmentContext context, EasyWriter 
writer) {
-    if (context.getSQLStatementType() == SqlStatementType.ANALYZE) {
-      return true;
-    } else {
-      return false;
-    }
+    return context.getSQLStatementType() == SqlStatementType.ANALYZE;
   }
 
   @Override
@@ -181,7 +181,7 @@ public class JSONFormatPlugin extends 
EasyFormatPlugin<JSONFormatConfig> {
 
     @Override
     public int hashCode() {
-      final int prime = 31;
+      int prime = 31;
       int result = 1;
       result = prime * result + ((extensions == null) ? 0 : 
extensions.hashCode());
       return result;
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java
index 701c5c5..707bdeb 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/sequencefile/SequenceFileFormatPlugin.java
@@ -17,13 +17,16 @@
  */
 package org.apache.drill.exec.store.easy.sequencefile;
 
+import java.io.IOException;
+import java.util.List;
+
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
-import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.store.RecordWriter;
@@ -38,9 +41,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileSplit;
 
-import java.io.IOException;
-import java.util.List;
-
 public class SequenceFileFormatPlugin extends 
EasyFormatPlugin<SequenceFileFormatConfig> {
   public SequenceFileFormatPlugin(String name, DrillbitContext context, 
Configuration fsConf,
                                   StoragePluginConfig storageConfig) {
@@ -59,7 +59,6 @@ public class SequenceFileFormatPlugin extends 
EasyFormatPlugin<SequenceFileForma
     return true;
   }
 
-
   @Override
   public AbstractGroupScan getGroupScan(String userName, FileSelection 
selection, List<SchemaPath> columns)
     throws IOException {
diff --git 
a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
 
b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
index d2d1a12..0a89557 100644
--- 
a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
+++ 
b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.drill.categories.SlowTest;
 import org.apache.drill.categories.VectorTest;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.client.DrillClient;
@@ -33,251 +34,246 @@ import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.categories.SlowTest;
-import org.junit.Test;
-
 import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
 import org.apache.drill.shaded.guava.com.google.common.io.Files;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-/* This class tests the existing date types. Simply using date types
+/* Tests the existing date types. Simply using date types
  * by casting from VarChar, performing basic functions and converting
  * back to VarChar.
  */
 @Category({SlowTest.class, VectorTest.class})
 public class TestDateTypes extends PopUnitTestBase {
-//    private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(TestDateTypes.class);
 
-    @Test
-    public void testDate() throws Exception {
-        try (RemoteServiceSet serviceSet = 
RemoteServiceSet.getLocalServiceSet();
-             Drillbit bit = new Drillbit(CONFIG, serviceSet);
-             DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
+  @Test
+  public void testDate() throws Exception {
+    try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+       Drillbit bit = new Drillbit(CONFIG, serviceSet);
+       DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
 
-            // run query.
-            bit.run();
-            client.connect();
-            List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
-                    
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_date.json"),
 Charsets.UTF_8).read()
-                            .replace("#{TEST_FILE}", 
"/test_simple_date.json"));
+      // run query.
+      bit.run();
+      client.connect();
+      List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+              
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_date.json"),
 Charsets.UTF_8).read()
+                      .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
-            RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
+      RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryDataBatch batch = results.get(0);
-            assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
+      QueryDataBatch batch = results.get(0);
+      assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
 
-            for (VectorWrapper<?> v : batchLoader) {
+      for (VectorWrapper<?> v : batchLoader) {
 
-                ValueVector.Accessor accessor = 
v.getValueVector().getAccessor();
+        ValueVector.Accessor accessor = v.getValueVector().getAccessor();
 
-                assertEquals((accessor.getObject(0).toString()), 
("1970-01-02"));
-                assertEquals((accessor.getObject(1).toString()), 
("2008-12-28"));
-                assertEquals((accessor.getObject(2).toString()), 
("2000-02-27"));
-            }
+        assertEquals((accessor.getObject(0).toString()), ("1970-01-02"));
+        assertEquals((accessor.getObject(1).toString()), ("2008-12-28"));
+        assertEquals((accessor.getObject(2).toString()), ("2000-02-27"));
+      }
 
-            batchLoader.clear();
-            for(QueryDataBatch b : results){
-              b.release();
-            }
-        }
+      batchLoader.clear();
+      for(QueryDataBatch b : results){
+        b.release();
+      }
     }
+  }
 
-    @Test
-    public void testSortDate() throws Exception {
-        try (RemoteServiceSet serviceSet = 
RemoteServiceSet.getLocalServiceSet();
-             Drillbit bit = new Drillbit(CONFIG, serviceSet);
-             DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
+  @Test
+  public void testSortDate() throws Exception {
+    try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+       Drillbit bit = new Drillbit(CONFIG, serviceSet);
+       DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
 
-            // run query.
-            bit.run();
-            client.connect();
-            List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
-                    
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_sort_date.json"),
 Charsets.UTF_8).read()
-                            .replace("#{TEST_FILE}", 
"/test_simple_date.json"));
+      // run query.
+      bit.run();
+      client.connect();
+      List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+              
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_sort_date.json"),
 Charsets.UTF_8).read()
+                      .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
-            RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
+      RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryDataBatch batch = results.get(1);
-            assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
+      QueryDataBatch batch = results.get(1);
+      assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
 
-            for (VectorWrapper<?> v : batchLoader) {
+      for (VectorWrapper<?> v : batchLoader) {
 
-                ValueVector.Accessor accessor = 
v.getValueVector().getAccessor();
+        ValueVector.Accessor accessor = v.getValueVector().getAccessor();
 
-                assertEquals((accessor.getObject(0).toString()), "1970-01-02");
-                assertEquals((accessor.getObject(1).toString()), "2000-02-27");
-                assertEquals((accessor.getObject(2).toString()), "2008-12-28");
-            }
+        assertEquals((accessor.getObject(0).toString()), "1970-01-02");
+        assertEquals((accessor.getObject(1).toString()), "2000-02-27");
+        assertEquals((accessor.getObject(2).toString()), "2008-12-28");
+      }
 
-            batchLoader.clear();
-            for(QueryDataBatch b : results){
-              b.release();
-            }
-        }
+      batchLoader.clear();
+      for(QueryDataBatch b : results){
+        b.release();
+      }
     }
+  }
 
-    @Test
-    public void testTimeStamp() throws Exception {
-        try (RemoteServiceSet serviceSet = 
RemoteServiceSet.getLocalServiceSet();
-             Drillbit bit = new Drillbit(CONFIG, serviceSet);
-             DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
+  @Test
+  public void testTimeStamp() throws Exception {
+    try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+       Drillbit bit = new Drillbit(CONFIG, serviceSet);
+       DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
 
-            // run query.
-            bit.run();
-            client.connect();
-            List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
-                    
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_timestamp.json"),
 Charsets.UTF_8).read()
-                            .replace("#{TEST_FILE}", 
"/test_simple_date.json"));
+      // run query.
+      bit.run();
+      client.connect();
+      List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+              
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_timestamp.json"),
 Charsets.UTF_8).read()
+                      .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
-            RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
+      RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryDataBatch batch = results.get(0);
-            assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
+      QueryDataBatch batch = results.get(0);
+      assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
 
-            for (VectorWrapper<?> v : batchLoader) {
+      for (VectorWrapper<?> v : batchLoader) {
 
-                ValueVector.Accessor accessor = 
v.getValueVector().getAccessor();
+        ValueVector.Accessor accessor = v.getValueVector().getAccessor();
 
-                assertEquals(accessor.getObject(0).toString(),"1970-01-02 
10:20:33.000");
-                assertEquals(accessor.getObject(1).toString(),"2008-12-28 
11:34:00.129");
-                assertEquals(accessor.getObject(2).toString(), "2000-02-27 
14:24:00.000");
-            }
+        assertEquals(accessor.getObject(0).toString(),"1970-01-02 
10:20:33.000");
+        assertEquals(accessor.getObject(1).toString(),"2008-12-28 
11:34:00.129");
+        assertEquals(accessor.getObject(2).toString(), "2000-02-27 
14:24:00.000");
+      }
 
-            batchLoader.clear();
-            for(QueryDataBatch b : results){
-              b.release();
-            }
-        }
+      batchLoader.clear();
+      for(QueryDataBatch b : results){
+        b.release();
+      }
     }
-
-    @Test
-    public void testInterval() throws Exception {
-        try (RemoteServiceSet serviceSet = 
RemoteServiceSet.getLocalServiceSet();
-             Drillbit bit = new Drillbit(CONFIG, serviceSet);
-             DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
-
-            // run query.
-            bit.run();
-            client.connect();
-            List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
-                    
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_interval.json"),
 Charsets.UTF_8).read()
-                            .replace("#{TEST_FILE}", 
"/test_simple_interval.json"));
-
-            RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
-
-            QueryDataBatch batch = results.get(0);
-            assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
-
-            Iterator<VectorWrapper<?>> itr = batchLoader.iterator();
-
-            ValueVector.Accessor accessor = 
itr.next().getValueVector().getAccessor();
-
-            // Check the interval type
-            assertEquals((accessor.getObject(0).toString()), ("2 years 2 
months 1 day 1:20:35.0"));
-            assertEquals((accessor.getObject(1).toString()), ("2 years 2 
months 0 days 0:0:0.0"));
-            assertEquals((accessor.getObject(2).toString()), ("0 years 0 
months 0 days 1:20:35.0"));
-            assertEquals((accessor.getObject(3).toString()),("2 years 2 months 
1 day 1:20:35.897"));
-            assertEquals((accessor.getObject(4).toString()), ("0 years 0 
months 0 days 0:0:35.4"));
-            assertEquals((accessor.getObject(5).toString()), ("1 year 10 
months 1 day 0:-39:-25.0"));
-
-            accessor = itr.next().getValueVector().getAccessor();
-
-            // Check the interval year type
-            assertEquals((accessor.getObject(0).toString()), ("2 years 2 
months "));
-            assertEquals((accessor.getObject(1).toString()), ("2 years 2 
months "));
-            assertEquals((accessor.getObject(2).toString()), ("0 years 0 
months "));
-            assertEquals((accessor.getObject(3).toString()), ("2 years 2 
months "));
-            assertEquals((accessor.getObject(4).toString()), ("0 years 0 
months "));
-            assertEquals((accessor.getObject(5).toString()), ("1 year 10 
months "));
-
-
-            accessor = itr.next().getValueVector().getAccessor();
-
-            // Check the interval day type
-            assertEquals((accessor.getObject(0).toString()), ("1 day 
1:20:35.0"));
-            assertEquals((accessor.getObject(1).toString()), ("0 days 
0:0:0.0"));
-            assertEquals((accessor.getObject(2).toString()), ("0 days 
1:20:35.0"));
-            assertEquals((accessor.getObject(3).toString()), ("1 day 
1:20:35.897"));
-            assertEquals((accessor.getObject(4).toString()), ("0 days 
0:0:35.4"));
-            assertEquals((accessor.getObject(5).toString()), ("1 day 
0:-39:-25.0"));
-
-            batchLoader.clear();
-            for(QueryDataBatch b : results){
-              b.release();
-            }
-        }
+  }
+
+  @Test
+  public void testInterval() throws Exception {
+    try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+         Drillbit bit = new Drillbit(CONFIG, serviceSet);
+         DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
+
+      // run query.
+      bit.run();
+      client.connect();
+      List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+              
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_interval.json"),
 Charsets.UTF_8).read()
+                      .replace("#{TEST_FILE}", "/test_simple_interval.json"));
+
+      RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
+
+      QueryDataBatch batch = results.get(0);
+      assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
+
+      Iterator<VectorWrapper<?>> itr = batchLoader.iterator();
+
+      ValueVector.Accessor accessor = 
itr.next().getValueVector().getAccessor();
+
+      // Check the interval type
+      assertEquals((accessor.getObject(0).toString()), ("2 years 2 months 1 
day 1:20:35.0"));
+      assertEquals((accessor.getObject(1).toString()), ("2 years 2 months 0 
days 0:0:0.0"));
+      assertEquals((accessor.getObject(2).toString()), ("0 years 0 months 0 
days 1:20:35.0"));
+      assertEquals((accessor.getObject(3).toString()),("2 years 2 months 1 day 
1:20:35.897"));
+      assertEquals((accessor.getObject(4).toString()), ("0 years 0 months 0 
days 0:0:35.4"));
+      assertEquals((accessor.getObject(5).toString()), ("1 year 10 months 1 
day 0:-39:-25.0"));
+
+      accessor = itr.next().getValueVector().getAccessor();
+
+      // Check the interval year type
+      assertEquals((accessor.getObject(0).toString()), ("2 years 2 months "));
+      assertEquals((accessor.getObject(1).toString()), ("2 years 2 months "));
+      assertEquals((accessor.getObject(2).toString()), ("0 years 0 months "));
+      assertEquals((accessor.getObject(3).toString()), ("2 years 2 months "));
+      assertEquals((accessor.getObject(4).toString()), ("0 years 0 months "));
+      assertEquals((accessor.getObject(5).toString()), ("1 year 10 months "));
+
+
+      accessor = itr.next().getValueVector().getAccessor();
+
+      // Check the interval day type
+      assertEquals((accessor.getObject(0).toString()), ("1 day 1:20:35.0"));
+      assertEquals((accessor.getObject(1).toString()), ("0 days 0:0:0.0"));
+      assertEquals((accessor.getObject(2).toString()), ("0 days 1:20:35.0"));
+      assertEquals((accessor.getObject(3).toString()), ("1 day 1:20:35.897"));
+      assertEquals((accessor.getObject(4).toString()), ("0 days 0:0:35.4"));
+      assertEquals((accessor.getObject(5).toString()), ("1 day 0:-39:-25.0"));
+
+      batchLoader.clear();
+      for(QueryDataBatch b : results){
+        b.release();
+      }
     }
+  }
 
-    @Test
-    public void testLiterals() throws Exception {
-        try (RemoteServiceSet serviceSet = 
RemoteServiceSet.getLocalServiceSet();
-             Drillbit bit = new Drillbit(CONFIG, serviceSet);
-             DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
+  @Test
+  public void testLiterals() throws Exception {
+    try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+         Drillbit bit = new Drillbit(CONFIG, serviceSet);
+         DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
 
-            // run query.
-            bit.run();
-            client.connect();
-            List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
-                    
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_all_date_literals.json"),
 Charsets.UTF_8).read()
-                            .replace("#{TEST_FILE}", 
"/test_simple_date.json"));
+      // run query.
+      bit.run();
+      client.connect();
+      List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+              
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_all_date_literals.json"),
 Charsets.UTF_8).read()
+                      .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
-            RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
+      RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryDataBatch batch = results.get(0);
-            assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
+      QueryDataBatch batch = results.get(0);
+      assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
 
-            String result[] = {"2008-02-27",
-                               "2008-02-27 01:02:03.000",
-                               "10:11:13.999",
-                               "2 years 2 months 3 days 0:1:3.89"};
+      String result[] = {"2008-02-27",
+                         "2008-02-27 01:02:03.000",
+                         "10:11:13.999",
+                         "2 years 2 months 3 days 0:1:3.89"};
 
-            int idx = 0;
+      int idx = 0;
 
-            for (VectorWrapper<?> v : batchLoader) {
+      for (VectorWrapper<?> v : batchLoader) {
 
-                ValueVector.Accessor accessor = 
v.getValueVector().getAccessor();
+        ValueVector.Accessor accessor = v.getValueVector().getAccessor();
 
-                assertEquals((accessor.getObject(0).toString()), 
(result[idx++]));
-            }
+        assertEquals((accessor.getObject(0).toString()), (result[idx++]));
+      }
 
-            batchLoader.clear();
-            for(QueryDataBatch b : results){
-              b.release();
-            }
-        }
+      batchLoader.clear();
+      for(QueryDataBatch b : results){
+        b.release();
+      }
     }
+  }
 
-    @Test
-    public void testDateAdd() throws Exception {
-        try (RemoteServiceSet serviceSet = 
RemoteServiceSet.getLocalServiceSet();
-             Drillbit bit = new Drillbit(CONFIG, serviceSet);
-             DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
-
-            // run query.
-            bit.run();
-            client.connect();
-            List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
-                    
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_date_add.json"),
 Charsets.UTF_8).read()
-                            .replace("#{TEST_FILE}", 
"/test_simple_date.json"));
-
-            RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
+  @Test
+  public void testDateAdd() throws Exception {
+    try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+         Drillbit bit = new Drillbit(CONFIG, serviceSet);
+         DrillClient client = new DrillClient(CONFIG, 
serviceSet.getCoordinator())) {
 
-            QueryDataBatch batch = results.get(0);
-            assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
+      // run query.
+      bit.run();
+      client.connect();
+      List<QueryDataBatch> results = 
client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+              
Files.asCharSource(DrillFileUtils.getResourceAsFile("/record/vector/test_date_add.json"),
 Charsets.UTF_8).read()
+                      .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
-            for (VectorWrapper<?> v : batchLoader) {
+      RecordBatchLoader batchLoader = new 
RecordBatchLoader(bit.getContext().getAllocator());
 
-                ValueVector.Accessor accessor = 
v.getValueVector().getAccessor();
+      QueryDataBatch batch = results.get(0);
+      assertTrue(batchLoader.load(batch.getHeader().getDef(), 
batch.getData()));
 
-                assertEquals((accessor.getObject(0).toString()), ("2008-03-27 
00:00:00.000"));
+      for (VectorWrapper<?> v : batchLoader) {
 
+        ValueVector.Accessor accessor = v.getValueVector().getAccessor();
 
-            }
+        assertEquals((accessor.getObject(0).toString()), ("2008-03-27 
00:00:00.000"));
+      }
 
-            batchLoader.clear();
-            for(QueryDataBatch b : results){
-              b.release();
-            }
-        }
+      batchLoader.clear();
+      for(QueryDataBatch b : results){
+        b.release();
+      }
     }
+  }
 }
diff --git a/exec/java-exec/src/test/resources/parquet/alltypes_repeated.json 
b/exec/java-exec/src/test/resources/parquet/alltypes_repeated.json
index 141c185..7ebf037 100644
--- a/exec/java-exec/src/test/resources/parquet/alltypes_repeated.json
+++ b/exec/java-exec/src/test/resources/parquet/alltypes_repeated.json
@@ -108,5 +108,5 @@
     "VARBINARY_col" : false,
     "VARCHAR_col" : [ "a string", "asdf", ""],
     "VAR16CHAR_col" : false,
-    "BIT_col" : [ true, true,  true]
-}
\ No newline at end of file
+    "BIT_col" : [ true, true, true]
+}

Reply via email to