You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by su...@apache.org on 2017/04/21 23:32:29 UTC

[2/3] drill git commit: DRILL-5323: Test tools for row sets

http://git-wip-us.apache.org/repos/asf/drill/blob/095a660b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
new file mode 100644
index 0000000..48657c7
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.test.rowSet;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * Builder of a row set schema expressed as a list of materialized
+ * fields. Optimized for use when creating schemas by hand in tests.
+ * <p>
+ * Example usage to create the following schema: <br>
+ * <tt>(c: INT, a: MAP(b: VARCHAR, d: INT, e: MAP(f: VARCHAR), g: INT), h: BIGINT)</tt>
+ * <p>
+ * Code:<pre><code>
+ *     BatchSchema batchSchema = new SchemaBuilder()
+ *        .add("c", MinorType.INT)
+ *        .addMap("a")
+ *          .addNullable("b", MinorType.VARCHAR)
+ *          .add("d", MinorType.INT)
+ *          .addMap("e")
+ *            .add("f", MinorType.VARCHAR)
+ *            .buildMap()
+ *          .add("g", MinorType.INT)
+ *          .buildMap()
+ *        .addArray("h", MinorType.BIGINT)
+ *        .build();
+ * </code</pre>
+ */
+
+public class SchemaBuilder {
+
+  /**
+   * Internal structure for building a map. A map is just a schema,
+   * but one that is part of a parent column.
+   */
+
+  public static class MapBuilder extends SchemaBuilder {
+    private final SchemaBuilder parent;
+    private final String memberName;
+
+    public MapBuilder(SchemaBuilder parent, String memberName) {
+      this.parent = parent;
+      this.memberName = memberName;
+    }
+
+    @Override
+    public BatchSchema build() {
+      throw new IllegalStateException("Cannot build for a nested schema");
+    }
+
+    @Override
+    public SchemaBuilder buildMap() {
+      MaterializedField col = MaterializedField.create(memberName,
+          MajorType.newBuilder()
+            .setMinorType(MinorType.MAP)
+            .setMode(DataMode.REQUIRED)
+            .build());
+      for (MaterializedField childCol : columns) {
+        col.addChild(childCol);
+      }
+      parent.finishMap(col);
+      return parent;
+    }
+  }
+
+  protected List<MaterializedField> columns = new ArrayList<>( );
+
+  public SchemaBuilder() { }
+
+  public SchemaBuilder add(String pathName, MajorType type) {
+    MaterializedField col = MaterializedField.create(pathName, type);
+    columns.add(col);
+    return this;
+  }
+
+  public SchemaBuilder add(String pathName, MinorType type, DataMode mode) {
+    return add(pathName, MajorType.newBuilder()
+        .setMinorType(type)
+        .setMode(mode)
+        .build());
+  }
+
+  public SchemaBuilder add(String pathName, MinorType type) {
+    return add(pathName, type, DataMode.REQUIRED);
+  }
+
+  public SchemaBuilder addNullable(String pathName, MinorType type) {
+    return add(pathName, type, DataMode.OPTIONAL);
+  }
+
+  public SchemaBuilder addArray(String pathName, MinorType type) {
+    return add(pathName, type, DataMode.REPEATED);
+  }
+
+  /**
+   * Add a map column. The returned schema builder is for the nested
+   * map. Building that map, using {@link MapBuilder#buildMap()},
+   * will return the original schema builder.
+   *
+   * @param pathName the name of the map column
+   * @return a builder for the map
+   */
+
+  public MapBuilder addMap(String pathName) {
+    return new MapBuilder(this, pathName);
+  }
+
+  public BatchSchema build() {
+    return new BatchSchema(SelectionVectorMode.NONE, columns);
+  }
+
+  void finishMap(MaterializedField map) {
+    columns.add(map);
+  }
+
+  public SchemaBuilder buildMap() {
+    throw new IllegalStateException("Cannot build map for a top-level schema");
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/095a660b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/package-info.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/package-info.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/package-info.java
new file mode 100644
index 0000000..0236a20
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/package-info.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Provides a set of tools to work with row sets when creating operator
+ * and "sub-operator" unit tests. A row set is a batch of Drill vectors,
+ * often called a "record batch." However, a record batch, in Drill, means
+ * not just the data, but also an operator on that data. The classes
+ * here work with the data itself, and can be used to test implementations
+ * of things such as code generated classes and so on.
+ * <p>
+ * The classes include tools for reading and writing row sets, comparing
+ * actual and expected results, and so on.
+ * <p>
+ * Drill defines a variety of record batch semantics, modeled here as
+ * distinct row set classes:
+ * <dl>
+ * <dt>RowSet</dt>
+ * <dd>The abstract definition of a row set that defines operations available
+ * on all row sets.</dd>
+ * <dt>SingleRowSet (abstract)</dt>
+ * <dd>Represents a row set that contains a single record batch (the typical
+ * case.</dd>
+ * <dt>DirectRowSet</dt>
+ * <dd>A read-only single row set without a selection vector.</dd>
+ * <dt>IndirectRowSet</dt>
+ * <dd>A read-only, single row set with an SV2. Note that the SV2 itself is
+ * writable (such as for sorting.)</dd>
+ * <dt>ExtendibleRowSet</dt>
+ * <dd>A write-only, single row set used to create a new row set. Because of
+ * the way Drill sets row counts, an extendible row set cannot be read; instead
+ * at the completion of the write the extendible row set becomes a direct or
+ * indirect row set.</dd>
+ * <dt>HyperRowSet</dt>
+ * <dd>A read-only row set made up of a collection of record batches, indexed via an
+ * SV4. As with the SV2, the SV4 itself is writable.</dt>
+ * </dl>
+ * This package contains a number of helper classes:
+ * <dl>
+ * <dt>RowSetWriter</dt>
+ * <dd>Writes data into an extendible row set.</dd>
+ * <dt>RowSetReader</dt>
+ * <dd>Reads data from any but an extendible row set.</dd>
+ * <dt>RowSetBuilder</dt>
+ * <dd>Creates and populates a row set in a fluent builder style.</dd>
+ * <dt>RowSetPrinter</dt>
+ * <dd>Prints a row set to stdout in a CSV-like form for easy debugging.</dd>
+ * <dt>RowSetComparision</dt>
+ * <dd>Used in tests to compare an "actual" row set against an "expected"
+ * row set. Does a complete check of row counts, types and values. If values
+ * are arrays (repeated), does a check of the entire array. Uses JUnit assertions
+ * to report comparison failures.</dd>
+ * <dt>SchemaBuilder</dt>
+ * <dd>Drill normally writes data to vectors, then "discovers" the row set schema based on the
+ * data written. For tests, it is usually far easier to simply declare a schema, then
+ * read and write data according to that schema. The schema builder provides a simple,
+ * fluent tool to create a row set schema. That schema then drives the row set readers
+ * and writers, the row set printer and the row set comparison.</dd>
+ * </dl>
+ */
+
+package org.apache.drill.test.rowSet;