Repository: drill
Updated Branches:
refs/heads/master e1649dd7d -> fde0a1df1
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/java/org/apache/drill/TestUnionDistinct.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestUnionDistinct.java b/exec/java-exec/src/test/java/org/apache/drill/TestUnionDistinct.java
index 056bc87..9d21bc9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestUnionDistinct.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestUnionDistinct.java
@@ -26,6 +26,7 @@ import org.apache.drill.common.types.TypeProtos;
import org.apache.drill.common.util.FileUtils;
import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
import org.apache.drill.exec.work.foreman.UnsupportedRelOperatorException;
+import org.junit.Ignore;
import org.junit.Test;
import java.util.List;
@@ -583,7 +584,7 @@ public class TestUnionDistinct extends BaseTestQuery {
}
@Test
- public void testUnionDistinctRightEmptyBatch() throws Exception {
+ public void testUnionDistinctRightEmptyDataBatch() throws Exception {
String rootSimple = FileUtils.getResourceAsFile("/store/json/booleanData.json").toURI().toString();
String queryRightEmptyBatch = String.format(
@@ -603,7 +604,7 @@ public class TestUnionDistinct extends BaseTestQuery {
}
@Test
- public void testUnionDistinctLeftEmptyBatch() throws Exception {
+ public void testUnionDistinctLeftEmptyDataBatch() throws Exception {
String rootSimple = FileUtils.getResourceAsFile("/store/json/booleanData.json").toURI().toString();
final String queryLeftBatch = String.format(
@@ -624,7 +625,7 @@ public class TestUnionDistinct extends BaseTestQuery {
}
@Test
- public void testUnionDistinctBothEmptyBatch() throws Exception {
+ public void testUnionDistinctBothEmptyDataBatch() throws Exception {
String rootSimple = FileUtils.getResourceAsFile("/store/json/booleanData.json").toURI().toString();
final String query = String.format(
"select key from dfs_test.`%s` where 1 = 0 " +
@@ -635,7 +636,7 @@ public class TestUnionDistinct extends BaseTestQuery {
final List<Pair<SchemaPath, TypeProtos.MajorType>> expectedSchema = Lists.newArrayList();
final TypeProtos.MajorType majorType = TypeProtos.MajorType.newBuilder()
- .setMinorType(TypeProtos.MinorType.INT)
+ .setMinorType(TypeProtos.MinorType.BIT) // field "key" has boolean type.
.setMode(TypeProtos.DataMode.OPTIONAL)
.build();
expectedSchema.add(Pair.of(SchemaPath.getSimplePath("key"), majorType));
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/java/org/apache/drill/exec/TestEmptyInputSql.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestEmptyInputSql.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestEmptyInputSql.java
new file mode 100644
index 0000000..bb707ca
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestEmptyInputSql.java
@@ -0,0 +1,203 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+import java.util.List;
+
+public class TestEmptyInputSql extends BaseTestQuery {
+
+ public final String SINGLE_EMPTY_JSON = "/scan/emptyInput/emptyJson/empty.json";
+ public final String SINGLE_EMPTY_CSVH = "/scan/emptyInput/emptyCsvH/empty.csvh";
+ public final String SINGLE_EMPTY_CSV = "/scan/emptyInput/emptyCsv/empty.csv";
+
+ /**
+ * Test with query against an empty file. Select clause has regular column reference, and an expression.
+ *
+ * regular column "key" is assigned with nullable-int
+ * expression "key + 100" is materialized with nullable-int as output type.
+ */
+ @Test
+ public void testQueryEmptyJson() throws Exception {
+ final String rootEmpty = FileUtils.getResourceAsFile(SINGLE_EMPTY_JSON).toURI().toString();
+ final String query = String.format("select key, key + 100 as key2 from dfs_test.`%s` ", rootEmpty);
+
+ final BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("key", TypeProtos.MinorType.INT)
+ .addNullable("key2", TypeProtos.MinorType.INT)
+ .build();
+
+ testBuilder()
+ .sqlQuery(query)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+ }
+
+ /**
+ * Test with query against an empty file. Select clause has one or more *
+ * star column is expanded into an empty list.
+ * @throws Exception
+ */
+ @Test
+ public void testQueryStarColEmptyJson() throws Exception {
+ final String rootEmpty = FileUtils.getResourceAsFile(SINGLE_EMPTY_JSON).toURI().toString();
+ final String query1 = String.format("select * from dfs_test.`%s` ", rootEmpty);
+
+ final BatchSchema expectedSchema = new SchemaBuilder()
+ .build();
+
+ testBuilder()
+ .sqlQuery(query1)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+
+ final String query2 = String.format("select *, * from dfs_test.`%s` ", rootEmpty);
+
+ testBuilder()
+ .sqlQuery(query2)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+ }
+
+ /**
+ * Test with query against an empty file. Select clause has one or more qualified *
+ * star column is expanded into an empty list.
+ * @throws Exception
+ */
+ @Test
+ public void testQueryQualifiedStarColEmptyJson() throws Exception {
+ final String rootEmpty = FileUtils.getResourceAsFile(SINGLE_EMPTY_JSON).toURI().toString();
+ final String query1 = String.format("select foo.* from dfs_test.`%s` as foo", rootEmpty);
+
+ final List<Pair<SchemaPath, TypeProtos.MajorType>> expectedSchema = Lists.newArrayList();
+
+ testBuilder()
+ .sqlQuery(query1)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+
+ final String query2 = String.format("select foo.*, foo.* from dfs_test.`%s` as foo", rootEmpty);
+
+ testBuilder()
+ .sqlQuery(query2)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+
+ }
+
+ @Test
+ public void testQueryMapArrayEmptyJson() throws Exception {
+ final String rootEmpty = FileUtils.getResourceAsFile(SINGLE_EMPTY_JSON).toURI().toString();
+ final String query = String.format("select foo.a.b as col1, foo.columns[2] as col2, foo.bar.columns[3] as col3 from dfs_test.`%s` as foo", rootEmpty);
+
+ final BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("col1", TypeProtos.MinorType.INT)
+ .addNullable("col2", TypeProtos.MinorType.INT)
+ .addNullable("col3", TypeProtos.MinorType.INT)
+ .build();
+
+ testBuilder()
+ .sqlQuery(query)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+ }
+
+ /**
+ * Test with query against an empty file. Select clause has three expressions.
+ * 1.0 + 100.0 as constant expression, is resolved to required FLOAT8
+ * cast(100 as varchar(100) is resolved to required varchar(100)
+ * cast(columns as varchar(100)) is resolved to nullable varchar(100).
+ */
+ @Test
+ public void testQueryConstExprEmptyJson() throws Exception {
+ final String rootEmpty = FileUtils.getResourceAsFile(SINGLE_EMPTY_JSON).toURI().toString();
+ final String query = String.format("select 1.0 + 100.0 as key, "
+ + " cast(100 as varchar(100)) as name, "
+ + " cast(columns as varchar(100)) as name2 "
+ + " from dfs_test.`%s` ", rootEmpty);
+
+ final BatchSchema expectedSchema = new SchemaBuilder()
+ .add("key", TypeProtos.MinorType.FLOAT8)
+ .add("name", TypeProtos.MinorType.VARCHAR, 100)
+ .addNullable("name2", TypeProtos.MinorType.VARCHAR, 100)
+ .build();
+
+ testBuilder()
+ .sqlQuery(query)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+ }
+
+ /**
+ * Test select * against empty csv with empty header. * is expanded into empty list of fields.
+ * @throws Exception
+ */
+ @Test
+ public void testQueryEmptyCsvH() throws Exception {
+ final String rootEmpty = FileUtils.getResourceAsFile(SINGLE_EMPTY_CSVH).toURI().toString();
+ final String query1 = String.format("select * from dfs_test.`%s` ", rootEmpty);
+
+ final BatchSchema expectedSchema = new SchemaBuilder()
+ .build();
+
+ testBuilder()
+ .sqlQuery(query1)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+ }
+
+ /**
+ * Test select * against empty csv file. * is exapnede into "columns : repeated-varchar",
+ * which is the default column from reading a csv file.
+ * @throws Exception
+ */
+ @Test
+ public void testQueryEmptyCsv() throws Exception {
+ final String rootEmpty = FileUtils.getResourceAsFile(SINGLE_EMPTY_CSV).toURI().toString();
+ final String query1 = String.format("select * from dfs_test.`%s` ", rootEmpty);
+
+ final BatchSchema expectedSchema = new SchemaBuilder()
+ .addArray("columns", TypeProtos.MinorType.VARCHAR)
+ .build();
+
+ testBuilder()
+ .sqlQuery(query1)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
index 598bdc2..4114a04 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
@@ -150,7 +150,7 @@ public class TestPartitionSender extends PlanTestBase {
Mockito.when(sv.get(i)).thenReturn(i);
}
- final TopNBatch.SimpleRecordBatch incoming = new TopNBatch.SimpleRecordBatch(container, sv, null);
+ final TopNBatch.SimpleSV4RecordBatch incoming = new TopNBatch.SimpleSV4RecordBatch(container, sv, null);
updateTestCluster(DRILLBITS_COUNT, null);
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
index 2a392d7..a11889d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
@@ -57,7 +57,7 @@ public class TestSimpleUnion extends ExecTest {
final FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
final SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
- final int[] counts = new int[]{100,50};
+ final int[] counts = new int[]{0, 100,50}; // first batch : 0-row schema-only batch.
int i = 0;
while(exec.next()) {
System.out.println("iteration count:" + exec.getRecordCount());
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
index 4f0fcbf..c53536a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
@@ -47,9 +47,11 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.TreeMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import static org.apache.drill.DrillTestWrapper.addToCombinedVectorResults;
import static org.apache.drill.exec.physical.base.AbstractBase.INIT_ALLOCATION;
import static org.apache.drill.exec.physical.base.AbstractBase.MAX_ALLOCATION;
@@ -69,8 +71,9 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
public static class MiniPlanTestBuilder {
protected List<Map<String, Object>> baselineRecords;
protected RecordBatch root;
- protected boolean expectedZeroBatch;
- protected BatchSchema expectedSchema;
+ protected Integer expectBatchNum = null;
+ protected BatchSchema expectSchema;
+ protected boolean expectZeroRow;
/**
* Specify the root operator for a MiniPlan.
@@ -87,8 +90,8 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
* @param batchSchema
* @return
*/
- public MiniPlanTestBuilder expectedSchema(BatchSchema batchSchema) {
- this.expectedSchema = batchSchema;
+ public MiniPlanTestBuilder expectSchema(BatchSchema batchSchema) {
+ this.expectSchema = batchSchema;
return this;
}
@@ -104,11 +107,11 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
Map<String, Object> ret = new HashMap<>();
int i = 0;
- Preconditions.checkArgument(expectedSchema != null , "Expected schema should be set before specify baseline values.");
- Preconditions.checkArgument(baselineValues.length == expectedSchema.getFieldCount(),
+ Preconditions.checkArgument(expectSchema != null , "Expected schema should be set before specify baseline values.");
+ Preconditions.checkArgument(baselineValues.length == expectSchema.getFieldCount(),
"Must supply the same number of baseline values as columns in expected schema.");
- for (MaterializedField field : expectedSchema) {
+ for (MaterializedField field : expectSchema) {
ret.put(SchemaPath.getSimplePath(field.getName()).toExpr(), baselineValues[i]);
i++;
}
@@ -119,11 +122,28 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
/**
* Specify one special case, where the operator tree should return 0 batch.
- * @param expectedZeroBatch
+ * @param expectNullBatch
* @return
*/
- public MiniPlanTestBuilder expectZeroBatch(boolean expectedZeroBatch) {
- this.expectedZeroBatch = expectedZeroBatch;
+ public MiniPlanTestBuilder expectNullBatch(boolean expectNullBatch) {
+ if (expectNullBatch) {
+ this.expectBatchNum = 0;
+ }
+ return this;
+ }
+
+ /**
+ * Specify the expected number of batches from operator tree.
+ * @param
+ * @return
+ */
+ public MiniPlanTestBuilder expectBatchNum(int expectBatchNum) {
+ this.expectBatchNum = expectBatchNum;
+ return this;
+ }
+
+ public MiniPlanTestBuilder expectZeroRow(boolean expectedZeroRow) {
+ this.expectZeroRow = expectedZeroRow;
return this;
}
@@ -131,16 +151,30 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
final BatchIterator batchIterator = new BatchIterator(root);
// verify case of zero batch.
- if (expectedZeroBatch) {
+ if (expectBatchNum != null && expectBatchNum == 0) {
if (batchIterator.iterator().hasNext()) {
- throw new AssertionError("Expected zero batches from scan. But scan return at least 1 batch!");
+ throw new AssertionError("Expected zero batches from operator tree. But operators return at least 1 batch!");
} else {
return; // successful
}
}
+ Map<String, List<Object>> actualSuperVectors = new TreeMap();
- Map<String, List<Object>> actualSuperVectors = DrillTestWrapper.addToCombinedVectorResults(batchIterator, expectedSchema);
- Map<String, List<Object>> expectedSuperVectors = DrillTestWrapper.translateRecordListToHeapVectors(baselineRecords);
+ int actualBatchNum = DrillTestWrapper.addToCombinedVectorResults(batchIterator, expectSchema, actualSuperVectors);
+ if (expectBatchNum != null) {
+ if (expectBatchNum != actualBatchNum) {
+ throw new AssertionError(String.format("Expected %s batches from operator tree. But operators return %s batch!", expectBatchNum, actualBatchNum));
+ }
+ }
+ Map<String, List<Object>> expectedSuperVectors;
+ if (!expectZeroRow) {
+ expectedSuperVectors = DrillTestWrapper.translateRecordListToHeapVectors(baselineRecords);
+ } else {
+ expectedSuperVectors = new TreeMap<>();
+ for (MaterializedField field : expectSchema) {
+ expectedSuperVectors.put(SchemaPath.getSimplePath(field.getName()).toExpr(), new ArrayList<>());
+ }
+ }
DrillTestWrapper.compareMergedVectors(expectedSuperVectors, actualSuperVectors);
}
}
@@ -221,7 +255,7 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
}
/**
- * Set initial memory reservation used by this operator's allocator. Default is {@link PhysicalOpUnitTestBase#INIT_ALLOCATION}
+ * Set initial memory reservation used by this operator's allocator. Default is {@link org.apache.drill.exec.physical.base.AbstractBase#INIT_ALLOCATION}
* @param initReservation
* @return
*/
@@ -231,7 +265,7 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
}
/**
- * Set max memory reservation used by this operator's allocator. Default is {@link PhysicalOpUnitTestBase#MAX_ALLOCATION}
+ * Set max memory reservation used by this operator's allocator. Default is {@link org.apache.drill.exec.physical.base.AbstractBase#MAX_ALLOCATION}
* @param maxAllocation
* @return
*/
@@ -366,7 +400,12 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
readers = TestUtilities.getJsonReadersFromInputFiles(fs, inputPaths, fragContext, columnsToRead);
}
- RecordBatch scanBatch = new ScanBatch(null, fragContext, readers);
+ List<RecordReader> readerList = new ArrayList<>();
+ while(readers.hasNext()) {
+ readerList.add(readers.next());
+ }
+
+ RecordBatch scanBatch = new ScanBatch(null, fragContext, readerList);
return scanBatch;
}
}
@@ -420,7 +459,7 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
}
}
- RecordBatch scanBatch = new ScanBatch(null, fragContext, readers.iterator());
+ RecordBatch scanBatch = new ScanBatch(null, fragContext, readers);
return scanBatch;
}
} // end of ParquetScanBuilder
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
index 7d09ca5..157f1d7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
@@ -207,7 +207,7 @@ public class PhysicalOpUnitTestBase extends ExecTest {
if (inputStreamsJSON != null) {
for (List<String> batchesJson : inputStreamsJSON) {
incomingStreams.add(new ScanBatch(null, fragContext,
- getRecordReadersForJsonBatches(batchesJson, fragContext)));
+ getReaderListForJsonBatches(batchesJson, fragContext)));
}
}
@@ -351,5 +351,13 @@ public class PhysicalOpUnitTestBase extends ExecTest {
return TestUtilities.getJsonReadersFromBatchString(jsonBatches, fragContext, Collections.singletonList(SchemaPath.getSimplePath("*")));
}
+ private List<RecordReader> getReaderListForJsonBatches(List<String> jsonBatches, FragmentContext fragContext) {
+ Iterator<RecordReader> readers = getRecordReadersForJsonBatches(jsonBatches, fragContext);
+ List<RecordReader> readerList = new ArrayList<>();
+ while(readers.hasNext()) {
+ readerList.add(readers.next());
+ }
+ return readerList;
+ }
}
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java
index d0a64f4..1a52a06 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestMiniPlan.java
@@ -30,7 +30,6 @@ import org.apache.drill.test.rowSet.SchemaBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.junit.BeforeClass;
-import org.junit.Ignore;
import org.junit.Test;
import java.util.Collections;
@@ -54,22 +53,6 @@ public class TestMiniPlan extends MiniPlanUnitTestBase {
}
@Test
- @Ignore("DRILL-5464: A bug in JsonRecordReader handling empty file")
- public void testEmptyJsonInput() throws Exception {
- String emptyFile = FileUtils.getResourceAsFile("/project/pushdown/empty.json").toURI().toString();
-
- RecordBatch scanBatch = new JsonScanBuilder()
- .fileSystem(fs)
- .inputPaths(Lists.newArrayList(emptyFile))
- .build();
-
- new MiniPlanTestBuilder()
- .root(scanBatch)
- .expectZeroBatch(true)
- .go();
- }
-
- @Test
public void testSimpleParquetScan() throws Exception {
String file = FileUtils.getResourceAsFile("/tpchmulti/region/01.parquet").toURI().toString();
@@ -85,7 +68,7 @@ public class TestMiniPlan extends MiniPlanUnitTestBase {
new MiniPlanTestBuilder()
.root(scanBatch)
- .expectedSchema(expectedSchema)
+ .expectSchema(expectedSchema)
.baselineValues(0L)
.baselineValues(1L)
.go();
@@ -107,7 +90,7 @@ public class TestMiniPlan extends MiniPlanUnitTestBase {
new MiniPlanTestBuilder()
.root(scanBatch)
- .expectedSchema(expectedSchema)
+ .expectSchema(expectedSchema)
.baselineValues(100L)
.go();
}
@@ -149,58 +132,12 @@ public class TestMiniPlan extends MiniPlanUnitTestBase {
new MiniPlanTestBuilder()
.root(batch)
- .expectedSchema(expectedSchema)
+ .expectSchema(expectedSchema)
.baselineValues(5l, 1l)
.baselineValues(5l, 5l)
.baselineValues(50l, 100l)
.go();
}
- @Test
- @Ignore ("DRILL-5327: A bug in UnionAll handling empty inputs from both sides")
- public void testUnionFilterAll() throws Exception {
- List<String> leftJsonBatches = Lists.newArrayList(
- "[{\"a\": 5, \"b\" : 1 }]");
-
- List<String> rightJsonBatches = Lists.newArrayList(
- "[{\"a\": 50, \"b\" : 10 }]");
-
- RecordBatch leftScan = new JsonScanBuilder()
- .jsonBatches(leftJsonBatches)
- .columnsToRead("a", "b")
- .build();
-
- RecordBatch leftFilter = new PopBuilder()
- .physicalOperator(new Filter(null, parseExpr("a < 0"), 1.0f))
- .addInput(leftScan)
- .build();
-
- RecordBatch rightScan = new JsonScanBuilder()
- .jsonBatches(rightJsonBatches)
- .columnsToRead("a", "b")
- .build();
-
- RecordBatch rightFilter = new PopBuilder()
- .physicalOperator(new Filter(null, parseExpr("a < 0"), 1.0f))
- .addInput(rightScan)
- .build();
-
- RecordBatch batch = new PopBuilder()
- .physicalOperator(new UnionAll(Collections.EMPTY_LIST)) // Children list is provided through RecordBatch
- .addInput(leftFilter)
- .addInput(rightFilter)
- .build();
-
- BatchSchema expectedSchema = new SchemaBuilder()
- .addNullable("a", TypeProtos.MinorType.BIGINT)
- .addNullable("b", TypeProtos.MinorType.BIGINT)
- .withSVMode(BatchSchema.SelectionVectorMode.NONE)
- .build();
-
- new MiniPlanTestBuilder()
- .root(batch)
- .expectedSchema(expectedSchema)
- .go();
- }
}
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java
new file mode 100644
index 0000000..1127314
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java
@@ -0,0 +1,572 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.drill.exec.physical.unit;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.physical.config.Filter;
+import org.apache.drill.exec.physical.config.FlattenPOP;
+import org.apache.drill.exec.physical.config.HashAggregate;
+import org.apache.drill.exec.physical.config.HashJoinPOP;
+import org.apache.drill.exec.physical.config.Limit;
+import org.apache.drill.exec.physical.config.MergeJoinPOP;
+import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.physical.config.StreamingAggregate;
+import org.apache.drill.exec.physical.config.UnionAll;
+import org.apache.drill.exec.planner.physical.AggPrelBase;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class TestNullInputMiniPlan extends MiniPlanUnitTestBase{
+ protected static DrillFileSystem fs;
+
+ public final String SINGLE_EMPTY_JSON = "/scan/emptyInput/emptyJson/empty.json";
+ public final String SINGLE_EMPTY_JSON2 = "/scan/emptyInput/emptyJson/empty2.json";
+ public final String SINGLE_JSON = "/scan/jsonTbl/1990/1.json"; // {id: 100, name : "John"}
+ public final String SINGLE_JSON2 = "/scan/jsonTbl/1991/2.json"; // {id: 1000, name : "Joe"}
+
+ @BeforeClass
+ public static void initFS() throws Exception {
+ Configuration conf = new Configuration();
+ conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
+ fs = new DrillFileSystem(conf);
+ }
+
+ /**
+ * Test ScanBatch with a single empty json file.
+ * @throws Exception
+ */
+ @Test
+ public void testEmptyJsonInput() throws Exception {
+ RecordBatch scanBatch = createScanBatchFromJson(SINGLE_EMPTY_JSON);
+
+ new MiniPlanTestBuilder()
+ .root(scanBatch)
+ .expectNullBatch(true)
+ .go();
+ }
+
+ /**
+ * Test ScanBatch with mixed json files.
+ * input is empty, data_file, empty, data_file
+ * */
+ @Test
+ public void testJsonInputMixedWithEmptyFiles1() throws Exception {
+ RecordBatch scanBatch = createScanBatchFromJson(SINGLE_EMPTY_JSON, SINGLE_JSON, SINGLE_EMPTY_JSON2, SINGLE_JSON2);
+
+ BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("id", TypeProtos.MinorType.BIGINT)
+ .addNullable("name", TypeProtos.MinorType.VARCHAR)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(scanBatch)
+ .expectSchema(expectedSchema)
+ .baselineValues(100L, "John")
+ .baselineValues(1000L, "Joe")
+ .expectBatchNum(2)
+ .go();
+
+ }
+
+ /**
+ * Test ScanBatch with mixed json files.
+ * input is empty, empty, data_file, data_file
+ * */
+ @Test
+ public void testJsonInputMixedWithEmptyFiles2() throws Exception {
+ RecordBatch scanBatch = createScanBatchFromJson(SINGLE_EMPTY_JSON, SINGLE_EMPTY_JSON2, SINGLE_JSON, SINGLE_JSON2);
+
+ BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("id", TypeProtos.MinorType.BIGINT)
+ .addNullable("name", TypeProtos.MinorType.VARCHAR)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(scanBatch)
+ .expectSchema(expectedSchema)
+ .baselineValues(100L, "John")
+ .baselineValues(1000L, "Joe")
+ .expectBatchNum(2)
+ .go();
+ }
+
+ /**
+ * Test ScanBatch with mixed json files.
+ * input is empty, data_file, data_file, empty
+ * */
+ @Test
+ public void testJsonInputMixedWithEmptyFiles3() throws Exception {
+ RecordBatch scanBatch = createScanBatchFromJson(SINGLE_EMPTY_JSON, SINGLE_JSON, SINGLE_JSON2, SINGLE_EMPTY_JSON2);
+
+ BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("id", TypeProtos.MinorType.BIGINT)
+ .addNullable("name", TypeProtos.MinorType.VARCHAR)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(scanBatch)
+ .expectSchema(expectedSchema)
+ .baselineValues(100L, "John")
+ .baselineValues(1000L, "Joe")
+ .expectBatchNum(2)
+ .go();
+ }
+
+ /**
+ * Test ScanBatch with mixed json files.
+ * input is data_file, data_file, empty, empty
+ * */
+ @Test
+ public void testJsonInputMixedWithEmptyFiles4() throws Exception {
+ RecordBatch scanBatch = createScanBatchFromJson(SINGLE_JSON, SINGLE_JSON2, SINGLE_EMPTY_JSON2, SINGLE_EMPTY_JSON2);
+
+ BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("id", TypeProtos.MinorType.BIGINT)
+ .addNullable("name", TypeProtos.MinorType.VARCHAR)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(scanBatch)
+ .expectSchema(expectedSchema)
+ .baselineValues(100L, "John")
+ .baselineValues(1000L, "Joe")
+ .expectBatchNum(2)
+ .go();
+ }
+
+ @Test
+ public void testProjectEmpty() throws Exception {
+ final PhysicalOperator project = new Project(parseExprs("x+5", "x"), null);
+ testSingleInputNullBatchHandling(project);
+ }
+
+ @Test
+ public void testFilterEmpty() throws Exception {
+ final PhysicalOperator filter = new Filter(null, parseExpr("a=5"), 1.0f);
+ testSingleInputNullBatchHandling(filter);
+ }
+
+ @Test
+ public void testHashAggEmpty() throws Exception {
+ final PhysicalOperator hashAgg = new HashAggregate(null, AggPrelBase.OperatorPhase.PHASE_1of1, parseExprs("a", "a"), parseExprs("sum(b)", "b_sum"), 1.0f);
+ testSingleInputNullBatchHandling(hashAgg);
+ }
+
+ @Test
+ public void testStreamingAggEmpty() throws Exception {
+ final PhysicalOperator hashAgg = new StreamingAggregate(null, parseExprs("a", "a"), parseExprs("sum(b)", "b_sum"), 1.0f);
+ testSingleInputNullBatchHandling(hashAgg);
+ }
+
+ @Test
+ public void testSortEmpty() throws Exception {
+ final PhysicalOperator sort = new ExternalSort(null,
+ Lists.newArrayList(ordering("b", RelFieldCollation.Direction.ASCENDING, RelFieldCollation.NullDirection.FIRST)), false);
+ testSingleInputNullBatchHandling(sort);
+ }
+
+ @Test
+ public void testLimitEmpty() throws Exception {
+ final PhysicalOperator limit = new Limit(null, 10, 5);
+ testSingleInputNullBatchHandling(limit);
+ }
+
+ @Test
+ public void testFlattenEmpty() throws Exception {
+ final PhysicalOperator flatten = new FlattenPOP(null, SchemaPath.getSimplePath("col1"));
+ testSingleInputNullBatchHandling(flatten);
+ }
+
+ @Test
+ public void testUnionEmptyBoth() throws Exception {
+ final PhysicalOperator unionAll = new UnionAll(Collections.EMPTY_LIST); // Children list is provided through RecordBatch
+ testTwoInputNullBatchHandling(unionAll);
+ }
+
+ @Test
+ public void testHashJoinEmptyBoth() throws Exception {
+ final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.INNER);
+ testTwoInputNullBatchHandling(join);
+ }
+
+ @Test
+ public void testLeftHashJoinEmptyBoth() throws Exception {
+ final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.LEFT);
+ testTwoInputNullBatchHandling(join);
+ }
+
+ @Test
+ public void testRightHashJoinEmptyBoth() throws Exception {
+ final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.RIGHT);
+ testTwoInputNullBatchHandling(join);
+ }
+
+ @Test
+ public void testFullHashJoinEmptyBoth() throws Exception {
+ final PhysicalOperator join = new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.FULL);
+ testTwoInputNullBatchHandling(join);
+ }
+
+ @Test
+ public void testMergeJoinEmptyBoth() throws Exception {
+ final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.INNER);
+ testTwoInputNullBatchHandling(join);
+ }
+
+ @Test
+ public void testLeftMergeJoinEmptyBoth() throws Exception {
+ final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.LEFT);
+ testTwoInputNullBatchHandling(join);
+ }
+
+ @Test
+ public void testRightMergeJoinEmptyBoth() throws Exception {
+ final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.RIGHT);
+ testTwoInputNullBatchHandling(join);
+ }
+
+ @Test
+ @Ignore("Full Merge join is not supported.")
+ public void testFullMergeJoinEmptyBoth() throws Exception {
+ final PhysicalOperator join = new MergeJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "b")), JoinRelType.FULL);
+ testTwoInputNullBatchHandling(join);
+ }
+
+ @Test
+ public void testUnionLeftEmtpy() throws Exception {
+ final PhysicalOperator unionAll = new UnionAll(Collections.EMPTY_LIST); // Children list is provided through RecordBatch
+
+ RecordBatch left = createScanBatchFromJson(SINGLE_EMPTY_JSON);
+
+ String file = FileUtils.getResourceAsFile("/tpchmulti/region/01.parquet").toURI().toString();
+
+ RecordBatch scanBatch = new ParquetScanBuilder()
+ .fileSystem(fs)
+ .columnsToRead("R_REGIONKEY")
+ .inputPaths(Lists.newArrayList(file))
+ .build();
+
+ RecordBatch projectBatch = new PopBuilder()
+ .physicalOperator(new Project(parseExprs("R_REGIONKEY+10", "regionkey"), null))
+ .addInput(scanBatch)
+ .build();
+
+ RecordBatch unionBatch = new PopBuilder()
+ .physicalOperator(unionAll)
+ .addInput(left)
+ .addInput(projectBatch)
+ .build();
+
+ BatchSchema expectedSchema = new SchemaBuilder()
+ .add("regionkey", TypeProtos.MinorType.BIGINT)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(unionBatch)
+ .expectSchema(expectedSchema)
+ .baselineValues(10L)
+ .baselineValues(11L)
+ .go();
+ }
+
+
+ @Test
+ public void testHashJoinLeftEmpty() throws Exception {
+ RecordBatch left = createScanBatchFromJson(SINGLE_EMPTY_JSON);
+
+ List<String> rightJsonBatches = Lists.newArrayList(
+ "[{\"a\": 50, \"b\" : 10 }]");
+
+ RecordBatch rightScan = new JsonScanBuilder()
+ .jsonBatches(rightJsonBatches)
+ .columnsToRead("a", "b")
+ .build();
+
+ RecordBatch joinBatch = new PopBuilder()
+ .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.INNER))
+ .addInput(left)
+ .addInput(rightScan)
+ .build();
+
+ BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("a", TypeProtos.MinorType.BIGINT)
+ .addNullable("b", TypeProtos.MinorType.BIGINT)
+ .withSVMode(BatchSchema.SelectionVectorMode.NONE)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(joinBatch)
+ .expectSchema(expectedSchema)
+ .expectZeroRow(true)
+ .go();
+ }
+
+ @Test
+ public void testHashJoinRightEmpty() throws Exception {
+ List<String> leftJsonBatches = Lists.newArrayList(
+ "[{\"a\": 50, \"b\" : 10 }]");
+
+ RecordBatch leftScan = new JsonScanBuilder()
+ .jsonBatches(leftJsonBatches)
+ .columnsToRead("a", "b")
+ .build();
+
+ RecordBatch right = createScanBatchFromJson(SINGLE_EMPTY_JSON);
+
+ RecordBatch joinBatch = new PopBuilder()
+ .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.INNER))
+ .addInput(leftScan)
+ .addInput(right)
+ .build();
+
+ BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("a", TypeProtos.MinorType.BIGINT)
+ .addNullable("b", TypeProtos.MinorType.BIGINT)
+ .withSVMode(BatchSchema.SelectionVectorMode.NONE)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(joinBatch)
+ .expectSchema(expectedSchema)
+ .expectZeroRow(true)
+ .go();
+ }
+
+
+ @Test
+ public void testLeftHashJoinLeftEmpty() throws Exception {
+ RecordBatch left = createScanBatchFromJson(SINGLE_EMPTY_JSON);
+
+ List<String> rightJsonBatches = Lists.newArrayList(
+ "[{\"a\": 50, \"b\" : 10 }]");
+
+ RecordBatch rightScan = new JsonScanBuilder()
+ .jsonBatches(rightJsonBatches)
+ .columnsToRead("a", "b")
+ .build();
+
+ RecordBatch joinBatch = new PopBuilder()
+ .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.LEFT))
+ .addInput(left)
+ .addInput(rightScan)
+ .build();
+
+ BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("a", TypeProtos.MinorType.BIGINT)
+ .addNullable("b", TypeProtos.MinorType.BIGINT)
+ .withSVMode(BatchSchema.SelectionVectorMode.NONE)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(joinBatch)
+ .expectSchema(expectedSchema)
+ .expectZeroRow(true)
+ .go();
+ }
+
+ @Test
+ public void testLeftHashJoinRightEmpty() throws Exception {
+ List<String> leftJsonBatches = Lists.newArrayList(
+ "[{\"a\": 50, \"b\" : 10 }]");
+
+ RecordBatch leftScan = new JsonScanBuilder()
+ .jsonBatches(leftJsonBatches)
+ .columnsToRead("a", "b")
+ .build();
+
+ RecordBatch right = createScanBatchFromJson(SINGLE_EMPTY_JSON);
+
+ RecordBatch joinBatch = new PopBuilder()
+ .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.LEFT))
+ .addInput(leftScan)
+ .addInput(right)
+ .build();
+
+ BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("a", TypeProtos.MinorType.BIGINT)
+ .addNullable("b", TypeProtos.MinorType.BIGINT)
+ .withSVMode(BatchSchema.SelectionVectorMode.NONE)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(joinBatch)
+ .expectSchema(expectedSchema)
+ .baselineValues(50L, 10L)
+ .go();
+ }
+
+ @Test
+ public void testUnionFilterAll() throws Exception {
+ List<String> leftJsonBatches = Lists.newArrayList(
+ "[{\"a\": 5, \"b\" : \"name1\" }]");
+
+ List<String> rightJsonBatches = Lists.newArrayList(
+ "[{\"a\": 50, \"b\" : \"name2\" }]");
+
+ RecordBatch leftScan = new JsonScanBuilder()
+ .jsonBatches(leftJsonBatches)
+ .columnsToRead("a", "b")
+ .build();
+
+ RecordBatch leftFilter = new PopBuilder()
+ .physicalOperator(new Filter(null, parseExpr("a < 0"), 1.0f))
+ .addInput(leftScan)
+ .build();
+
+ RecordBatch rightScan = new JsonScanBuilder()
+ .jsonBatches(rightJsonBatches)
+ .columnsToRead("a", "b")
+ .build();
+
+ RecordBatch rightFilter = new PopBuilder()
+ .physicalOperator(new Filter(null, parseExpr("a < 0"), 1.0f))
+ .addInput(rightScan)
+ .build();
+
+ RecordBatch batch = new PopBuilder()
+ .physicalOperator(new UnionAll(Collections.EMPTY_LIST)) // Children list is provided through RecordBatch
+ .addInput(leftFilter)
+ .addInput(rightFilter)
+ .build();
+
+ BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("a", TypeProtos.MinorType.BIGINT)
+ .addNullable("b", TypeProtos.MinorType.VARCHAR)
+ .withSVMode(BatchSchema.SelectionVectorMode.NONE)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(batch)
+ .expectSchema(expectedSchema)
+ .expectZeroRow(true)
+ .go();
+ }
+
+ @Test
+ public void testOutputProjectEmpty() throws Exception {
+ final PhysicalOperator project = new Project(
+ parseExprs(
+ "x", "col1",
+ "x + 100", "col2",
+ "100.0", "col3",
+ "cast(nonExist as varchar(100))", "col4"), null, true);
+
+ BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("col1", TypeProtos.MinorType.INT)
+ .addNullable("col2", TypeProtos.MinorType.INT)
+ .add("col3", TypeProtos.MinorType.FLOAT8)
+ .addNullable("col4", TypeProtos.MinorType.VARCHAR, 100)
+ .withSVMode(BatchSchema.SelectionVectorMode.NONE)
+ .build();
+
+ final RecordBatch input = createScanBatchFromJson(SINGLE_EMPTY_JSON);
+
+ RecordBatch batch = new PopBuilder()
+ .physicalOperator(project) // Children list is provided through RecordBatch
+ .addInput(input)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(batch)
+ .expectSchema(expectedSchema)
+ .expectZeroRow(true)
+ .go();
+ }
+
+ /**
+ * Given a physical, first construct scan batch from one single empty json, then construct scan batch from
+ * multiple empty json files. In both case, verify that the output is a NullBatch.
+ * @param pop
+ * @throws Exception
+ */
+ private void testSingleInputNullBatchHandling(PhysicalOperator pop) throws Exception {
+ final RecordBatch input = createScanBatchFromJson(SINGLE_EMPTY_JSON);
+
+ RecordBatch batch = new PopBuilder()
+ .physicalOperator(pop)
+ .addInput(input)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(batch)
+ .expectNullBatch(true)
+ .go();
+
+ final RecordBatch input2 = createScanBatchFromJson(SINGLE_EMPTY_JSON, SINGLE_EMPTY_JSON2);;
+ RecordBatch batch2 = new PopBuilder()
+ .physicalOperator(pop)
+ .addInput(input2)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(batch2)
+ .expectNullBatch(true)
+ .go();
+ }
+
+ private void testTwoInputNullBatchHandling(PhysicalOperator pop) throws Exception {
+ RecordBatch left = createScanBatchFromJson(SINGLE_EMPTY_JSON);
+ RecordBatch right = createScanBatchFromJson(SINGLE_EMPTY_JSON);
+
+ RecordBatch joinBatch = new PopBuilder()
+ .physicalOperator(pop)
+ .addInput(left)
+ .addInput(right)
+ .build();
+
+ new MiniPlanTestBuilder()
+ .root(joinBatch)
+ .expectNullBatch(true)
+ .go();
+ }
+
+ private RecordBatch createScanBatchFromJson(String... resourcePaths) throws Exception {
+ List<String> inputPaths = new ArrayList<>();
+
+ for (String resource : resourcePaths) {
+ inputPaths.add(FileUtils.getResourceAsFile(resource).toURI().toString());
+ }
+
+ RecordBatch scanBatch = new JsonScanBuilder()
+ .fileSystem(fs)
+ .inputPaths(inputPaths)
+ .build();
+
+ return scanBatch;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestImplicitFileColumns.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestImplicitFileColumns.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestImplicitFileColumns.java
index 3974448..d1a16df 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestImplicitFileColumns.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestImplicitFileColumns.java
@@ -20,9 +20,13 @@ package org.apache.drill.exec.store;
import com.google.common.base.Charsets;
import com.google.common.io.Files;
import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.FileUtils;
import org.apache.drill.common.util.TestTools;
+import org.apache.drill.exec.record.BatchSchema;
import org.apache.drill.exec.util.JsonStringArrayList;
import org.apache.drill.exec.util.Text;
+import org.apache.drill.test.rowSet.SchemaBuilder;
import org.apache.hadoop.fs.Path;
import org.junit.Before;
import org.junit.Rule;
@@ -36,6 +40,9 @@ public class TestImplicitFileColumns extends BaseTestQuery {
public static final String MAIN = "main";
public static final String NESTED = "nested";
public static final String CSV = "csv";
+ public final String JSON_TBL = "/scan/jsonTbl"; // 1990/1.json : {id:100, name: "John"}, 1991/2.json : {id: 1000, name : "Joe"}
+ public final String PARQUET_TBL = "/multilevel/parquet/"; // 1990/Q1/orders_1990_q1.parquet, ...
+ public final String CSV_TBL = "/multilevel/csv"; // 1990/Q1/orders_1990_q1.csv, ..
private static final JsonStringArrayList<Text> mainColumnValues = new JsonStringArrayList<Text>() {{
add(new Text(MAIN));
@@ -147,4 +154,66 @@ public class TestImplicitFileColumns extends BaseTestQuery {
}
}
+ @Test
+ public void testStarColumnJson() throws Exception {
+ final String rootEmpty = FileUtils.getResourceAsFile(JSON_TBL).toURI().toString();
+ final String query1 = String.format("select * from dfs_test.`%s` ", rootEmpty);
+
+ final BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("dir0", TypeProtos.MinorType.VARCHAR)
+ .addNullable("id", TypeProtos.MinorType.BIGINT)
+ .addNullable("name", TypeProtos.MinorType.VARCHAR)
+ .build();
+
+ testBuilder()
+ .sqlQuery(query1)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+ }
+
+ @Test
+ public void testStarColumnParquet() throws Exception {
+ final String rootEmpty = FileUtils.getResourceAsFile(PARQUET_TBL).toURI().toString();
+ final String query1 = String.format("select * from dfs_test.`%s` ", rootEmpty);
+
+ final BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("dir0", TypeProtos.MinorType.VARCHAR)
+ .addNullable("dir1", TypeProtos.MinorType.VARCHAR)
+ .add("o_orderkey", TypeProtos.MinorType.INT)
+ .add("o_custkey", TypeProtos.MinorType.INT)
+ .add("o_orderstatus", TypeProtos.MinorType.VARCHAR)
+ .add("o_totalprice", TypeProtos.MinorType.FLOAT8)
+ .add("o_orderdate", TypeProtos.MinorType.DATE)
+ .add("o_orderpriority", TypeProtos.MinorType.VARCHAR)
+ .add("o_clerk", TypeProtos.MinorType.VARCHAR)
+ .add("o_shippriority", TypeProtos.MinorType.INT)
+ .add("o_comment", TypeProtos.MinorType.VARCHAR)
+ .build();
+
+ testBuilder()
+ .sqlQuery(query1)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+ }
+
+ @Test
+ public void testStarColumnCsv() throws Exception {
+ final String rootEmpty = FileUtils.getResourceAsFile(CSV_TBL).toURI().toString();
+ final String query1 = String.format("select * from dfs_test.`%s` ", rootEmpty);
+
+ final BatchSchema expectedSchema = new SchemaBuilder()
+ .addNullable("dir0", TypeProtos.MinorType.VARCHAR)
+ .addNullable("dir1", TypeProtos.MinorType.VARCHAR)
+ .addArray("columns", TypeProtos.MinorType.VARCHAR)
+ .build();
+
+ testBuilder()
+ .sqlQuery(query1)
+ .schemaBaseLine(expectedSchema)
+ .build()
+ .run();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/resources/scan/emptyInput/emptyCsv/empty.csv
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/scan/emptyInput/emptyCsv/empty.csv b/exec/java-exec/src/test/resources/scan/emptyInput/emptyCsv/empty.csv
new file mode 100644
index 0000000..e69de29
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/resources/scan/emptyInput/emptyCsvH/empty.csvh
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/scan/emptyInput/emptyCsvH/empty.csvh b/exec/java-exec/src/test/resources/scan/emptyInput/emptyCsvH/empty.csvh
new file mode 100644
index 0000000..e69de29
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/resources/scan/emptyInput/emptyJson/empty.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/scan/emptyInput/emptyJson/empty.json b/exec/java-exec/src/test/resources/scan/emptyInput/emptyJson/empty.json
new file mode 100644
index 0000000..e69de29
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/resources/scan/emptyInput/emptyJson/empty2.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/scan/emptyInput/emptyJson/empty2.json b/exec/java-exec/src/test/resources/scan/emptyInput/emptyJson/empty2.json
new file mode 100644
index 0000000..e69de29
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/resources/scan/jsonTbl/1990/1.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/scan/jsonTbl/1990/1.json b/exec/java-exec/src/test/resources/scan/jsonTbl/1990/1.json
new file mode 100644
index 0000000..e9f1e9a
--- /dev/null
+++ b/exec/java-exec/src/test/resources/scan/jsonTbl/1990/1.json
@@ -0,0 +1,2 @@
+{id: 100, name : "John"}
+
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/java-exec/src/test/resources/scan/jsonTbl/1991/2.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/scan/jsonTbl/1991/2.json b/exec/java-exec/src/test/resources/scan/jsonTbl/1991/2.json
new file mode 100644
index 0000000..7496839
--- /dev/null
+++ b/exec/java-exec/src/test/resources/scan/jsonTbl/1991/2.json
@@ -0,0 +1 @@
+{id: 1000, name : "Joe"}
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/vector/src/main/codegen/templates/BasicTypeHelper.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/BasicTypeHelper.java b/exec/vector/src/main/codegen/templates/BasicTypeHelper.java
index 016199a..c589ed7 100644
--- a/exec/vector/src/main/codegen/templates/BasicTypeHelper.java
+++ b/exec/vector/src/main/codegen/templates/BasicTypeHelper.java
@@ -16,6 +16,9 @@
* limitations under the License.
*/
+import org.apache.drill.exec.vector.UntypedNullHolder;
+import org.apache.drill.exec.vector.UntypedNullVector;
+
<@pp.dropOutputFile />
<@pp.changeOutputFile name="/org/apache/drill/exec/expr/BasicTypeHelper.java" />
@@ -108,6 +111,8 @@ public class BasicTypeHelper {
</#list>
case GENERIC_OBJECT :
return ObjectVector.class ;
+ case NULL:
+ return UntypedNullVector.class;
default:
break;
}
@@ -271,6 +276,8 @@ public class BasicTypeHelper {
</#list>
case GENERIC_OBJECT:
return new ObjectVector(field, allocator) ;
+ case NULL:
+ return new UntypedNullVector(field, allocator);
default:
break;
}
@@ -348,6 +355,8 @@ public class BasicTypeHelper {
case GENERIC_OBJECT:
((ObjectVector) vector).getMutator().setSafe(index, (ObjectHolder) holder);
return;
+ case NULL:
+ ((UntypedNullVector) vector).getMutator().setSafe(index, (UntypedNullHolder) holder);
default:
throw new UnsupportedOperationException(buildErrorMessage("set value", type));
}
@@ -376,7 +385,9 @@ public class BasicTypeHelper {
</#list>
case GENERIC_OBJECT:
((ObjectVector) vector).getMutator().setSafe(index, (ObjectHolder) holder);
- default:
+ case NULL:
+ ((UntypedNullVector) vector).getMutator().setSafe(index, (UntypedNullHolder) holder);
+ default:
throw new UnsupportedOperationException(buildErrorMessage("set value safe", type));
}
}
@@ -428,6 +439,8 @@ public class BasicTypeHelper {
</#list>
case GENERIC_OBJECT:
return new ObjectHolder();
+ case NULL:
+ return new UntypedNullHolder();
default:
throw new UnsupportedOperationException(buildErrorMessage("create value holder", type));
}
@@ -451,6 +464,8 @@ public class BasicTypeHelper {
}
</#list>
</#list>
+ case NULL:
+ return true;
default:
throw new UnsupportedOperationException(buildErrorMessage("check is null", type));
}
@@ -532,7 +547,9 @@ public class BasicTypeHelper {
}
</#list>
</#list>
-
+ else if (holder instanceof UntypedNullHolder) {
+ return UntypedNullHolder.TYPE;
+ }
throw new UnsupportedOperationException("ValueHolder is not supported for 'getValueHolderType' method.");
}
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/vector/src/main/codegen/templates/ValueHolders.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/ValueHolders.java b/exec/vector/src/main/codegen/templates/ValueHolders.java
index 11607b4..f134049 100644
--- a/exec/vector/src/main/codegen/templates/ValueHolders.java
+++ b/exec/vector/src/main/codegen/templates/ValueHolders.java
@@ -35,9 +35,7 @@ package org.apache.drill.exec.expr.holders;
public final class ${className} implements ValueHolder{
public static final MajorType TYPE = Types.${mode.name?lower_case}(MinorType.${minor.class?upper_case});
-
- public MajorType getType() {return TYPE;}
-
+
<#if mode.name == "Repeated">
/** The first index (inclusive) into the Vector. **/
@@ -93,7 +91,9 @@ public final class ${className} implements ValueHolder{
return ((buffer.getInt(start) & 0x80000000) != 0);
}
</#if></#if>
-
+
+ public MajorType getType() {return TYPE;}
+
@Deprecated
public int hashCode(){
throw new UnsupportedOperationException();
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index 1ecedc6..e2b44a7 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -207,6 +207,16 @@ public class MaterializedField {
return builder.toString();
}
+ /**
+ * Return true if two fields have identical MinorType and Mode.
+ * @param that
+ * @return
+ */
+ public boolean hasSameTypeAndMode(MaterializedField that) {
+ return (getType().getMinorType() == that.getType().getMinorType())
+ && (getType().getMode() == that.getType().getMode());
+ }
+
private String toString(Collection<?> collection, int maxLen) {
StringBuilder builder = new StringBuilder();
builder.append(" [");
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java
new file mode 100644
index 0000000..a205eda
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullHolder.java
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.expr.holders.ValueHolder;
+
+public class UntypedNullHolder implements ValueHolder {
+ public static final TypeProtos.MajorType TYPE = Types.optional(TypeProtos.MinorType.NULL);
+ public static final int WIDTH = 0;
+ public int isSet = 1;
+
+ public TypeProtos.MajorType getType() {return TYPE;}
+
+ @Deprecated
+ public int hashCode(){
+ throw new UnsupportedOperationException();
+ }
+
+ /*
+ * Reason for deprecation is that ValueHolders are potential scalar replacements
+ * and hence we don't want any methods to be invoked on them.
+ */
+ @Deprecated
+ public String toString(){
+ throw new UnsupportedOperationException();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/drill/blob/fde0a1df/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java
new file mode 100644
index 0000000..8288fe2
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java
@@ -0,0 +1,270 @@
+/*
+ * 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;
+
+
+import com.google.common.base.Preconditions;
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.SerializedField;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+
+import static org.apache.calcite.sql.parser.impl.SqlParserImplConstants.C;
+
+/** UntypedNullVector is to represent a value vector with {@link org.apache.drill.common.types.MinorType#NULL}
+ * All values in the vector represent two semantic implications: 1) the value is unknown, 2) the type is unknown.
+ * Because of this, we only have to keep track of the number of values in value vector,
+ * and there is no allocated buffer to back up this value vector. Therefore, the majority of
+ * methods in this class is either no-op, or throws {@link UnsupportedOperationException}.
+ *
+ */
+public final class UntypedNullVector extends BaseDataValueVector implements FixedWidthVector {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UntypedNullVector.class);
+
+ /**
+ * Width of each fixed-width value.
+ */
+ public static final int VALUE_WIDTH = 0;
+
+ private final Accessor accessor = new Accessor();
+ private final Mutator mutator = new Mutator();
+ private int valueCount;
+
+ public UntypedNullVector(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator);
+ valueCount = 0;
+ }
+
+ @Override
+ public FieldReader getReader() { throw new UnsupportedOperationException(); }
+
+ @Override
+ public int getBufferSizeFor(final int valueCount) {
+ return 0;
+ }
+
+ @Override
+ public int getValueCapacity(){
+ return ValueVector.MAX_ROW_COUNT;
+ }
+
+ @Override
+ public Accessor getAccessor() { return accessor; }
+
+ @Override
+ public Mutator getMutator() { return mutator; }
+
+ @Override
+ public void setInitialCapacity(final int valueCount) {
+ }
+
+ @Override
+ public void allocateNew() {
+ }
+
+ @Override
+ public boolean allocateNewSafe() {
+ return true;
+ }
+
+ @Override
+ public void allocateNew(final int valueCount) {
+ }
+
+ @Override
+ public void reset() {
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public void zeroVector() {
+ }
+
+ @Override
+ public void load(SerializedField metadata, DrillBuf buffer) {
+ Preconditions.checkArgument(this.field.getName().equals(metadata.getNamePart().getName()),
+ "The field %s doesn't match the provided metadata %s.", this.field, metadata);
+ final int actualLength = metadata.getBufferLength();
+ final int valueCount = metadata.getValueCount();
+ final int expectedLength = valueCount * VALUE_WIDTH;
+ assert actualLength == expectedLength : String.format("Expected to load %d bytes but actually loaded %d bytes", expectedLength, actualLength);
+
+ this.valueCount = valueCount;
+ }
+
+ @Override
+ public TransferPair getTransferPair(BufferAllocator allocator){
+ return new TransferImpl(getField(), allocator);
+ }
+
+ @Override
+ public TransferPair getTransferPair(String ref, BufferAllocator allocator){
+ return new TransferImpl(getField().withPath(ref), allocator);
+ }
+
+ @Override
+ public TransferPair makeTransferPair(ValueVector to) {
+ return new TransferImpl((UntypedNullVector) to);
+ }
+
+ public void transferTo(UntypedNullVector target){
+ }
+
+ public void splitAndTransferTo(int startIndex, int length, UntypedNullVector target) {
+ }
+
+ @Override
+ public int getPayloadByteCount(int valueCount) {
+ return 0;
+ }
+
+ private class TransferImpl implements TransferPair{
+ private UntypedNullVector to;
+
+ public TransferImpl(MaterializedField field, BufferAllocator allocator){
+ to = new UntypedNullVector(field, allocator);
+ }
+
+ public TransferImpl(UntypedNullVector to) {
+ this.to = to;
+ }
+
+ @Override
+ public UntypedNullVector getTo(){
+ return to;
+ }
+
+ @Override
+ public void transfer(){
+ transferTo(to);
+ }
+
+ @Override
+ public void splitAndTransfer(int startIndex, int length) {
+ checkBounds(startIndex);
+ checkBounds(startIndex + length - 1);
+ splitAndTransferTo(startIndex, length, to);
+ }
+
+ @Override
+ public void copyValueSafe(int fromIndex, int toIndex) {
+ checkBounds(fromIndex);
+ to.copyFromSafe(fromIndex, toIndex, UntypedNullVector.this);
+ }
+ }
+
+ public void copyFrom(int fromIndex, int thisIndex, UntypedNullVector from){
+ }
+
+ public void copyFromSafe(int fromIndex, int thisIndex, UntypedNullVector from){
+ }
+
+ private void checkBounds(int index) {
+ if (index < 0 || index >= valueCount) {
+ throw new IndexOutOfBoundsException(String.format(
+ "index: %d, expected: range(0, %d-1))", index, valueCount));
+ }
+ }
+ @Override
+ public void copyEntry(int toIndex, ValueVector from, int fromIndex) {
+ ((UntypedNullVector) from).data.getBytes(fromIndex * 4, data, toIndex * 4, 4);
+ }
+
+ public final class Accessor extends BaseAccessor {
+ @Override
+ public int getValueCount() {
+ return valueCount;
+ }
+
+ @Override
+ public boolean isNull(int index){
+ checkBounds(index);
+ return true;
+ }
+
+ public int isSet(int index) {
+ checkBounds(index);
+ return 0;
+ }
+
+ @Override
+ public Object getObject(int index) {
+ checkBounds(index);
+ return null;
+ }
+
+ public void get(int index, UntypedNullHolder holder) {
+ checkBounds(index);
+ }
+
+ }
+
+ /**
+ * UntypedNullVector.Mutator throws Exception for most of its mutate operations, except for the ones that set
+ * value counts.
+ *
+ */
+ public final class Mutator extends BaseMutator {
+
+ private Mutator() {}
+
+ public void set(int index, UntypedNullHolder holder) {
+ throw new UnsupportedOperationException("UntypedNullVector does not support set");
+ }
+
+ public void set(int index, int isSet, UntypedNullHolder holder) {
+ throw new UnsupportedOperationException("UntypedNullVector does not support set");
+ }
+
+ public void setSafe(int index, UntypedNullHolder holder) {
+ throw new UnsupportedOperationException("UntypedNullVector does not support setSafe");
+ }
+
+ public void setSafe(int index, int isSet, UntypedNullHolder holder) {
+ throw new UnsupportedOperationException("UntypedNullVector does not support setSafe");
+ }
+
+ public void setScalar(int index, UntypedNullHolder holder) throws VectorOverflowException {
+ throw new UnsupportedOperationException("UntypedNullVector does not support setScalar");
+ }
+
+ public void setArrayItem(int index, UntypedNullHolder holder) throws VectorOverflowException {
+ throw new UnsupportedOperationException("UntypedNullVector does not support setArrayItem");
+ }
+
+ @Override
+ public void generateTestData(int size) {
+ setValueCount(size);
+ }
+
+ public void generateTestDataAlt(int size) {
+ setValueCount(size);
+ }
+
+ @Override
+ public void setValueCount(int valueCount) {
+ UntypedNullVector.this.valueCount = valueCount;
+ }
+ }
+
+}
\ No newline at end of file
|