You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ja...@apache.org on 2016/01/11 08:53:49 UTC

[01/12] drill git commit: DRILL-4241: initial commit

Repository: drill
Updated Branches:
  refs/heads/master f964908ae -> 392d1f7e9


DRILL-4241: initial commit


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

Branch: refs/heads/master
Commit: 5dfb451222b4259e274304bb360e2b000bcd26af
Parents: f964908
Author: Jacques Nadeau <ja...@gmail.com>
Authored: Wed Nov 18 18:45:56 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 22:46:16 2016 -0800

----------------------------------------------------------------------
 contrib/storage-kudu/.gitignore                 |  15 ++
 contrib/storage-kudu/README.md                  |   1 +
 contrib/storage-kudu/pom.xml                    |  87 ++++++++
 .../drill/exec/store/kudu/DrillKuduTable.java   |  36 ++++
 .../drill/exec/store/kudu/KuduGroupScan.java    | 203 +++++++++++++++++++
 .../drill/exec/store/kudu/KuduRecordReader.java | 178 ++++++++++++++++
 .../exec/store/kudu/KuduScanBatchCreator.java   |  57 ++++++
 .../drill/exec/store/kudu/KuduScanSpec.java     |  38 ++++
 .../exec/store/kudu/KuduSchemaFactory.java      |  95 +++++++++
 .../exec/store/kudu/KuduStoragePlugin.java      |  99 +++++++++
 .../store/kudu/KuduStoragePluginConfig.java     |  75 +++++++
 .../drill/exec/store/kudu/KuduSubScan.java      | 135 ++++++++++++
 .../resources/bootstrap-storage-plugins.json    |   9 +
 .../src/main/resources/drill-module.conf        |  24 +++
 .../drill/store/kudu/TestKuduConnect.java       |  76 +++++++
 .../apache/drill/store/kudu/TestKuduPlugin.java |  12 ++
 .../drill/store/kudu/TestKuduTableProvider.java |  91 +++++++++
 .../storage-kudu/src/test/resources/logback.xml |  64 ++++++
 18 files changed, 1295 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/.gitignore
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/.gitignore b/contrib/storage-kudu/.gitignore
new file mode 100644
index 0000000..f290bae
--- /dev/null
+++ b/contrib/storage-kudu/.gitignore
@@ -0,0 +1,15 @@
+.project
+.buildpath
+.classpath
+.checkstyle
+.settings/
+.idea/
+TAGS
+*.log
+*.lck
+*.iml
+target/
+*.DS_Store
+*.patch
+*~
+git.properties

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/README.md
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/README.md b/contrib/storage-kudu/README.md
new file mode 100644
index 0000000..f4597d4
--- /dev/null
+++ b/contrib/storage-kudu/README.md
@@ -0,0 +1 @@
+# drill-storage-kudu

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/pom.xml b/contrib/storage-kudu/pom.xml
new file mode 100644
index 0000000..7e57ca8
--- /dev/null
+++ b/contrib/storage-kudu/pom.xml
@@ -0,0 +1,87 @@
+<?xml version="1.0"?>
+<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor 
+  license agreements. See the NOTICE file distributed with this work for additional 
+  information regarding copyright ownership. The ASF licenses this file to 
+  You under the Apache License, Version 2.0 (the "License"); you may not use 
+  this file except in compliance with the License. You may obtain a copy of 
+  the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required 
+  by applicable law or agreed to in writing, software distributed under the 
+  License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 
+  OF ANY KIND, either express or implied. See the License for the specific 
+  language governing permissions and limitations under the License. -->
+<project
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+  xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>drill-contrib-parent</artifactId>
+    <groupId>org.apache.drill.contrib</groupId>
+    <version>1.3.0</version>
+  </parent>
+
+  <artifactId>drill-storage-kudu</artifactId>
+  <version>1.3.0-SNAPSHOT</version>
+
+  <name>contrib/kudu-storage-plugin</name>
+
+  <properties>
+    <drill.version>1.3.0</drill.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.drill.exec</groupId>
+      <artifactId>drill-java-exec</artifactId>
+      <version>${drill.version}</version>
+    </dependency>
+
+    <!-- Test dependencies -->
+    <dependency>
+      <groupId>org.apache.drill.exec</groupId>
+      <artifactId>drill-java-exec</artifactId>
+      <classifier>tests</classifier>
+      <version>${drill.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-common</artifactId>
+      <classifier>tests</classifier>
+      <version>${drill.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.kududb</groupId>
+      <artifactId>kudu-client</artifactId>
+      <version>0.5.0</version>
+    </dependency>
+
+  </dependencies>
+
+  <repositories>
+    <repository>
+      <id>drill-1016</id>
+      <url>https://repository.apache.org/content/repositories/orgapachedrill-1016/</url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+    <repository>
+      <id>cdh.repo</id>
+      <name>Cloudera Repositories</name>
+      <url>https://repository.cloudera.com/artifactory/cloudera-repos</url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <build>
+    <plugins>
+
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/DrillKuduTable.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/DrillKuduTable.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/DrillKuduTable.java
new file mode 100644
index 0000000..3b9c757
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/DrillKuduTable.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.kudu;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.drill.exec.planner.logical.DynamicDrillTable;
+
+public class DrillKuduTable extends DynamicDrillTable {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillKuduTable.class);
+
+  public DrillKuduTable(String storageEngineName, KuduStoragePlugin plugin, KuduScanSpec scanSpec) {
+    super(plugin, storageEngineName, scanSpec);
+  }
+
+  @Override
+  public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+    return super.getRowType(typeFactory);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
new file mode 100644
index 0000000..bc543d9
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.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.store.kudu;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.kudu.KuduSubScan.KuduSubScanSpec;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+@JsonTypeName("kudu-scan")
+public class KuduGroupScan extends AbstractGroupScan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduGroupScan.class);
+
+  private KuduStoragePluginConfig storagePluginConfig;
+  private List<SchemaPath> columns;
+  private KuduScanSpec kuduScanSpec;
+  private KuduStoragePlugin storagePlugin;
+  private boolean filterPushedDown = false;
+
+
+  @JsonCreator
+  public KuduGroupScan(@JsonProperty("kuduScanSpec") KuduScanSpec kuduScanSpec,
+                        @JsonProperty("storage") KuduStoragePluginConfig storagePluginConfig,
+                        @JsonProperty("columns") List<SchemaPath> columns,
+                        @JacksonInject StoragePluginRegistry pluginRegistry) throws IOException, ExecutionSetupException {
+    this((KuduStoragePlugin) pluginRegistry.getPlugin(storagePluginConfig), kuduScanSpec, columns);
+  }
+
+  public KuduGroupScan(KuduStoragePlugin storagePlugin, KuduScanSpec scanSpec,
+      List<SchemaPath> columns) {
+    super((String) null);
+    this.storagePlugin = storagePlugin;
+    this.storagePluginConfig = storagePlugin.getConfig();
+    this.kuduScanSpec = scanSpec;
+    this.columns = columns == null || columns.size() == 0? ALL_COLUMNS : columns;
+  }
+
+  /**
+   * Private constructor, used for cloning.
+   * @param that The KuduGroupScan to clone
+   */
+  private KuduGroupScan(KuduGroupScan that) {
+    super(that);
+    this.columns = that.columns;
+    this.kuduScanSpec = that.kuduScanSpec;
+    this.storagePlugin = that.storagePlugin;
+    this.storagePluginConfig = that.storagePluginConfig;
+    this.filterPushedDown = that.filterPushedDown;
+  }
+
+  @Override
+  public GroupScan clone(List<SchemaPath> columns) {
+    KuduGroupScan newScan = new KuduGroupScan(this);
+    newScan.columns = columns;
+    return newScan;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    return Collections.EMPTY_LIST;
+  }
+
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    return 1;
+  }
+
+
+  /**
+   *
+   * @param incomingEndpoints
+   */
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
+  }
+
+
+  @Override
+  public KuduSubScan getSpecificScan(int minorFragmentId) {
+    return new KuduSubScan(storagePlugin, storagePluginConfig,
+        ImmutableList.of(new KuduSubScanSpec(kuduScanSpec.getTableName())),
+        this.columns);
+  }
+
+  // KuduStoragePlugin plugin, KuduStoragePluginConfig config,
+  // List<KuduSubScanSpec> tabletInfoList, List<SchemaPath> columns
+  @Override
+  public ScanStats getScanStats() {
+    return ScanStats.TRIVIAL_TABLE;
+  }
+
+  @Override
+  @JsonIgnore
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    return new KuduGroupScan(this);
+  }
+
+  @JsonIgnore
+  public KuduStoragePlugin getStoragePlugin() {
+    return storagePlugin;
+  }
+
+  @JsonIgnore
+  public String getTableName() {
+    return getKuduScanSpec().getTableName();
+  }
+
+  @Override
+  public String getDigest() {
+    return toString();
+  }
+
+  @Override
+  public String toString() {
+    return "KuduGroupScan [KuduScanSpec="
+        + kuduScanSpec + ", columns="
+        + columns + "]";
+  }
+
+  @JsonProperty("storage")
+  public KuduStoragePluginConfig getStorageConfig() {
+    return this.storagePluginConfig;
+  }
+
+  @JsonProperty
+  public List<SchemaPath> getColumns() {
+    return columns;
+  }
+
+  @JsonProperty
+  public KuduScanSpec getKuduScanSpec() {
+    return kuduScanSpec;
+  }
+
+  @Override
+  @JsonIgnore
+  public boolean canPushdownProjects(List<SchemaPath> columns) {
+    return true;
+  }
+
+  @JsonIgnore
+  public void setFilterPushedDown(boolean b) {
+    this.filterPushedDown = true;
+  }
+
+  @JsonIgnore
+  public boolean isFilterPushedDown() {
+    return filterPushedDown;
+  }
+
+  /**
+   * Empty constructor, do not use, only for testing.
+   */
+  @VisibleForTesting
+  public KuduGroupScan() {
+    super((String)null);
+  }
+
+  /**
+   * Do not use, only for testing.
+   */
+  @VisibleForTesting
+  public void setKuduScanSpec(KuduScanSpec kuduScanSpec) {
+    this.kuduScanSpec = kuduScanSpec;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
new file mode 100644
index 0000000..0200527
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
@@ -0,0 +1,178 @@
+/**
+ * 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.store.kudu;
+
+import io.netty.buffer.DrillBuf;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.store.AbstractRecordReader;
+import org.apache.drill.exec.store.kudu.KuduSubScan.KuduSubScanSpec;
+import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
+import org.kududb.ColumnSchema;
+import org.kududb.Type;
+import org.kududb.client.KuduClient;
+import org.kududb.client.KuduScanner;
+import org.kududb.client.KuduTable;
+import org.kududb.client.RowResult;
+import org.kududb.client.RowResultIterator;
+import org.kududb.client.shaded.com.google.common.collect.ImmutableMap;
+
+public class KuduRecordReader extends AbstractRecordReader {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduRecordReader.class);
+
+  private static final int TARGET_RECORD_COUNT = 4000;
+
+  private final KuduClient client;
+  private final KuduSubScanSpec scanSpec;
+  private KuduTable table;
+  private VectorContainerWriter containerWriter;
+  private MapWriter writer;
+  private KuduScanner scanner;
+  private RowResultIterator iterator;
+  private DrillBuf buffer;
+
+  public KuduRecordReader(KuduClient client, KuduSubScan.KuduSubScanSpec subScanSpec,
+      List<SchemaPath> projectedColumns, FragmentContext context) {
+    setColumns(projectedColumns);
+    this.client = client;
+    buffer = context.getManagedBuffer();
+    scanSpec = subScanSpec;
+  }
+
+  @Override
+  public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
+    try {
+      KuduTable table = client.openTable(scanSpec.getTableName());
+      scanner = client.newScannerBuilder(table).build();
+      containerWriter = new VectorContainerWriter(output);
+      writer = containerWriter.rootAsMap();
+    } catch (Exception e) {
+      throw new ExecutionSetupException(e);
+    }
+  }
+
+  static final Map<Type, MajorType> TYPES;
+
+  static {
+    TYPES = ImmutableMap.<Type, MajorType> builder()
+        .put(Type.BINARY, Types.optional(MinorType.VARBINARY))
+        .put(Type.BOOL, Types.optional(MinorType.BIT))
+        .put(Type.DOUBLE, Types.optional(MinorType.FLOAT8))
+        .put(Type.FLOAT, Types.optional(MinorType.FLOAT4))
+        .put(Type.INT16, Types.optional(MinorType.INT))
+        .put(Type.INT32, Types.optional(MinorType.INT))
+        .put(Type.INT8, Types.optional(MinorType.INT))
+        .put(Type.INT64, Types.optional(MinorType.BIGINT))
+        .put(Type.STRING, Types.optional(MinorType.VARCHAR))
+        .put(Type.TIMESTAMP, Types.optional(MinorType.TIMESTAMP))
+        .build();
+  }
+
+  @Override
+  public int next() {
+    int rowCount = 0;
+    try {
+      while (iterator == null || !iterator.hasNext()) {
+        if (!scanner.hasMoreRows()) {
+          iterator = null;
+          break;
+        }
+        iterator = scanner.nextRows();
+
+        for (; rowCount < 4095 && iterator.hasNext(); rowCount++) {
+          writer.setPosition(rowCount);
+          RowResult result = iterator.next();
+          int i = 0;
+          for (ColumnSchema column : result.getColumnProjection().getColumns()) {
+            switch (column.getType()) {
+            case STRING: {
+              final ByteBuffer buf = result.getBinary(i);
+              final int length = buf.remaining();
+              ensure(length);
+              buffer.setBytes(0, buf);
+              writer.varChar(column.getName()).writeVarChar(0, length, buffer);
+              break;
+            }
+            case BINARY: {
+              final ByteBuffer buf = result.getBinary(i);
+              final int length = buf.remaining();
+              ensure(length);
+              buffer.setBytes(0, buf);
+              writer.varBinary(column.getName()).writeVarBinary(0, length, buffer);
+              break;
+            }
+            case INT8:
+              writer.integer(column.getName()).writeInt(result.getByte(i));
+              break;
+            case INT16:
+              writer.integer(column.getName()).writeInt(result.getShort(i));
+              break;
+            case INT32:
+              writer.integer(column.getName()).writeInt(result.getInt(i));
+              break;
+            case INT64:
+              writer.bigInt(column.getName()).writeBigInt(result.getLong(i));
+              break;
+            case FLOAT:
+              writer.float4(column.getName()).writeFloat4(result.getFloat(i));
+              break;
+            case DOUBLE:
+              writer.float8(column.getName()).writeFloat8(result.getDouble(i));
+              break;
+            case BOOL:
+              writer.bit(column.getName()).writeBit(result.getBoolean(i) ? 1 : 0);
+              break;
+            case TIMESTAMP:
+              writer.timeStamp(column.getName()).writeTimeStamp(result.getLong(i) / 1000);
+              break;
+            default:
+              throw new UnsupportedOperationException("unsupported type " + column.getType());
+            }
+
+            i++;
+          }
+        }
+      }
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+    containerWriter.setValueCount(rowCount);
+    return rowCount;
+  }
+
+  private void ensure(final int length) {
+    buffer = buffer.reallocIfNeeded(length);
+  }
+
+  @Override
+  public void close() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduScanBatchCreator.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduScanBatchCreator.java
new file mode 100644
index 0000000..b3c2c4e
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduScanBatchCreator.java
@@ -0,0 +1,57 @@
+/**
+ * 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.store.kudu;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.store.RecordReader;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class KuduScanBatchCreator implements BatchCreator<KuduSubScan>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduScanBatchCreator.class);
+
+  @Override
+  public ScanBatch getBatch(FragmentContext context, KuduSubScan subScan, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    Preconditions.checkArgument(children.isEmpty());
+    List<RecordReader> readers = Lists.newArrayList();
+    List<SchemaPath> columns = null;
+
+    for (KuduSubScan.KuduSubScanSpec scanSpec : subScan.getTabletScanSpecList()) {
+      try {
+        if ((columns = subScan.getColumns())==null) {
+          columns = GroupScan.ALL_COLUMNS;
+        }
+        readers.add(new KuduRecordReader(subScan.getStorageEngine().getClient(), scanSpec, columns, context));
+      } catch (Exception e1) {
+        throw new ExecutionSetupException(e1);
+      }
+    }
+    return new ScanBatch(subScan, context, readers.iterator());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduScanSpec.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduScanSpec.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduScanSpec.java
new file mode 100644
index 0000000..b669f79
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduScanSpec.java
@@ -0,0 +1,38 @@
+/**
+ * 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.store.kudu;
+
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class KuduScanSpec {
+
+  private final String tableName;
+
+  @JsonCreator
+  public KuduScanSpec(@JsonProperty("tableName") String tableName) {
+    this.tableName = tableName;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
new file mode 100644
index 0000000..294eabe
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
@@ -0,0 +1,95 @@
+/**
+ * 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.store.kudu;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.SchemaConfig;
+import org.apache.drill.exec.store.SchemaFactory;
+import org.kududb.client.ListTablesResponse;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Sets;
+
+public class KuduSchemaFactory implements SchemaFactory {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduSchemaFactory.class);
+
+  final String schemaName;
+  final KuduStoragePlugin plugin;
+
+  public KuduSchemaFactory(KuduStoragePlugin plugin, String name) throws IOException {
+    this.plugin = plugin;
+    this.schemaName = name;
+  }
+
+  @Override
+  public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
+    KuduTables schema = new KuduTables(schemaName);
+    SchemaPlus hPlus = parent.add(schemaName, schema);
+    schema.setHolder(hPlus);
+  }
+
+  class KuduTables extends AbstractSchema {
+
+    public KuduTables(String name) {
+      super(ImmutableList.<String>of(), name);
+    }
+
+    public void setHolder(SchemaPlus plusOfThis) {
+    }
+
+    @Override
+    public AbstractSchema getSubSchema(String name) {
+      return null;
+    }
+
+    @Override
+    public Set<String> getSubSchemaNames() {
+      return Collections.emptySet();
+    }
+
+    @Override
+    public Table getTable(String name) {
+      KuduScanSpec scanSpec = new KuduScanSpec(name);
+      return new DrillKuduTable(schemaName, plugin, scanSpec);
+    }
+
+    @Override
+    public Set<String> getTableNames() {
+      try {
+        ListTablesResponse tablesList = plugin.getClient().getTablesList();
+        return Sets.newHashSet(tablesList.getTablesList());
+      } catch (Exception e) {
+        logger.warn("Failure reading kudu tables.", e);
+        return Collections.EMPTY_SET;
+      }
+    }
+
+    @Override
+    public String getTypeName() {
+      return KuduStoragePluginConfig.NAME;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java
new file mode 100644
index 0000000..5e981b8
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java
@@ -0,0 +1,99 @@
+/**
+ * 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.store.kudu;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.exec.ops.OptimizerRulesContext;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.AbstractStoragePlugin;
+import org.apache.drill.exec.store.SchemaConfig;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.kududb.client.KuduClient;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+public class KuduStoragePlugin extends AbstractStoragePlugin {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduStoragePlugin.class);
+
+  private final DrillbitContext context;
+  private final KuduStoragePluginConfig engineConfig;
+  private final KuduSchemaFactory schemaFactory;
+
+  @SuppressWarnings("unused")
+  private final String name;
+  private final KuduClient client;
+
+  public KuduStoragePlugin(KuduStoragePluginConfig configuration, DrillbitContext context, String name)
+      throws IOException {
+    this.context = context;
+    this.schemaFactory = new KuduSchemaFactory(this, name);
+    this.engineConfig = configuration;
+    this.name = name;
+    this.client = new KuduClient.KuduClientBuilder(configuration.getMasterAddresses()).build();
+  }
+
+  @Override
+  public void start() throws IOException {
+
+  }
+
+  public KuduClient getClient() {
+    return client;
+  }
+
+  @Override
+  public void close() throws Exception {
+    client.close();
+  }
+
+  public DrillbitContext getContext() {
+    return this.context;
+  }
+
+  @Override
+  public boolean supportsRead() {
+    return true;
+  }
+
+  @Override
+  public KuduGroupScan getPhysicalScan(String userName, JSONOptions selection) throws IOException {
+    KuduScanSpec scanSpec = selection.getListWith(new ObjectMapper(), new TypeReference<KuduScanSpec>() {});
+    return new KuduGroupScan(this, scanSpec, null);
+  }
+
+  @Override
+  public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
+    schemaFactory.registerSchemas(schemaConfig, parent);
+  }
+
+  @Override
+  public KuduStoragePluginConfig getConfig() {
+    return engineConfig;
+  }
+
+  @Override
+  public Set<StoragePluginOptimizerRule> getOptimizerRules(OptimizerRulesContext optimizerRulesContext) {
+    return Collections.EMPTY_SET;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePluginConfig.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePluginConfig.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePluginConfig.java
new file mode 100644
index 0000000..e07f967
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePluginConfig.java
@@ -0,0 +1,75 @@
+/**
+ * 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.store.kudu;
+
+import org.apache.drill.common.logical.StoragePluginConfigBase;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName(KuduStoragePluginConfig.NAME)
+public class KuduStoragePluginConfig extends StoragePluginConfigBase {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduStoragePluginConfig.class);
+
+  public static final String NAME = "kudu";
+
+  private final String masterAddresses;
+
+  @JsonCreator
+  public KuduStoragePluginConfig(@JsonProperty("masterAddresses") String masterAddresses) {
+    this.masterAddresses = masterAddresses;
+  }
+
+  public String getMasterAddresses() {
+    return masterAddresses;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((masterAddresses == null) ? 0 : masterAddresses.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    KuduStoragePluginConfig other = (KuduStoragePluginConfig) obj;
+    if (masterAddresses == null) {
+      if (other.masterAddresses != null) {
+        return false;
+      }
+    } else if (!masterAddresses.equals(other.masterAddresses)) {
+      return false;
+    }
+    return true;
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSubScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSubScan.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSubScan.java
new file mode 100644
index 0000000..267ee77
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSubScan.java
@@ -0,0 +1,135 @@
+/**
+ * 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.store.kudu;
+
+import java.util.Iterator;
+import java.util.LinkedList;
+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.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+
+// Class containing information for reading a single Kudu tablet
+@JsonTypeName("kudu-tablet-scan")
+public class KuduSubScan extends AbstractBase implements SubScan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduSubScan.class);
+
+  @JsonProperty
+  public final KuduStoragePluginConfig storage;
+
+
+  private final KuduStoragePlugin kuduStoragePlugin;
+  private final List<KuduSubScanSpec> tabletScanSpecList;
+  private final List<SchemaPath> columns;
+
+  @JsonCreator
+  public KuduSubScan(@JacksonInject StoragePluginRegistry registry,
+                      @JsonProperty("storage") StoragePluginConfig storage,
+      @JsonProperty("tabletScanSpecList") LinkedList<KuduSubScanSpec> tabletScanSpecList,
+                      @JsonProperty("columns") List<SchemaPath> columns) throws ExecutionSetupException {
+    super((String) null);
+    kuduStoragePlugin = (KuduStoragePlugin) registry.getPlugin(storage);
+    this.tabletScanSpecList = tabletScanSpecList;
+    this.storage = (KuduStoragePluginConfig) storage;
+    this.columns = columns;
+  }
+
+  public KuduSubScan(KuduStoragePlugin plugin, KuduStoragePluginConfig config,
+      List<KuduSubScanSpec> tabletInfoList, List<SchemaPath> columns) {
+    super((String) null);
+    kuduStoragePlugin = plugin;
+    storage = config;
+    this.tabletScanSpecList = tabletInfoList;
+    this.columns = columns;
+  }
+
+  public List<KuduSubScanSpec> getTabletScanSpecList() {
+    return tabletScanSpecList;
+  }
+
+  @JsonIgnore
+  public KuduStoragePluginConfig getStorageConfig() {
+    return storage;
+  }
+
+  public List<SchemaPath> getColumns() {
+    return columns;
+  }
+
+  @Override
+  public boolean isExecutable() {
+    return false;
+  }
+
+  @JsonIgnore
+  public KuduStoragePlugin getStorageEngine(){
+    return kuduStoragePlugin;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitSubScan(this, value);
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    return new KuduSubScan(kuduStoragePlugin, storage, tabletScanSpecList, columns);
+  }
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  public static class KuduSubScanSpec {
+
+    private final String tableName;
+
+    @JsonCreator
+    public KuduSubScanSpec(@JsonProperty("tableName") String tableName) {
+      this.tableName = tableName;
+    }
+
+    public String getTableName() {
+      return tableName;
+    }
+
+  }
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.HBASE_SUB_SCAN_VALUE;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json
new file mode 100644
index 0000000..3ba12c0
--- /dev/null
+++ b/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json
@@ -0,0 +1,9 @@
+{
+  "storage":{
+    kudu : {
+      type:"kudu",
+      masterAddresses: "172.31.1.99",
+      enabled: true
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/resources/drill-module.conf b/contrib/storage-kudu/src/main/resources/drill-module.conf
new file mode 100644
index 0000000..e225600
--- /dev/null
+++ b/contrib/storage-kudu/src/main/resources/drill-module.conf
@@ -0,0 +1,24 @@
+// 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.
+//
+//  This file tells Drill to consider this module when class path scanning.
+//  This file can also include any supplementary configuration information.
+//  This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
+
+drill: {
+  classpath.scanning: {
+    packages += "org.apache.drill.exec.store.kudu"
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
new file mode 100644
index 0000000..a6fc69c
--- /dev/null
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
@@ -0,0 +1,76 @@
+package org.apache.drill.store.kudu;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+import org.kududb.ColumnSchema;
+import org.kududb.Schema;
+import org.kududb.Type;
+import org.kududb.client.Insert;
+import org.kududb.client.KuduClient;
+import org.kududb.client.KuduScanner;
+import org.kududb.client.KuduSession;
+import org.kududb.client.KuduTable;
+import org.kududb.client.ListTablesResponse;
+import org.kududb.client.PartialRow;
+import org.kududb.client.RowResult;
+import org.kududb.client.RowResultIterator;
+
+
+public class TestKuduConnect {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestKuduConnect.class);
+
+  public static final String KUDU_MASTER = "172.31.1.99";
+  public static final String KUDU_TABLE = "demo";
+
+  @Test
+  public void abc() throws Exception {
+
+    try (KuduClient client = new KuduClient.KuduClientBuilder(KUDU_MASTER).build()) {
+
+      ListTablesResponse tables = client.getTablesList(KUDU_TABLE);
+      if (!tables.getTablesList().isEmpty()) {
+        client.deleteTable(KUDU_TABLE);
+      }
+      ;
+
+      List<ColumnSchema> columns = new ArrayList<>(5);
+      columns.add(new ColumnSchema.ColumnSchemaBuilder("key", Type.INT32).key(true).build());
+      columns.add(new ColumnSchema.ColumnSchemaBuilder("binary", Type.BINARY).build());
+      columns.add(new ColumnSchema.ColumnSchemaBuilder("boolean", Type.BOOL).build());
+      columns.add(new ColumnSchema.ColumnSchemaBuilder("float", Type.FLOAT).build());
+      columns.add(new ColumnSchema.ColumnSchemaBuilder("string", Type.STRING).build());
+
+      Schema schema = new Schema(columns);
+      client.createTable(KUDU_TABLE, schema);
+
+      KuduTable table = client.openTable(KUDU_TABLE);
+
+      KuduSession session = client.newSession();
+      for (int i = 0; i < 3; i++) {
+        Insert insert = table.newInsert();
+        PartialRow row = insert.getRow();
+        row.addInt(0, i);
+        row.addBinary(1, ("Row " + i).getBytes());
+        row.addBoolean(2, i % 2 == 0);
+        row.addFloat(3, i + 0.01f);
+        row.addString(4, ("Row " + i));
+        session.apply(insert);
+      }
+
+      List<String> projectColumns = new ArrayList<>(1);
+      projectColumns.add("float");
+      KuduScanner scanner = client.newScannerBuilder(table)
+          .setProjectedColumnNames(projectColumns)
+          .build();
+      while (scanner.hasMoreRows()) {
+        RowResultIterator results = scanner.nextRows();
+        while (results.hasNext()) {
+          RowResult result = results.next();
+          System.out.println(result.toStringLongFormat());
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
new file mode 100644
index 0000000..1a6211f
--- /dev/null
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
@@ -0,0 +1,12 @@
+package org.apache.drill.store.kudu;
+
+import org.apache.drill.BaseTestQuery;
+import org.junit.Test;
+
+public class TestKuduPlugin extends BaseTestQuery {
+
+  @Test
+  public void testBasicQuery() throws Exception {
+    test("select * from kudu.demo;");
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java
new file mode 100644
index 0000000..ca8f9e1
--- /dev/null
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java
@@ -0,0 +1,91 @@
+/**
+ * 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.store.kudu;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.Map.Entry;
+
+import org.apache.drill.common.config.LogicalPlanPersistence;
+import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
+import org.apache.drill.exec.store.kudu.config.KuduPStoreProvider;
+import org.apache.drill.exec.store.sys.PStore;
+import org.apache.drill.exec.store.sys.PStoreConfig;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestKuduTableProvider extends BaseKuduTest {
+
+  private static KuduPStoreProvider provider;
+
+  @BeforeClass // mask Kudu cluster start function
+  public static void setUpBeforeTestKuduTableProvider() throws Exception {
+    provider = new KuduPStoreProvider(storagePluginConfig.getKuduConf(), "drill_store");
+    provider.start();
+  }
+
+  @Test
+  public void testTableProvider() throws IOException {
+    LogicalPlanPersistence lp = PhysicalPlanReaderTestFactory.defaultLogicalPlanPersistence(config);
+    PStore<String> kuduStore = provider.getStore(PStoreConfig.newJacksonBuilder(lp.getMapper(), String.class).name("kudu").build());
+    kuduStore.put("", "v0");
+    kuduStore.put("k1", "v1");
+    kuduStore.put("k2", "v2");
+    kuduStore.put("k3", "v3");
+    kuduStore.put("k4", "v4");
+    kuduStore.put("k5", "v5");
+    kuduStore.put(".test", "testValue");
+
+    assertEquals("v0", kuduStore.get(""));
+    assertEquals("testValue", kuduStore.get(".test"));
+
+    int rowCount = 0;
+    for (Entry<String, String> entry : kuduStore) {
+      rowCount++;
+      System.out.println(entry.getKey() + "=" + entry.getValue());
+    }
+    assertEquals(7, rowCount);
+
+    PStore<String> kuduTestStore = provider.getStore(PStoreConfig.newJacksonBuilder(lp.getMapper(), String.class).name("kudu.test").build());
+    kuduTestStore.put("", "v0");
+    kuduTestStore.put("k1", "v1");
+    kuduTestStore.put("k2", "v2");
+    kuduTestStore.put("k3", "v3");
+    kuduTestStore.put("k4", "v4");
+    kuduTestStore.put(".test", "testValue");
+
+    assertEquals("v0", kuduStore.get(""));
+    assertEquals("testValue", kuduStore.get(".test"));
+
+    rowCount = 0;
+    for (Entry<String, String> entry : kuduTestStore) {
+      rowCount++;
+      System.out.println(entry.getKey() + "=" + entry.getValue());
+    }
+    assertEquals(6, rowCount);
+  }
+
+  @AfterClass
+  public static void tearDownTestKuduTableProvider() {
+    if (provider != null) {
+      provider.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5dfb4512/contrib/storage-kudu/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/resources/logback.xml b/contrib/storage-kudu/src/test/resources/logback.xml
new file mode 100644
index 0000000..6ef172b
--- /dev/null
+++ b/contrib/storage-kudu/src/test/resources/logback.xml
@@ -0,0 +1,64 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<configuration>
+  <timestamp key="bySecond" datePattern="yyyyMMdd'T'HHmmss"/>
+
+  <appender name="SOCKET"
+    class="de.huxhorn.lilith.logback.appender.ClassicMultiplexSocketAppender">
+    <Compressing>true</Compressing>
+    <ReconnectionDelay>10000</ReconnectionDelay>
+    <IncludeCallerData>true</IncludeCallerData>
+    <RemoteHosts>${LILITH_HOSTNAME:-localhost}</RemoteHosts>
+  </appender>
+
+  <appender name="FILE" class="ch.qos.logback.core.FileAppender">
+    <!-- The property 'logback.log.dir' is defined in pom.xml -->
+    <file>${logback.log.dir:-./target/surefire-reports}/hbase-tests-${bySecond}.log</file>
+    <append>false</append>
+    <encoder>
+      <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n</pattern>
+    </encoder>
+   </appender>
+
+  <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <encoder>
+      <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n</pattern>
+    </encoder>
+  </appender>
+
+  <logger name="org.apache.drill" additivity="false">
+    <level value="info" />
+    <appender-ref ref="FILE" />
+  </logger>
+
+  <logger name="org.apache.drill" additivity="false">
+    <level value="debug" />
+    <appender-ref ref="SOCKET" />
+  </logger>
+
+  <logger name="org.apache.hadoop" additivity="false">
+    <level value="info" />
+    <appender-ref ref="FILE" />
+  </logger>
+
+  <root>
+    <level value="error" />
+    <appender-ref ref="STDOUT" />
+  </root>
+
+</configuration>


[05/12] drill git commit: DRILL-4241: Improve record reader and type mappings

Posted by ja...@apache.org.
DRILL-4241: Improve record reader and type mappings


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

Branch: refs/heads/master
Commit: b1d9d791c5fac028ab4cb54870dc0f7e2d6f5177
Parents: 697579b
Author: Todd Lipcon <to...@cloudera.com>
Authored: Thu Nov 19 15:00:00 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 22:54:21 2016 -0800

----------------------------------------------------------------------
 .../drill/exec/store/kudu/KuduRecordReader.java | 319 +++++++++++++++----
 1 file changed, 250 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/b1d9d791/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
index 8247e01..17427f1 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
@@ -17,33 +17,45 @@
  */
 package org.apache.drill.exec.store.kudu;
 
-import io.netty.buffer.DrillBuf;
-
 import java.nio.ByteBuffer;
 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;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.store.AbstractRecordReader;
 import org.apache.drill.exec.store.kudu.KuduSubScan.KuduSubScanSpec;
-import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
-import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableBitVector;
+import org.apache.drill.exec.vector.NullableFloat4Vector;
+import org.apache.drill.exec.vector.NullableFloat8Vector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarBinaryVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
 import org.kududb.ColumnSchema;
+import org.kududb.Schema;
 import org.kududb.Type;
 import org.kududb.client.KuduClient;
 import org.kududb.client.KuduScanner;
+import org.kududb.client.KuduScanner.KuduScannerBuilder;
 import org.kududb.client.KuduTable;
 import org.kududb.client.RowResult;
 import org.kududb.client.RowResultIterator;
 import org.kududb.client.shaded.com.google.common.collect.ImmutableMap;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 public class KuduRecordReader extends AbstractRecordReader {
@@ -53,32 +65,36 @@ public class KuduRecordReader extends AbstractRecordReader {
 
   private final KuduClient client;
   private final KuduSubScanSpec scanSpec;
-  private VectorContainerWriter containerWriter;
-  private MapWriter writer;
   private KuduScanner scanner;
   private RowResultIterator iterator;
-  private DrillBuf buffer;
+  
+  
+  private OutputMutator output;
+  private ImmutableList<ValueVector> vectors;
+  private ImmutableList<Copier<?>> copiers;
 
   public KuduRecordReader(KuduClient client, KuduSubScan.KuduSubScanSpec subScanSpec,
       List<SchemaPath> projectedColumns, FragmentContext context) {
     setColumns(projectedColumns);
     this.client = client;
-    buffer = context.getManagedBuffer();
     scanSpec = subScanSpec;
   }
 
   @Override
   public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
+    this.output = output;
     try {
       KuduTable table = client.openTable(scanSpec.getTableName());
-      List<String> colNames = Lists.newArrayList();
-      for (SchemaPath p : this.getColumns()) {
-        colNames.add(p.getAsUnescapedPath());
+      
+      KuduScannerBuilder builder = client.newScannerBuilder(table);
+      if (!isStarQuery()) {
+        List<String> colNames = Lists.newArrayList();
+        for (SchemaPath p : this.getColumns()) {
+          colNames.add(p.getAsUnescapedPath());
+        }
+        builder.setProjectedColumnNames(colNames);
       }
-      scanner = client.newScannerBuilder(table)
-          .setProjectedColumnNames(colNames).build();
-      containerWriter = new VectorContainerWriter(output);
-      writer = containerWriter.rootAsMap();
+      scanner = builder.build();
     } catch (Exception e) {
       throw new ExecutionSetupException(e);
     }
@@ -92,9 +108,9 @@ public class KuduRecordReader extends AbstractRecordReader {
         .put(Type.BOOL, Types.optional(MinorType.BIT))
         .put(Type.DOUBLE, Types.optional(MinorType.FLOAT8))
         .put(Type.FLOAT, Types.optional(MinorType.FLOAT4))
-        .put(Type.INT16, Types.optional(MinorType.INT))
+        .put(Type.INT8, Types.optional(MinorType.TINYINT))
+        .put(Type.INT16, Types.optional(MinorType.SMALLINT))
         .put(Type.INT32, Types.optional(MinorType.INT))
-        .put(Type.INT8, Types.optional(MinorType.INT))
         .put(Type.INT64, Types.optional(MinorType.BIGINT))
         .put(Type.STRING, Types.optional(MinorType.VARCHAR))
         .put(Type.TIMESTAMP, Types.optional(MinorType.TIMESTAMP))
@@ -113,74 +129,239 @@ public class KuduRecordReader extends AbstractRecordReader {
         iterator = scanner.nextRows();
       }
       for (; rowCount < TARGET_RECORD_COUNT && iterator.hasNext(); rowCount++) {
-        writer.setPosition(rowCount);
-        addRowResult(iterator.next());
+        addRowResult(iterator.next(), rowCount);
       }
     } catch (Exception ex) {
       throw new RuntimeException(ex);
     }
-    containerWriter.setValueCount(rowCount);
+    for (ValueVector vv : vectors) {
+      vv.getMutator().setValueCount(rowCount);
+    }
     return rowCount;
   }
+  
+  @SuppressWarnings("unchecked")
+  private void initCopiers(Schema schema) throws SchemaChangeException {
+    ImmutableList.Builder<ValueVector> vectorBuilder = ImmutableList.builder();
+    ImmutableList.Builder<Copier<?>> copierBuilder = ImmutableList.builder();
+    
+    for (int i = 0; i < schema.getColumnCount(); i++) {
+      ColumnSchema col = schema.getColumnByIndex(i);  
+      
+      final String name = col.getName();
+      final Type kuduType = col.getType();
+      MajorType majorType = TYPES.get(kuduType);
+      if (majorType == null) {
+        logger.warn("Ignoring column that is unsupported.", UserException
+            .unsupportedError()
+            .message(
+                "A column you queried has a data type that is not currently supported by the JDBC storage plugin. "
+                    + "The column's name was %s and its Kudu data type was %s. ",
+                name, kuduType.toString())
+            .addContext("column Name", name)
+            .addContext("plugin", "kudu")
+            .build(logger));
+
+        continue;
+      }
+      MinorType minorType = majorType.getMinorType();
+      MaterializedField field = MaterializedField.create(name, majorType);
+      final Class<? extends ValueVector> clazz = (Class<? extends ValueVector>) TypeHelper.getValueVectorClass(
+          minorType, majorType.getMode());
+      ValueVector vector = output.addField(field, clazz);
+      vector.allocateNew();
+      vectorBuilder.add(vector);
+      copierBuilder.add(getCopier(kuduType, i, vector));
+    }
+
+    vectors = vectorBuilder.build();
+    copiers = copierBuilder.build();
+  }
+
+  private void addRowResult(RowResult result, int rowIndex) throws SchemaChangeException {
+    if (copiers == null) {
+      initCopiers(result.getColumnProjection());
+    }
+    
+    for (Copier<?> c : copiers) {
+      c.copy(result, rowIndex);
+    }
+  }
+
+
+  @Override
+  public void close() {
+  }
+  
+
+  private Copier<?> getCopier(Type kuduType, int offset, ValueVector v) {
+
+    if (v instanceof NullableBigIntVector) {
+      return new BigIntCopier(offset, (NullableBigIntVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableFloat4Vector) {
+      return new Float4Copier(offset, (NullableFloat4Vector.Mutator) v.getMutator());
+    } else if (v instanceof NullableFloat8Vector) {
+      return new Float8Copier(offset, (NullableFloat8Vector.Mutator) v.getMutator());
+    } else if (v instanceof NullableIntVector) {
+      return new IntCopier(offset, (NullableIntVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableVarCharVector) {
+      return new VarCharCopier(offset, (NullableVarCharVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableVarBinaryVector) {
+      return new VarBinaryCopier(offset, (NullableVarBinaryVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableTimeStampVector) {
+      return new TimeStampCopier(offset, (NullableTimeStampVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableBitVector) {
+      return new BitCopier(offset, (NullableBitVector.Mutator) v.getMutator());
+    }
+
+    throw new IllegalArgumentException("Unknown how to handle vector.");
+  }
+  
+  private abstract class Copier<T extends ValueVector.Mutator> {
+    protected final int columnIndex;
+    protected final T mutator;
+
+    public Copier(int columnIndex, T mutator) {
+      this.columnIndex = columnIndex;
+      this.mutator = mutator;
+    }
+
+    abstract void copy(RowResult result, int index);
+  }
+
+  private class IntCopier extends Copier<NullableIntVector.Mutator> {
+    public IntCopier(int offset, NullableIntVector.Mutator mutator) {
+      super(offset, mutator);
+    }
+
+    @Override
+    void copy(RowResult result, int index) {
+      if (result.isNull(columnIndex)) {
+        mutator.setNull(index);
+      } else {
+        mutator.setSafe(index, result.getInt(columnIndex));
+      }
+    }
+  }
+
+  private class BigIntCopier extends Copier<NullableBigIntVector.Mutator> {
+    public BigIntCopier(int offset, NullableBigIntVector.Mutator mutator) {
+      super(offset, mutator);
+    }
+
+    @Override
+    void copy(RowResult result, int index) {
+      if (result.isNull(columnIndex)) {
+        mutator.setNull(index);
+      } else {
+        mutator.setSafe(index, result.getLong(columnIndex));
+      }
+    }
+  }
+
+  private class Float4Copier extends Copier<NullableFloat4Vector.Mutator> {
+
+    public Float4Copier(int columnIndex, NullableFloat4Vector.Mutator mutator) {
+      super(columnIndex, mutator);
+    }
 
-  private void addRowResult(RowResult result) {
-    int i = 0;
-    for (ColumnSchema column : result.getColumnProjection().getColumns()) {
-      switch (column.getType()) {
-      case STRING: {
-        final ByteBuffer buf = result.getBinary(i);
-        final int length = buf.remaining();
-        ensure(length);
-        buffer.setBytes(0, buf);
-        writer.varChar(column.getName()).writeVarChar(0, length, buffer);
-        break;
+    @Override
+    void copy(RowResult result, int index) {
+      if (result.isNull(columnIndex)) {
+        mutator.setNull(index);
+      } else {
+        mutator.setSafe(index, result.getFloat(columnIndex));
       }
-      case BINARY: {
-        final ByteBuffer buf = result.getBinary(i);
-        final int length = buf.remaining();
-        ensure(length);
-        buffer.setBytes(0, buf);
-        writer.varBinary(column.getName()).writeVarBinary(0, length, buffer);
-        break;
+    }
+
+  }
+
+
+  private class Float8Copier extends Copier<NullableFloat8Vector.Mutator> {
+
+    public Float8Copier(int columnIndex, NullableFloat8Vector.Mutator mutator) {
+      super(columnIndex, mutator);
+    }
+
+    @Override
+    void copy(RowResult result, int index) {
+      if (result.isNull(columnIndex)) {
+        mutator.setNull(index);
+      } else {
+        mutator.setSafe(index, result.getDouble(columnIndex));
       }
-      case INT8:
-        writer.integer(column.getName()).writeInt(result.getByte(i));
-        break;
-      case INT16:
-        writer.integer(column.getName()).writeInt(result.getShort(i));
-        break;
-      case INT32:
-        writer.integer(column.getName()).writeInt(result.getInt(i));
-        break;
-      case INT64:
-        writer.bigInt(column.getName()).writeBigInt(result.getLong(i));
-        break;
-      case FLOAT:
-        writer.float4(column.getName()).writeFloat4(result.getFloat(i));
-        break;
-      case DOUBLE:
-        writer.float8(column.getName()).writeFloat8(result.getDouble(i));
-        break;
-      case BOOL:
-        writer.bit(column.getName()).writeBit(result.getBoolean(i) ? 1 : 0);
-        break;
-      case TIMESTAMP:
-        writer.timeStamp(column.getName()).writeTimeStamp(result.getLong(i) / 1000);
-        break;
-      default:
-        throw new UnsupportedOperationException("unsupported type " + column.getType());
+    }
+  }
+
+  // TODO: decimal copier
+  
+  private class VarCharCopier extends Copier<NullableVarCharVector.Mutator> {
+
+    public VarCharCopier(int columnIndex, NullableVarCharVector.Mutator mutator) {
+      super(columnIndex, mutator);
+    }
+
+    @Override
+    void copy(RowResult result, int index) {
+      if (result.isNull(columnIndex)) {
+        mutator.setNull(index);
+      } else {
+        ByteBuffer value = result.getBinary(columnIndex);
+        mutator.setSafe(index, value, 0, value.remaining());
       }
+    }
+  }
 
-      i++;
+  private class VarBinaryCopier extends Copier<NullableVarBinaryVector.Mutator> {
+
+    public VarBinaryCopier(int columnIndex, NullableVarBinaryVector.Mutator mutator) {
+      super(columnIndex, mutator);
+    }
+
+    @Override
+    void copy(RowResult result, int index) {
+      if (result.isNull(columnIndex)) {
+        mutator.setNull(index);
+      } else {
+        ByteBuffer value = result.getBinary(columnIndex);
+        mutator.setSafe(index, value, 0, value.remaining());
+      }
     }
   }
 
-  private void ensure(final int length) {
-    buffer = buffer.reallocIfNeeded(length);
+  // TODO: DateCopier
+  // TODO: TimeCopier
+  
+  private class TimeStampCopier extends Copier<NullableTimeStampVector.Mutator> {
+
+    public TimeStampCopier(int columnIndex, NullableTimeStampVector.Mutator mutator) {
+      super(columnIndex, mutator);
+    }
+
+    @Override
+    void copy(RowResult result, int index) {
+      if (result.isNull(columnIndex)) {
+        mutator.setNull(index);
+      } else {
+        long ts = result.getLong(columnIndex);
+        mutator.setSafe(index, ts / 1000);
+      }
+    }
   }
 
-  @Override
-  public void close() {
+  private class BitCopier extends Copier<NullableBitVector.Mutator> {
+    public BitCopier(int columnIndex, NullableBitVector.Mutator mutator) {
+      super(columnIndex, mutator);
+    }
+
+    @Override
+    void copy(RowResult result, int index) {
+      if (result.isNull(columnIndex)) {
+        mutator.setNull(index);
+      } else {
+        mutator.setSafe(index, result.getBoolean(columnIndex) ? 1 : 0);
+      }
+    }
   }
 
 }


[09/12] drill git commit: DRILL-4241: Create table with multiple tablets in kudu test.

Posted by ja...@apache.org.
DRILL-4241: Create table with multiple tablets in kudu test.


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

Branch: refs/heads/master
Commit: 6a406aa18481560b47716496307d9a6f59a7586b
Parents: 4f3f401
Author: Amit Hadke <am...@gmail.com>
Authored: Thu Nov 19 16:16:17 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 22:54:21 2016 -0800

----------------------------------------------------------------------
 .../drill/store/kudu/TestKuduConnect.java       | 39 +++++++++++++++-----
 1 file changed, 30 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/6a406aa1/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
index 7fac9ad..5f36f80 100644
--- a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
@@ -24,6 +24,7 @@ import org.junit.Test;
 import org.kududb.ColumnSchema;
 import org.kududb.Schema;
 import org.kududb.Type;
+import org.kududb.client.CreateTableBuilder;
 import org.kududb.client.Insert;
 import org.kududb.client.KuduClient;
 import org.kududb.client.KuduScanner;
@@ -33,24 +34,24 @@ import org.kududb.client.ListTablesResponse;
 import org.kududb.client.PartialRow;
 import org.kududb.client.RowResult;
 import org.kududb.client.RowResultIterator;
+import org.kududb.client.SessionConfiguration;
+
+import static org.kududb.Type.STRING;
 
 
 public class TestKuduConnect {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestKuduConnect.class);
 
   public static final String KUDU_MASTER = "172.31.1.99";
-  public static final String KUDU_TABLE = "demo";
 
-  @Test
-  public void abc() throws Exception {
+  public static void createKuduTable(String tableName, int tablets, int replicas, int rows) throws Exception {
 
     try (KuduClient client = new KuduClient.KuduClientBuilder(KUDU_MASTER).build()) {
 
-      ListTablesResponse tables = client.getTablesList(KUDU_TABLE);
+      ListTablesResponse tables = client.getTablesList(tableName);
       if (!tables.getTablesList().isEmpty()) {
-        client.deleteTable(KUDU_TABLE);
+        client.deleteTable(tableName);
       }
-      ;
 
       List<ColumnSchema> columns = new ArrayList<>(5);
       columns.add(new ColumnSchema.ColumnSchemaBuilder("key", Type.INT32).key(true).build());
@@ -60,12 +61,22 @@ public class TestKuduConnect {
       columns.add(new ColumnSchema.ColumnSchemaBuilder("string", Type.STRING).nullable(true).build());
 
       Schema schema = new Schema(columns);
-      client.createTable(KUDU_TABLE, schema);
 
-      KuduTable table = client.openTable(KUDU_TABLE);
+      CreateTableBuilder builder = new CreateTableBuilder();
+      builder.setNumReplicas(replicas);
+      for (int i = 1; i < tablets; i++) {
+        PartialRow splitRow = schema.newPartialRow();
+        splitRow.addInt("key", i*1000);
+        builder.addSplitRow(splitRow);
+      }
+
+      client.createTable(tableName, schema, builder);
+
+      KuduTable table = client.openTable(tableName);
 
       KuduSession session = client.newSession();
-      for (int i = 0; i < 3; i++) {
+      session.setFlushMode(SessionConfiguration.FlushMode.AUTO_FLUSH_SYNC);
+      for (int i = 0; i < rows; i++) {
         Insert insert = table.newInsert();
         PartialRow row = insert.getRow();
         row.addInt(0, i);
@@ -90,4 +101,14 @@ public class TestKuduConnect {
       }
     }
   }
+
+  @Test
+  public void abc() throws Exception {
+    createKuduTable("demo", 1, 1, 3);
+  }
+
+  @Test
+  public void def() throws Exception {
+    createKuduTable("demo-large-splits", 6, /* replicas */ 1, 6000);
+  }
 }


[03/12] drill git commit: DRILL-4241: Various Type Fixes

Posted by ja...@apache.org.
DRILL-4241: Various Type Fixes


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

Branch: refs/heads/master
Commit: 8aac12b41dac299b91f3ada2e3742bc36f3454a6
Parents: 805e041
Author: Todd Lipcon <to...@cloudera.com>
Authored: Sun Jan 10 22:50:30 2016 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 22:54:03 2016 -0800

----------------------------------------------------------------------
 .../drill/exec/store/kudu/KuduRecordReader.java | 112 ++++++++++---------
 1 file changed, 57 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/8aac12b4/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
index 0200527..fefb649 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
@@ -102,63 +102,13 @@ public class KuduRecordReader extends AbstractRecordReader {
       while (iterator == null || !iterator.hasNext()) {
         if (!scanner.hasMoreRows()) {
           iterator = null;
-          break;
+          return 0;
         }
         iterator = scanner.nextRows();
-
-        for (; rowCount < 4095 && iterator.hasNext(); rowCount++) {
-          writer.setPosition(rowCount);
-          RowResult result = iterator.next();
-          int i = 0;
-          for (ColumnSchema column : result.getColumnProjection().getColumns()) {
-            switch (column.getType()) {
-            case STRING: {
-              final ByteBuffer buf = result.getBinary(i);
-              final int length = buf.remaining();
-              ensure(length);
-              buffer.setBytes(0, buf);
-              writer.varChar(column.getName()).writeVarChar(0, length, buffer);
-              break;
-            }
-            case BINARY: {
-              final ByteBuffer buf = result.getBinary(i);
-              final int length = buf.remaining();
-              ensure(length);
-              buffer.setBytes(0, buf);
-              writer.varBinary(column.getName()).writeVarBinary(0, length, buffer);
-              break;
-            }
-            case INT8:
-              writer.integer(column.getName()).writeInt(result.getByte(i));
-              break;
-            case INT16:
-              writer.integer(column.getName()).writeInt(result.getShort(i));
-              break;
-            case INT32:
-              writer.integer(column.getName()).writeInt(result.getInt(i));
-              break;
-            case INT64:
-              writer.bigInt(column.getName()).writeBigInt(result.getLong(i));
-              break;
-            case FLOAT:
-              writer.float4(column.getName()).writeFloat4(result.getFloat(i));
-              break;
-            case DOUBLE:
-              writer.float8(column.getName()).writeFloat8(result.getDouble(i));
-              break;
-            case BOOL:
-              writer.bit(column.getName()).writeBit(result.getBoolean(i) ? 1 : 0);
-              break;
-            case TIMESTAMP:
-              writer.timeStamp(column.getName()).writeTimeStamp(result.getLong(i) / 1000);
-              break;
-            default:
-              throw new UnsupportedOperationException("unsupported type " + column.getType());
-            }
-
-            i++;
-          }
-        }
+      }
+      for (; rowCount < 4095 && iterator.hasNext(); rowCount++) {
+        writer.setPosition(rowCount);
+        addRowResult(iterator.next());
       }
     } catch (Exception ex) {
       throw new RuntimeException(ex);
@@ -167,6 +117,58 @@ public class KuduRecordReader extends AbstractRecordReader {
     return rowCount;
   }
 
+  private void addRowResult(RowResult result) {
+    int i = 0;
+    for (ColumnSchema column : result.getColumnProjection().getColumns()) {
+      switch (column.getType()) {
+      case STRING: {
+        final ByteBuffer buf = result.getBinary(i);
+        final int length = buf.remaining();
+        ensure(length);
+        buffer.setBytes(0, buf);
+        writer.varChar(column.getName()).writeVarChar(0, length, buffer);
+        break;
+      }
+      case BINARY: {
+        final ByteBuffer buf = result.getBinary(i);
+        final int length = buf.remaining();
+        ensure(length);
+        buffer.setBytes(0, buf);
+        writer.varBinary(column.getName()).writeVarBinary(0, length, buffer);
+        break;
+      }
+      case INT8:
+        writer.integer(column.getName()).writeInt(result.getByte(i));
+        break;
+      case INT16:
+        writer.integer(column.getName()).writeInt(result.getShort(i));
+        break;
+      case INT32:
+        writer.integer(column.getName()).writeInt(result.getInt(i));
+        break;
+      case INT64:
+        writer.bigInt(column.getName()).writeBigInt(result.getLong(i));
+        break;
+      case FLOAT:
+        writer.float4(column.getName()).writeFloat4(result.getFloat(i));
+        break;
+      case DOUBLE:
+        writer.float8(column.getName()).writeFloat8(result.getDouble(i));
+        break;
+      case BOOL:
+        writer.bit(column.getName()).writeBit(result.getBoolean(i) ? 1 : 0);
+        break;
+      case TIMESTAMP:
+        writer.timeStamp(column.getName()).writeTimeStamp(result.getLong(i) / 1000);
+        break;
+      default:
+        throw new UnsupportedOperationException("unsupported type " + column.getType());
+      }
+
+      i++;
+    }
+  }
+
   private void ensure(final int length) {
     buffer = buffer.reallocIfNeeded(length);
   }


[10/12] drill git commit: DRILL-4241: Rewrite RecordReader to support NULLs and be less Java-like

Posted by ja...@apache.org.
DRILL-4241: Rewrite RecordReader to support NULLs and be less Java-like


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

Branch: refs/heads/master
Commit: 4ba515539e1715e08c953d56d5648578d4a8ba29
Parents: 6a406aa
Author: Todd Lipcon <to...@cloudera.com>
Authored: Thu Nov 19 16:15:36 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 22:54:21 2016 -0800

----------------------------------------------------------------------
 .../drill/exec/store/kudu/KuduRecordReader.java | 353 ++++++++-----------
 1 file changed, 152 insertions(+), 201 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4ba51553/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
index adbdb83..6b7877d 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
@@ -35,6 +35,11 @@ import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.store.AbstractRecordReader;
 import org.apache.drill.exec.store.kudu.KuduSubScan.KuduSubScanSpec;
+import org.apache.drill.exec.vector.BigIntVector;
+import org.apache.drill.exec.vector.BitVector;
+import org.apache.drill.exec.vector.Float4Vector;
+import org.apache.drill.exec.vector.Float8Vector;
+import org.apache.drill.exec.vector.IntVector;
 import org.apache.drill.exec.vector.NullableBigIntVector;
 import org.apache.drill.exec.vector.NullableBitVector;
 import org.apache.drill.exec.vector.NullableFloat4Vector;
@@ -43,7 +48,10 @@ import org.apache.drill.exec.vector.NullableIntVector;
 import org.apache.drill.exec.vector.NullableTimeStampVector;
 import org.apache.drill.exec.vector.NullableVarBinaryVector;
 import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.TimeStampVector;
 import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarBinaryVector;
+import org.apache.drill.exec.vector.VarCharVector;
 import org.kududb.ColumnSchema;
 import org.kududb.Schema;
 import org.kududb.Type;
@@ -68,10 +76,15 @@ public class KuduRecordReader extends AbstractRecordReader {
   private KuduScanner scanner;
   private RowResultIterator iterator;
   
-  
   private OutputMutator output;
-  private ImmutableList<ValueVector> vectors;
-  private ImmutableList<Copier<?>> copiers;
+
+  private static class ProjectedColumnInfo {
+    int index;
+    ValueVector vv;
+    ColumnSchema kuduColumn;
+  }
+  private ImmutableList<ProjectedColumnInfo> projectedCols;
+
 
   public KuduRecordReader(KuduClient client, KuduSubScan.KuduSubScanSpec subScanSpec,
       List<SchemaPath> projectedColumns, FragmentContext context) {
@@ -104,20 +117,20 @@ public class KuduRecordReader extends AbstractRecordReader {
     }
   }
 
-  static final Map<Type, MajorType> TYPES;
+  static final Map<Type,MinorType> TYPES;
 
   static {
-    TYPES = ImmutableMap.<Type, MajorType> builder()
-        .put(Type.BINARY, Types.optional(MinorType.VARBINARY))
-        .put(Type.BOOL, Types.optional(MinorType.BIT))
-        .put(Type.DOUBLE, Types.optional(MinorType.FLOAT8))
-        .put(Type.FLOAT, Types.optional(MinorType.FLOAT4))
-        .put(Type.INT8, Types.optional(MinorType.TINYINT))
-        .put(Type.INT16, Types.optional(MinorType.SMALLINT))
-        .put(Type.INT32, Types.optional(MinorType.INT))
-        .put(Type.INT64, Types.optional(MinorType.BIGINT))
-        .put(Type.STRING, Types.optional(MinorType.VARCHAR))
-        .put(Type.TIMESTAMP, Types.optional(MinorType.TIMESTAMP))
+    TYPES = ImmutableMap.<Type, MinorType> builder()
+        .put(Type.BINARY, MinorType.VARBINARY)
+        .put(Type.BOOL, MinorType.BIT)
+        .put(Type.DOUBLE, MinorType.FLOAT8)
+        .put(Type.FLOAT, MinorType.FLOAT4)
+        .put(Type.INT8, MinorType.INT)
+        .put(Type.INT16, MinorType.INT)
+        .put(Type.INT32, MinorType.INT)
+        .put(Type.INT64, MinorType.BIGINT)
+        .put(Type.STRING, MinorType.VARCHAR)
+        .put(Type.TIMESTAMP, MinorType.TIMESTAMP)
         .build();
   }
 
@@ -138,24 +151,23 @@ public class KuduRecordReader extends AbstractRecordReader {
     } catch (Exception ex) {
       throw new RuntimeException(ex);
     }
-    for (ValueVector vv : vectors) {
-      vv.getMutator().setValueCount(rowCount);
+    for (ProjectedColumnInfo pci : projectedCols) {
+      pci.vv.getMutator().setValueCount(rowCount);
     }
     return rowCount;
   }
   
   @SuppressWarnings("unchecked")
-  private void initCopiers(Schema schema) throws SchemaChangeException {
-    ImmutableList.Builder<ValueVector> vectorBuilder = ImmutableList.builder();
-    ImmutableList.Builder<Copier<?>> copierBuilder = ImmutableList.builder();
+  private void initCols(Schema schema) throws SchemaChangeException {
+    ImmutableList.Builder<ProjectedColumnInfo> pciBuilder = ImmutableList.builder();
     
     for (int i = 0; i < schema.getColumnCount(); i++) {
       ColumnSchema col = schema.getColumnByIndex(i);  
       
       final String name = col.getName();
       final Type kuduType = col.getType();
-      MajorType majorType = TYPES.get(kuduType);
-      if (majorType == null) {
+      MinorType minorType = TYPES.get(kuduType);
+      if (minorType == null) {
         logger.warn("Ignoring column that is unsupported.", UserException
             .unsupportedError()
             .message(
@@ -168,204 +180,143 @@ public class KuduRecordReader extends AbstractRecordReader {
 
         continue;
       }
-      MinorType minorType = majorType.getMinorType();
+      MajorType majorType;
+      if (col.isNullable()) {
+        majorType = Types.optional(minorType);
+      } else {
+        majorType = Types.required(minorType);
+      }
       MaterializedField field = MaterializedField.create(name, majorType);
       final Class<? extends ValueVector> clazz = (Class<? extends ValueVector>) TypeHelper.getValueVectorClass(
           minorType, majorType.getMode());
       ValueVector vector = output.addField(field, clazz);
       vector.allocateNew();
-      vectorBuilder.add(vector);
-      copierBuilder.add(getCopier(kuduType, i, vector));
+      
+      ProjectedColumnInfo pci = new ProjectedColumnInfo();
+      pci.vv = vector;
+      pci.kuduColumn = col;
+      pci.index = i;
+      pciBuilder.add(pci);
     }
 
-    vectors = vectorBuilder.build();
-    copiers = copierBuilder.build();
+    projectedCols = pciBuilder.build();
   }
-
+  
   private void addRowResult(RowResult result, int rowIndex) throws SchemaChangeException {
-    if (copiers == null) {
-      initCopiers(result.getColumnProjection());
+    if (projectedCols == null) {
+      initCols(result.getColumnProjection());
     }
     
-    for (Copier<?> c : copiers) {
-      c.copy(result, rowIndex);
-    }
-  }
-
-
-  @Override
-  public void close() {
-  }
-  
-
-  private Copier<?> getCopier(Type kuduType, int offset, ValueVector v) {
-
-    if (v instanceof NullableBigIntVector) {
-      return new BigIntCopier(offset, (NullableBigIntVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableFloat4Vector) {
-      return new Float4Copier(offset, (NullableFloat4Vector.Mutator) v.getMutator());
-    } else if (v instanceof NullableFloat8Vector) {
-      return new Float8Copier(offset, (NullableFloat8Vector.Mutator) v.getMutator());
-    } else if (v instanceof NullableIntVector) {
-      return new IntCopier(offset, (NullableIntVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableVarCharVector) {
-      return new VarCharCopier(offset, (NullableVarCharVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableVarBinaryVector) {
-      return new VarBinaryCopier(offset, (NullableVarBinaryVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableTimeStampVector) {
-      return new TimeStampCopier(offset, (NullableTimeStampVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableBitVector) {
-      return new BitCopier(offset, (NullableBitVector.Mutator) v.getMutator());
-    }
-
-    throw new IllegalArgumentException("Unknown how to handle vector.");
-  }
-  
-  private abstract class Copier<T extends ValueVector.Mutator> {
-    protected final int columnIndex;
-    protected final T mutator;
-
-    public Copier(int columnIndex, T mutator) {
-      this.columnIndex = columnIndex;
-      this.mutator = mutator;
-    }
-
-    abstract void copy(RowResult result, int index);
-  }
-
-  private class IntCopier extends Copier<NullableIntVector.Mutator> {
-    public IntCopier(int offset, NullableIntVector.Mutator mutator) {
-      super(offset, mutator);
-    }
-
-    @Override
-    void copy(RowResult result, int index) {
-      if (result.isNull(columnIndex)) {
-        mutator.setNull(index);
-      } else {
-        mutator.setSafe(index, result.getInt(columnIndex));
-      }
-    }
-  }
-
-  private class BigIntCopier extends Copier<NullableBigIntVector.Mutator> {
-    public BigIntCopier(int offset, NullableBigIntVector.Mutator mutator) {
-      super(offset, mutator);
-    }
-
-    @Override
-    void copy(RowResult result, int index) {
-      if (result.isNull(columnIndex)) {
-        mutator.setNull(index);
-      } else {
-        mutator.setSafe(index, result.getLong(columnIndex));
+    for (ProjectedColumnInfo pci : projectedCols) {
+      if (result.isNull(pci.index)) {
+        continue;
       }
-    }
-  }
-
-  private class Float4Copier extends Copier<NullableFloat4Vector.Mutator> {
-
-    public Float4Copier(int columnIndex, NullableFloat4Vector.Mutator mutator) {
-      super(columnIndex, mutator);
-    }
-
-    @Override
-    void copy(RowResult result, int index) {
-      if (result.isNull(columnIndex)) {
-        mutator.setNull(index);
-      } else {
-        mutator.setSafe(index, result.getFloat(columnIndex));
+      switch (pci.kuduColumn.getType()) {
+      case BINARY:
+      {
+        ByteBuffer value = result.getBinary(pci.index);
+        if (pci.kuduColumn.isNullable()) {
+          ((NullableVarBinaryVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, value, 0, value.remaining());
+        } else {
+          ((VarBinaryVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, value, 0, value.remaining());          
+        }
+        break;
       }
-    }
-
-  }
-
-
-  private class Float8Copier extends Copier<NullableFloat8Vector.Mutator> {
-
-    public Float8Copier(int columnIndex, NullableFloat8Vector.Mutator mutator) {
-      super(columnIndex, mutator);
-    }
-
-    @Override
-    void copy(RowResult result, int index) {
-      if (result.isNull(columnIndex)) {
-        mutator.setNull(index);
-      } else {
-        mutator.setSafe(index, result.getDouble(columnIndex));
+      case STRING:
+      {
+        ByteBuffer value = result.getBinary(pci.index);
+        if (pci.kuduColumn.isNullable()) {
+          ((NullableVarCharVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, value, 0, value.remaining());
+        } else {
+          ((VarCharVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, value, 0, value.remaining());          
+        }
+        break;
       }
-    }
-  }
-
-  // TODO: decimal copier
-  
-  private class VarCharCopier extends Copier<NullableVarCharVector.Mutator> {
-
-    public VarCharCopier(int columnIndex, NullableVarCharVector.Mutator mutator) {
-      super(columnIndex, mutator);
-    }
-
-    @Override
-    void copy(RowResult result, int index) {
-      if (result.isNull(columnIndex)) {
-        mutator.setNull(index);
-      } else {
-        ByteBuffer value = result.getBinary(columnIndex);
-        mutator.setSafe(index, value, 0, value.remaining());
+      case BOOL:
+        if (pci.kuduColumn.isNullable()) {
+          ((NullableBitVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getBoolean(pci.index) ? 1 : 0);
+        } else {
+          ((BitVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getBoolean(pci.index) ? 1 : 0);
+        }
+        break;
+      case DOUBLE:
+        if (pci.kuduColumn.isNullable()) {
+          ((NullableFloat8Vector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getDouble(pci.index));
+        } else {
+          ((Float8Vector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getDouble(pci.index));
+        }
+        break;
+      case FLOAT:
+        if (pci.kuduColumn.isNullable()) {
+          ((NullableFloat4Vector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getFloat(pci.index));
+        } else {
+          ((Float4Vector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getFloat(pci.index));
+        }
+        break;
+      case INT16:
+        if (pci.kuduColumn.isNullable()) {
+          ((NullableIntVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getShort(pci.index));
+        } else {
+          ((IntVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getShort(pci.index));
+        }
+        break;
+      case INT32:
+        if (pci.kuduColumn.isNullable()) {
+          ((NullableIntVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getInt(pci.index));
+        } else {
+          ((IntVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getInt(pci.index));
+        }
+        break;
+      case INT8:
+        if (pci.kuduColumn.isNullable()) {
+          ((NullableIntVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getByte(pci.index));
+        } else {
+          ((IntVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getByte(pci.index));
+        }
+        break;
+      case INT64:
+        if (pci.kuduColumn.isNullable()) {
+          ((NullableBigIntVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getLong(pci.index));
+        } else {
+          ((BigIntVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getLong(pci.index));
+        }
+        break;
+      case TIMESTAMP:
+        if (pci.kuduColumn.isNullable()) {
+          ((NullableTimeStampVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getLong(pci.index));
+        } else {
+          ((TimeStampVector.Mutator)pci.vv.getMutator())
+            .setSafe(rowIndex, result.getLong(pci.index));
+        }
+        break;
+      default:
+        throw new SchemaChangeException("unknown type"); // TODO make better
       }
     }
   }
 
-  private class VarBinaryCopier extends Copier<NullableVarBinaryVector.Mutator> {
-
-    public VarBinaryCopier(int columnIndex, NullableVarBinaryVector.Mutator mutator) {
-      super(columnIndex, mutator);
-    }
 
-    @Override
-    void copy(RowResult result, int index) {
-      if (result.isNull(columnIndex)) {
-        mutator.setNull(index);
-      } else {
-        ByteBuffer value = result.getBinary(columnIndex);
-        mutator.setSafe(index, value, 0, value.remaining());
-      }
-    }
+  @Override
+  public void close() {
   }
-
-  // TODO: DateCopier
-  // TODO: TimeCopier
   
-  private class TimeStampCopier extends Copier<NullableTimeStampVector.Mutator> {
-
-    public TimeStampCopier(int columnIndex, NullableTimeStampVector.Mutator mutator) {
-      super(columnIndex, mutator);
-    }
-
-    @Override
-    void copy(RowResult result, int index) {
-      if (result.isNull(columnIndex)) {
-        mutator.setNull(index);
-      } else {
-        long ts = result.getLong(columnIndex);
-        mutator.setSafe(index, ts / 1000);
-      }
-    }
-  }
-
-  private class BitCopier extends Copier<NullableBitVector.Mutator> {
-    public BitCopier(int columnIndex, NullableBitVector.Mutator mutator) {
-      super(columnIndex, mutator);
-    }
-
-    @Override
-    void copy(RowResult result, int index) {
-      if (result.isNull(columnIndex)) {
-        mutator.setNull(index);
-      } else {
-        mutator.setSafe(index, result.getBoolean(columnIndex) ? 1 : 0);
-      }
-    }
-  }
-
 }


[06/12] drill git commit: DRILL-4241: Add parallelization and assignment

Posted by ja...@apache.org.
DRILL-4241: Add parallelization and assignment


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

Branch: refs/heads/master
Commit: 4f3f401e4c299b007218e5231ad43f9977f01dd2
Parents: b1d9d79
Author: Steven Phillips <sm...@apache.org>
Authored: Thu Nov 19 16:00:40 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 22:54:21 2016 -0800

----------------------------------------------------------------------
 .../drill/exec/store/kudu/KuduGroupScan.java    | 99 +++++++++++++++++++-
 .../drill/exec/store/kudu/KuduRecordReader.java |  6 +-
 .../drill/exec/store/kudu/KuduSubScan.java      | 16 +++-
 3 files changed, 114 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4f3f401e/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
index bc543d9..8403632 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
@@ -18,9 +18,14 @@
 package org.apache.drill.exec.store.kudu;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.EndpointAffinity;
@@ -40,16 +45,27 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import org.apache.drill.exec.store.schedule.AffinityCreator;
+import org.apache.drill.exec.store.schedule.AssignmentCreator;
+import org.apache.drill.exec.store.schedule.CompleteWork;
+import org.apache.drill.exec.store.schedule.EndpointByteMap;
+import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
+import org.kududb.client.LocatedTablet;
+import org.kududb.client.LocatedTablet.Replica;
 
 @JsonTypeName("kudu-scan")
 public class KuduGroupScan extends AbstractGroupScan {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduGroupScan.class);
+  private static final long DEFAULT_TABLET_SIZE = 1000;
 
   private KuduStoragePluginConfig storagePluginConfig;
   private List<SchemaPath> columns;
   private KuduScanSpec kuduScanSpec;
   private KuduStoragePlugin storagePlugin;
   private boolean filterPushedDown = false;
+  private List<KuduWork> kuduWorkList = Lists.newArrayList();
+  private ListMultimap<Integer,KuduWork> assignments;
+  private List<EndpointAffinity> affinities;
 
 
   @JsonCreator
@@ -67,6 +83,67 @@ public class KuduGroupScan extends AbstractGroupScan {
     this.storagePluginConfig = storagePlugin.getConfig();
     this.kuduScanSpec = scanSpec;
     this.columns = columns == null || columns.size() == 0? ALL_COLUMNS : columns;
+    init();
+  }
+
+  private void init() {
+    String tableName = kuduScanSpec.getTableName();
+    Collection<DrillbitEndpoint> endpoints = storagePlugin.getContext().getBits();
+    Map<String,DrillbitEndpoint> endpointMap = Maps.newHashMap();
+    for (DrillbitEndpoint endpoint : endpoints) {
+      endpointMap.put(endpoint.getAddress(), endpoint);
+    }
+    try {
+      List<LocatedTablet> locations = storagePlugin.getClient().openTable(tableName).getTabletsLocations(10000);
+      for (LocatedTablet tablet : locations) {
+        KuduWork work = new KuduWork(tablet.getPartition().getPartitionKeyStart(), tablet.getPartition().getPartitionKeyEnd());
+        for (Replica replica : tablet.getReplicas()) {
+          String host = replica.getRpcHost();
+          DrillbitEndpoint ep = endpointMap.get(host);
+          if (ep != null) {
+            work.getByteMap().add(ep, DEFAULT_TABLET_SIZE);
+          }
+        }
+        kuduWorkList.add(work);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static class KuduWork implements CompleteWork {
+
+    private EndpointByteMapImpl byteMap = new EndpointByteMapImpl();
+    private byte[] partitionKeyStart;
+    private byte[] partitionKeyEnd;
+
+    public KuduWork(byte[] partitionKeyStart, byte[] partitionKeyEnd) {
+      this.partitionKeyStart = partitionKeyStart;
+      this.partitionKeyEnd = partitionKeyEnd;
+    }
+
+    public byte[] getPartitionKeyStart() {
+      return partitionKeyStart;
+    }
+
+    public byte[] getPartitionKeyEnd() {
+      return partitionKeyEnd;
+    }
+
+    @Override
+    public long getTotalBytes() {
+      return DEFAULT_TABLET_SIZE;
+    }
+
+    @Override
+    public EndpointByteMap getByteMap() {
+      return byteMap;
+    }
+
+    @Override
+    public int compareTo(CompleteWork o) {
+      return 0;
+    }
   }
 
   /**
@@ -80,6 +157,8 @@ public class KuduGroupScan extends AbstractGroupScan {
     this.storagePlugin = that.storagePlugin;
     this.storagePluginConfig = that.storagePluginConfig;
     this.filterPushedDown = that.filterPushedDown;
+    this.kuduWorkList = that.kuduWorkList;
+    this.assignments = that.assignments;
   }
 
   @Override
@@ -91,13 +170,16 @@ public class KuduGroupScan extends AbstractGroupScan {
 
   @Override
   public List<EndpointAffinity> getOperatorAffinity() {
-    return Collections.EMPTY_LIST;
+    if (affinities == null) {
+      affinities = AffinityCreator.getAffinityMap(kuduWorkList);
+    }
+    return affinities;
   }
 
 
   @Override
   public int getMaxParallelizationWidth() {
-    return 1;
+    return kuduWorkList.size();
   }
 
 
@@ -107,14 +189,21 @@ public class KuduGroupScan extends AbstractGroupScan {
    */
   @Override
   public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
+    assignments = AssignmentCreator.getMappings(incomingEndpoints, kuduWorkList, storagePlugin.getContext());
   }
 
 
   @Override
   public KuduSubScan getSpecificScan(int minorFragmentId) {
-    return new KuduSubScan(storagePlugin, storagePluginConfig,
-        ImmutableList.of(new KuduSubScanSpec(kuduScanSpec.getTableName())),
-        this.columns);
+    List<KuduWork> workList = assignments.get(minorFragmentId);
+
+    List<KuduSubScanSpec> scanSpecList = Lists.newArrayList();
+
+    for (KuduWork work : workList) {
+      scanSpecList.add(new KuduSubScanSpec(getTableName(), work.getPartitionKeyStart(), work.getPartitionKeyEnd()));
+    }
+
+    return new KuduSubScan(storagePlugin, storagePluginConfig, scanSpecList, this.columns);
   }
 
   // KuduStoragePlugin plugin, KuduStoragePluginConfig config,

http://git-wip-us.apache.org/repos/asf/drill/blob/4f3f401e/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
index 17427f1..adbdb83 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
@@ -78,6 +78,7 @@ public class KuduRecordReader extends AbstractRecordReader {
     setColumns(projectedColumns);
     this.client = client;
     scanSpec = subScanSpec;
+    logger.debug("Scan spec: {}", subScanSpec);
   }
 
   @Override
@@ -94,7 +95,10 @@ public class KuduRecordReader extends AbstractRecordReader {
         }
         builder.setProjectedColumnNames(colNames);
       }
-      scanner = builder.build();
+      scanner = builder
+              .lowerBoundPartitionKeyRaw(scanSpec.getStartKey())
+              .exclusiveUpperBoundPartitionKeyRaw(scanSpec.getEndKey())
+              .build();
     } catch (Exception e) {
       throw new ExecutionSetupException(e);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/4f3f401e/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSubScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSubScan.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSubScan.java
index 267ee77..9025db7 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSubScan.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSubScan.java
@@ -115,16 +115,30 @@ public class KuduSubScan extends AbstractBase implements SubScan {
   public static class KuduSubScanSpec {
 
     private final String tableName;
+    private final byte[] startKey;
+    private final byte[] endKey;
 
     @JsonCreator
-    public KuduSubScanSpec(@JsonProperty("tableName") String tableName) {
+    public KuduSubScanSpec(@JsonProperty("tableName") String tableName,
+                           @JsonProperty("startKey") byte[] startKey,
+                           @JsonProperty("endKey") byte[] endKey) {
       this.tableName = tableName;
+      this.startKey = startKey;
+      this.endKey = endKey;
     }
 
     public String getTableName() {
       return tableName;
     }
 
+    public byte[] getStartKey() {
+      return startKey;
+    }
+
+    public byte[] getEndKey() {
+      return endKey;
+    }
+
   }
 
   @Override


[04/12] drill git commit: DRILL-4241: Add pushdown of column projections

Posted by ja...@apache.org.
DRILL-4241: Add pushdown of column projections


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

Branch: refs/heads/master
Commit: 2c0e53a85c72c735f88ffb1272f86ed957c4774b
Parents: 8aac12b
Author: Todd Lipcon <to...@cloudera.com>
Authored: Thu Nov 19 14:15:58 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 22:54:20 2016 -0800

----------------------------------------------------------------------
 .../apache/drill/exec/store/kudu/KuduRecordReader.java  | 12 +++++++++---
 1 file changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/2c0e53a8/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
index fefb649..8247e01 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
@@ -44,6 +44,8 @@ import org.kududb.client.RowResult;
 import org.kududb.client.RowResultIterator;
 import org.kududb.client.shaded.com.google.common.collect.ImmutableMap;
 
+import com.google.common.collect.Lists;
+
 public class KuduRecordReader extends AbstractRecordReader {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduRecordReader.class);
 
@@ -51,7 +53,6 @@ public class KuduRecordReader extends AbstractRecordReader {
 
   private final KuduClient client;
   private final KuduSubScanSpec scanSpec;
-  private KuduTable table;
   private VectorContainerWriter containerWriter;
   private MapWriter writer;
   private KuduScanner scanner;
@@ -70,7 +71,12 @@ public class KuduRecordReader extends AbstractRecordReader {
   public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
     try {
       KuduTable table = client.openTable(scanSpec.getTableName());
-      scanner = client.newScannerBuilder(table).build();
+      List<String> colNames = Lists.newArrayList();
+      for (SchemaPath p : this.getColumns()) {
+        colNames.add(p.getAsUnescapedPath());
+      }
+      scanner = client.newScannerBuilder(table)
+          .setProjectedColumnNames(colNames).build();
       containerWriter = new VectorContainerWriter(output);
       writer = containerWriter.rootAsMap();
     } catch (Exception e) {
@@ -106,7 +112,7 @@ public class KuduRecordReader extends AbstractRecordReader {
         }
         iterator = scanner.nextRows();
       }
-      for (; rowCount < 4095 && iterator.hasNext(); rowCount++) {
+      for (; rowCount < TARGET_RECORD_COUNT && iterator.hasNext(); rowCount++) {
         writer.setPosition(rowCount);
         addRowResult(iterator.next());
       }


[12/12] drill git commit: DRILL-4241: Add Single Tablet Writer

Posted by ja...@apache.org.
DRILL-4241: Add Single Tablet Writer

- Also move to a test bootstrap
- Update to the latest kudu and Drill
- Add plugin to Drill distribution
- Checkstyle and directory cleanup

This closes #314.


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

Branch: refs/heads/master
Commit: 392d1f7e9398fb1bee8f67b0d49a82436c3145fb
Parents: 3694909
Author: Jacques Nadeau <ja...@apache.org>
Authored: Thu Nov 19 18:20:03 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 23:46:44 2016 -0800

----------------------------------------------------------------------
 contrib/pom.xml                                 |   1 +
 contrib/storage-kudu/pom.xml                    | 114 ++++++++++--
 .../storage-kudu/src/main/codegen/config.fmpp   |  23 +++
 .../codegen/templates/KuduRecordWriter.java     | 175 +++++++++++++++++++
 .../drill/exec/store/kudu/KuduRecordReader.java | 119 ++++++-------
 .../exec/store/kudu/KuduRecordWriterImpl.java   | 174 ++++++++++++++++++
 .../exec/store/kudu/KuduSchemaFactory.java      |  21 +++
 .../exec/store/kudu/KuduStoragePlugin.java      |  13 +-
 .../drill/exec/store/kudu/KuduWriter.java       |  79 +++++++++
 .../exec/store/kudu/KuduWriterBatchCreator.java |  43 +++++
 .../resources/bootstrap-storage-plugins.json    |   4 +-
 .../src/main/resources/checkstyle-config.xml    |  42 -----
 .../main/resources/checkstyle-suppressions.xml  |  19 --
 .../drill/store/kudu/TestKuduConnect.java       |   9 +-
 .../apache/drill/store/kudu/TestKuduPlugin.java |  10 ++
 .../resources/bootstrap-storage-plugins.json    |   9 +
 distribution/pom.xml                            |   5 +
 distribution/src/assemble/bin.xml               |   1 +
 18 files changed, 706 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/pom.xml b/contrib/pom.xml
index bc95910..87b84c1 100644
--- a/contrib/pom.xml
+++ b/contrib/pom.xml
@@ -36,6 +36,7 @@
     <module>storage-hive</module>
     <module>storage-mongo</module>
     <module>storage-jdbc</module>
+    <module>storage-kudu</module>
     <module>sqlline</module>
     <module>data</module>
     <module>gis</module>

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/pom.xml b/contrib/storage-kudu/pom.xml
index 7e57ca8..cca319f 100644
--- a/contrib/storage-kudu/pom.xml
+++ b/contrib/storage-kudu/pom.xml
@@ -16,23 +16,19 @@
   <parent>
     <artifactId>drill-contrib-parent</artifactId>
     <groupId>org.apache.drill.contrib</groupId>
-    <version>1.3.0</version>
+    <version>1.5.0-SNAPSHOT</version>
   </parent>
 
-  <artifactId>drill-storage-kudu</artifactId>
-  <version>1.3.0-SNAPSHOT</version>
+  <artifactId>drill-kudu-storage</artifactId>
 
   <name>contrib/kudu-storage-plugin</name>
 
-  <properties>
-    <drill.version>1.3.0</drill.version>
-  </properties>
 
   <dependencies>
     <dependency>
       <groupId>org.apache.drill.exec</groupId>
       <artifactId>drill-java-exec</artifactId>
-      <version>${drill.version}</version>
+      <version>${project.version}</version>
     </dependency>
 
     <!-- Test dependencies -->
@@ -40,7 +36,7 @@
       <groupId>org.apache.drill.exec</groupId>
       <artifactId>drill-java-exec</artifactId>
       <classifier>tests</classifier>
-      <version>${drill.version}</version>
+      <version>${project.version}</version>
       <scope>test</scope>
     </dependency>
 
@@ -48,27 +44,20 @@
       <groupId>org.apache.drill</groupId>
       <artifactId>drill-common</artifactId>
       <classifier>tests</classifier>
-      <version>${drill.version}</version>
+      <version>${project.version}</version>
       <scope>test</scope>
     </dependency>
 
     <dependency>
       <groupId>org.kududb</groupId>
       <artifactId>kudu-client</artifactId>
-      <version>0.5.0</version>
+      <version>0.6.0</version>
     </dependency>
 
   </dependencies>
 
   <repositories>
     <repository>
-      <id>drill-1016</id>
-      <url>https://repository.apache.org/content/repositories/orgapachedrill-1016/</url>
-      <snapshots>
-        <enabled>false</enabled>
-      </snapshots>
-    </repository>
-    <repository>
       <id>cdh.repo</id>
       <name>Cloudera Repositories</name>
       <url>https://repository.cloudera.com/artifactory/cloudera-repos</url>
@@ -78,9 +67,98 @@
     </repository>
   </repositories>
 
+  <pluginRepositories>
+    <pluginRepository>
+      <id>apache-snapshots</id>
+      <url>https://repository.apache.org/content/groups/snapshots/</url>
+      <snapshots>
+        <enabled>true</enabled>
+      </snapshots>
+      <releases>
+        <enabled>false</enabled>
+      </releases>
+      
+    </pluginRepository>
+  </pluginRepositories>
   <build>
     <plugins>
-
+    <plugin>
+        <artifactId>maven-resources-plugin</artifactId>
+        <executions>
+          <execution> <!-- copy all templates/data in the same location to compile them at once -->
+            <id>copy-fmpp-resources</id>
+            <phase>initialize</phase>
+            <goals>
+              <goal>copy-resources</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>${project.build.directory}/codegen</outputDirectory>
+              <resources>
+                <resource>
+                  <directory>src/main/codegen</directory>
+                  <filtering>false</filtering>
+                </resource>
+              </resources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>2.8</version>
+        <executions>
+          <!-- Extract ValueVectorTypes.tdd from drill-vector.jar and put 
+            it under ${project.build.directory}/codegen/data where all freemarker data 
+            files are. -->
+          <execution>
+            <id>unpack-vector-types</id>
+            <phase>initialize</phase>
+            <goals>
+              <goal>unpack</goal>
+            </goals>
+            <configuration>
+              <artifactItems>
+                <artifactItem>
+                  <groupId>org.apache.drill.exec</groupId>
+                  <artifactId>vector</artifactId>
+                  <version>${project.version}</version>
+                  <type>jar</type>
+                  <overWrite>true</overWrite>
+                  <outputDirectory>${project.build.directory}/</outputDirectory>
+                  <includes>codegen/data/ValueVectorTypes.tdd</includes>
+                </artifactItem>
+              </artifactItems>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin> <!-- generate sources from fmpp -->
+        <groupId>org.apache.drill.tools</groupId>
+        <artifactId>drill-fmpp-maven-plugin</artifactId>
+        <version>${project.version}</version>
+        <dependencies>
+          <dependency>
+            <groupId>org.freemarker</groupId>
+            <artifactId>freemarker</artifactId>
+            <version>2.3.19</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <id>generate-fmpp</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>generate</goal>
+            </goals>
+            <configuration>
+              <config>${project.build.directory}/codegen/config.fmpp</config>
+              <output>${project.build.directory}/generated-sources</output>
+              <templates>${project.build.directory}/codegen/templates</templates>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/main/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/codegen/config.fmpp b/contrib/storage-kudu/src/main/codegen/config.fmpp
new file mode 100644
index 0000000..40a29b4
--- /dev/null
+++ b/contrib/storage-kudu/src/main/codegen/config.fmpp
@@ -0,0 +1,23 @@
+# 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.
+
+data: {
+    vv: tdd(../data/ValueVectorTypes.tdd),
+
+}
+freemarkerLinks: {
+    includes: includes/
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/main/codegen/templates/KuduRecordWriter.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/codegen/templates/KuduRecordWriter.java b/contrib/storage-kudu/src/main/codegen/templates/KuduRecordWriter.java
new file mode 100644
index 0000000..01c7c28
--- /dev/null
+++ b/contrib/storage-kudu/src/main/codegen/templates/KuduRecordWriter.java
@@ -0,0 +1,175 @@
+/**
+ * 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.
+ */
+
+<@pp.dropOutputFile />
+<@pp.changeOutputFile name="org/apache/drill/exec/store/kudu/KuduRecordWriter.java" />
+/**
+ * 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.store.kudu;
+
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.fn.JsonOutput;
+import java.io.IOException;
+import java.lang.UnsupportedOperationException;
+import java.util.List;
+import com.google.common.collect.Lists;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.vector.*;
+import org.apache.drill.exec.util.DecimalUtility;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.parquet.io.api.RecordConsumer;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.io.api.Binary;
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.common.types.TypeProtos;
+import org.joda.time.DateTimeUtils;
+import java.io.IOException;
+import java.lang.UnsupportedOperationException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import com.google.common.collect.Lists;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.vector.*;
+import org.apache.drill.exec.util.DecimalUtility;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.parquet.io.api.RecordConsumer;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.io.api.Binary;
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.common.types.TypeProtos;
+import org.joda.time.DateTimeUtils;
+import java.io.IOException;
+import java.lang.UnsupportedOperationException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.kududb.client.*;
+import org.apache.drill.exec.store.*;
+
+public abstract class KuduRecordWriter extends AbstractRecordWriter implements RecordWriter {
+
+    private PartialRow row;
+
+    public void setUp(PartialRow row) {
+      this.row = row;
+    }
+
+  <#list vv.types as type>
+    <#list type.minor as minor>
+      <#list vv.modes as mode>
+      
+        <#if mode.prefix == "Repeated" || 
+        minor.class == "TinyInt" ||
+        minor.class == "UInt1" ||
+        minor.class == "UInt2" ||
+        minor.class == "SmallInt" ||
+        minor.class == "Time" ||
+        minor.class == "Decimal9" ||
+        minor.class == "Decimal18" ||
+        minor.class == "Date" ||
+        minor.class == "UInt4" ||
+        minor.class == "Decimal28Sparse" ||
+        minor.class == "Decimal38Sparse" ||
+        minor.class?contains("Interval")
+        >
+        
+        <#else>
+          @Override
+          public FieldConverter getNew${mode.prefix}${minor.class}Converter(int fieldId, String fieldName, FieldReader reader) {
+            return new ${mode.prefix}${minor.class}KuduConverter(fieldId, fieldName, reader);
+          }
+      
+          public class ${mode.prefix}${minor.class}KuduConverter extends FieldConverter {
+            private Nullable${minor.class}Holder holder = new Nullable${minor.class}Holder();
+      
+            public ${mode.prefix}${minor.class}KuduConverter(int fieldId, String fieldName, FieldReader reader) {
+              super(fieldId, fieldName, reader);
+            }
+      
+            @Override
+            public void writeField() throws IOException {
+         
+          <#if mode.prefix == "Nullable" >
+            if (!reader.isSet()) {
+              return;
+            }
+          </#if>
+          
+            reader.read(holder);
+            
+            <#if minor.class == "Float4">
+              row.addFloat(fieldId, holder.value);
+            <#elseif minor.class == "TimeStamp">
+              row.addLong(fieldId, holder.value*1000);
+            <#elseif minor.class == "Int">
+              row.addInt(fieldId, holder.value);
+            <#elseif minor.class == "BigInt">
+              row.addLong(fieldId, holder.value);
+            <#elseif minor.class == "Float8">
+              row.addDouble(fieldId, holder.value);
+            <#elseif minor.class == "Bit">
+              row.addBoolean(fieldId, holder.value == 1);
+            <#elseif minor.class == "VarChar" >
+              byte[] bytes = new byte[holder.end - holder.start];
+              holder.buffer.getBytes(holder.start, bytes);
+              row.addStringUtf8(fieldId, bytes);
+            <#elseif minor.class == "VarBinary">
+              byte[] bytes = new byte[holder.end - holder.start];
+              holder.buffer.getBytes(holder.start, bytes);
+              row.addBinary(fieldId, bytes);
+              reader.read(holder);
+            <#else>
+              throw new UnsupportedOperationException();
+            </#if>
+            }
+          }
+          </#if>
+      </#list>
+    </#list>
+  </#list>
+  }

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
index a97df77..abd2ab7 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
@@ -75,7 +75,7 @@ public class KuduRecordReader extends AbstractRecordReader {
   private final KuduSubScanSpec scanSpec;
   private KuduScanner scanner;
   private RowResultIterator iterator;
-  
+
   private OutputMutator output;
   private OperatorContext context;
 
@@ -84,8 +84,8 @@ public class KuduRecordReader extends AbstractRecordReader {
     ValueVector vv;
     ColumnSchema kuduColumn;
   }
-  private ImmutableList<ProjectedColumnInfo> projectedCols;
 
+  private ImmutableList<ProjectedColumnInfo> projectedCols;
 
   public KuduRecordReader(KuduClient client, KuduSubScan.KuduSubScanSpec subScanSpec,
       List<SchemaPath> projectedColumns, FragmentContext context) {
@@ -101,7 +101,7 @@ public class KuduRecordReader extends AbstractRecordReader {
     this.context = context;
     try {
       KuduTable table = client.openTable(scanSpec.getTableName());
-      
+
       KuduScannerBuilder builder = client.newScannerBuilder(table);
       if (!isStarQuery()) {
         List<String> colNames = Lists.newArrayList();
@@ -114,9 +114,9 @@ public class KuduRecordReader extends AbstractRecordReader {
       context.getStats().startWait();
       try {
         scanner = builder
-                .lowerBoundPartitionKeyRaw(scanSpec.getStartKey())
-                .exclusiveUpperBoundPartitionKeyRaw(scanSpec.getEndKey())
-                .build();
+            .lowerBoundPartitionKeyRaw(scanSpec.getStartKey())
+            .exclusiveUpperBoundPartitionKeyRaw(scanSpec.getEndKey())
+            .build();
       } finally {
         context.getStats().stopWait();
       }
@@ -125,7 +125,7 @@ public class KuduRecordReader extends AbstractRecordReader {
     }
   }
 
-  static final Map<Type,MinorType> TYPES;
+  static final Map<Type, MinorType> TYPES;
 
   static {
     TYPES = ImmutableMap.<Type, MinorType> builder()
@@ -169,14 +169,14 @@ public class KuduRecordReader extends AbstractRecordReader {
     }
     return rowCount;
   }
-  
+
   @SuppressWarnings("unchecked")
   private void initCols(Schema schema) throws SchemaChangeException {
     ImmutableList.Builder<ProjectedColumnInfo> pciBuilder = ImmutableList.builder();
-    
+
     for (int i = 0; i < schema.getColumnCount(); i++) {
-      ColumnSchema col = schema.getColumnByIndex(i);  
-      
+      ColumnSchema col = schema.getColumnByIndex(i);
+
       final String name = col.getName();
       final Type kuduType = col.getType();
       MinorType minorType = TYPES.get(kuduType);
@@ -184,7 +184,7 @@ public class KuduRecordReader extends AbstractRecordReader {
         logger.warn("Ignoring column that is unsupported.", UserException
             .unsupportedError()
             .message(
-                "A column you queried has a data type that is not currently supported by the JDBC storage plugin. "
+                "A column you queried has a data type that is not currently supported by the Kudu storage plugin. "
                     + "The column's name was %s and its Kudu data type was %s. ",
                 name, kuduType.toString())
             .addContext("column Name", name)
@@ -204,7 +204,7 @@ public class KuduRecordReader extends AbstractRecordReader {
           minorType, majorType.getMode());
       ValueVector vector = output.addField(field, clazz);
       vector.allocateNew();
-      
+
       ProjectedColumnInfo pci = new ProjectedColumnInfo();
       pci.vv = vector;
       pci.kuduColumn = col;
@@ -214,111 +214,109 @@ public class KuduRecordReader extends AbstractRecordReader {
 
     projectedCols = pciBuilder.build();
   }
-  
+
   private void addRowResult(RowResult result, int rowIndex) throws SchemaChangeException {
     if (projectedCols == null) {
       initCols(result.getColumnProjection());
     }
-    
+
     for (ProjectedColumnInfo pci : projectedCols) {
       if (result.isNull(pci.index)) {
         continue;
       }
       switch (pci.kuduColumn.getType()) {
-      case BINARY:
-      {
+      case BINARY: {
         ByteBuffer value = result.getBinary(pci.index);
         if (pci.kuduColumn.isNullable()) {
-          ((NullableVarBinaryVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, value, 0, value.remaining());
+          ((NullableVarBinaryVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, value, 0, value.remaining());
         } else {
-          ((VarBinaryVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, value, 0, value.remaining());          
+          ((VarBinaryVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, value, 0, value.remaining());
         }
         break;
       }
-      case STRING:
-      {
+      case STRING: {
         ByteBuffer value = result.getBinary(pci.index);
         if (pci.kuduColumn.isNullable()) {
-          ((NullableVarCharVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, value, 0, value.remaining());
+          ((NullableVarCharVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, value, 0, value.remaining());
         } else {
-          ((VarCharVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, value, 0, value.remaining());          
+          ((VarCharVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, value, 0, value.remaining());
         }
         break;
       }
       case BOOL:
         if (pci.kuduColumn.isNullable()) {
-          ((NullableBitVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getBoolean(pci.index) ? 1 : 0);
+          ((NullableBitVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getBoolean(pci.index) ? 1 : 0);
         } else {
-          ((BitVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getBoolean(pci.index) ? 1 : 0);
+          ((BitVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getBoolean(pci.index) ? 1 : 0);
         }
         break;
       case DOUBLE:
         if (pci.kuduColumn.isNullable()) {
-          ((NullableFloat8Vector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getDouble(pci.index));
+          ((NullableFloat8Vector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getDouble(pci.index));
         } else {
-          ((Float8Vector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getDouble(pci.index));
+          ((Float8Vector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getDouble(pci.index));
         }
         break;
       case FLOAT:
         if (pci.kuduColumn.isNullable()) {
-          ((NullableFloat4Vector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getFloat(pci.index));
+          ((NullableFloat4Vector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getFloat(pci.index));
         } else {
-          ((Float4Vector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getFloat(pci.index));
+          ((Float4Vector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getFloat(pci.index));
         }
         break;
       case INT16:
         if (pci.kuduColumn.isNullable()) {
-          ((NullableIntVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getShort(pci.index));
+          ((NullableIntVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getShort(pci.index));
         } else {
-          ((IntVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getShort(pci.index));
+          ((IntVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getShort(pci.index));
         }
         break;
       case INT32:
         if (pci.kuduColumn.isNullable()) {
-          ((NullableIntVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getInt(pci.index));
+          ((NullableIntVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getInt(pci.index));
         } else {
-          ((IntVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getInt(pci.index));
+          ((IntVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getInt(pci.index));
         }
         break;
       case INT8:
         if (pci.kuduColumn.isNullable()) {
-          ((NullableIntVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getByte(pci.index));
+          ((NullableIntVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getByte(pci.index));
         } else {
-          ((IntVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getByte(pci.index));
+          ((IntVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getByte(pci.index));
         }
         break;
       case INT64:
         if (pci.kuduColumn.isNullable()) {
-          ((NullableBigIntVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getLong(pci.index));
+          ((NullableBigIntVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getLong(pci.index));
         } else {
-          ((BigIntVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getLong(pci.index));
+          ((BigIntVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getLong(pci.index));
         }
         break;
       case TIMESTAMP:
         if (pci.kuduColumn.isNullable()) {
-          ((NullableTimeStampVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getLong(pci.index) / 1000);
+          ((NullableTimeStampVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getLong(pci.index) / 1000);
         } else {
-          ((TimeStampVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getLong(pci.index) / 1000);
+          ((TimeStampVector.Mutator) pci.vv.getMutator())
+              .setSafe(rowIndex, result.getLong(pci.index) / 1000);
         }
         break;
       default:
@@ -327,9 +325,8 @@ public class KuduRecordReader extends AbstractRecordReader {
     }
   }
 
-
   @Override
   public void close() {
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordWriterImpl.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordWriterImpl.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordWriterImpl.java
new file mode 100644
index 0000000..6b39cc5
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordWriterImpl.java
@@ -0,0 +1,174 @@
+/**
+ * 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.store.kudu;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.kududb.ColumnSchema;
+import org.kududb.Schema;
+import org.kududb.Type;
+import org.kududb.client.Insert;
+import org.kududb.client.KuduClient;
+import org.kududb.client.KuduSession;
+import org.kududb.client.KuduTable;
+import org.kududb.client.OperationResponse;
+import org.kududb.client.SessionConfiguration.FlushMode;
+
+public class KuduRecordWriterImpl extends KuduRecordWriter {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduRecordWriterImpl.class);
+
+  private static final int FLUSH_FREQUENCY = 100;
+
+  private final KuduClient client;
+  private final String name;
+  private final OperatorContext context;
+  private KuduTable table;
+  private KuduSession session;
+
+  private Insert insert;
+  private int recordsSinceFlush;
+
+  public KuduRecordWriterImpl(OperatorContext context, KuduClient client, String name) {
+    this.client = client;
+    this.name = name;
+    this.context = context;
+    session = client.newSession();
+    session.setFlushMode(FlushMode.MANUAL_FLUSH);
+  }
+
+  @Override
+  public void init(Map<String, String> writerOptions) throws IOException {
+
+  }
+
+  @Override
+  public void updateSchema(VectorAccessible batch) throws IOException {
+    BatchSchema schema = batch.getSchema();
+    int i = 0;
+
+    try {
+      if (!checkForTable(name)) {
+        List<ColumnSchema> columns = new ArrayList<>();
+        for (MaterializedField f : schema) {
+          columns.add(new ColumnSchema.ColumnSchemaBuilder(f.getLastName(), getType(f.getType()))
+              .nullable(f.getType().getMode() == DataMode.OPTIONAL)
+              .key(i == 0).build());
+          i++;
+        }
+        Schema kuduSchema = new Schema(columns);
+        table = client.createTable(name, kuduSchema);
+      }
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  private boolean checkForTable(String name) throws Exception {
+    return !client.getTablesList(name).getTablesList().isEmpty();
+  }
+
+  private Type getType(MajorType t) {
+
+    if(t.getMode() == DataMode.REPEATED){
+      throw UserException
+      .dataWriteError()
+      .message("Kudu does not support array types.")
+      .build(logger);
+    }
+
+    switch (t.getMinorType()) {
+    case BIGINT:
+      return Type.INT64;
+    case BIT:
+      return Type.BOOL;
+    case FLOAT4:
+      return Type.FLOAT;
+    case FLOAT8:
+      return Type.DOUBLE;
+    case INT:
+      return Type.INT32;
+    case TIMESTAMP:
+      return Type.TIMESTAMP;
+    case VARBINARY:
+      return Type.BINARY;
+    case VARCHAR:
+      return Type.STRING;
+    default:
+      throw UserException
+        .dataWriteError()
+          .message("Data type: '%s' not supported in Kudu.", t.getMinorType().name())
+          .build(logger);
+    }
+  }
+
+  @Override
+  public void startRecord() throws IOException {
+    insert = table.newInsert();
+    setUp(insert.getRow());
+  }
+
+  @Override
+  public void endRecord() throws IOException {
+    try {
+      session.apply(insert);
+      recordsSinceFlush++;
+      if (recordsSinceFlush == FLUSH_FREQUENCY) {
+        flush();
+        recordsSinceFlush = 0;
+      }
+      insert = null;
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public void abort() throws IOException {
+  }
+
+  private void flush() throws IOException {
+    try {
+      // context.getStats().startWait();
+      List<OperationResponse> responses = session.flush();
+      for (OperationResponse response : responses) {
+        if (response.hasRowError()) {
+          throw new IOException(response.getRowError().toString());
+        }
+      }
+    } catch (Exception e) {
+      throw new IOException(e);
+    } finally {
+      // context.getStats().stopWait();
+    }
+  }
+
+  @Override
+  public void cleanup() throws IOException {
+    flush();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
index 7ea4f2f..af2775d 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
@@ -19,11 +19,15 @@ package org.apache.drill.exec.store.kudu;
 
 import java.io.IOException;
 import java.util.Collections;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Writer;
+import org.apache.drill.exec.planner.logical.CreateTableEntry;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.SchemaFactory;
@@ -97,6 +101,23 @@ public class KuduSchemaFactory implements SchemaFactory {
     }
 
     @Override
+    public CreateTableEntry createNewTable(final String tableName, List<String> partitionColumns) {
+      return new CreateTableEntry(){
+
+        @Override
+        public Writer getWriter(PhysicalOperator child) throws IOException {
+          return new KuduWriter(child, tableName, plugin);
+        }
+
+        @Override
+        public List<String> getPartitionColumns() {
+          return Collections.emptyList();
+        }
+
+      };
+    }
+
+    @Override
     public void dropTable(String tableName) {
       try {
         plugin.getClient().deleteTable(tableName);

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java
index 5e981b8..15aa469 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java
@@ -18,16 +18,12 @@
 package org.apache.drill.exec.store.kudu;
 
 import java.io.IOException;
-import java.util.Collections;
-import java.util.Set;
 
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.JSONOptions;
-import org.apache.drill.exec.ops.OptimizerRulesContext;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.StoragePluginOptimizerRule;
 import org.kududb.client.KuduClient;
 
 import com.fasterxml.jackson.core.type.TypeReference;
@@ -83,6 +79,11 @@ public class KuduStoragePlugin extends AbstractStoragePlugin {
   }
 
   @Override
+  public boolean supportsWrite() {
+    return true;
+  }
+
+  @Override
   public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
     schemaFactory.registerSchemas(schemaConfig, parent);
   }
@@ -92,8 +93,4 @@ public class KuduStoragePlugin extends AbstractStoragePlugin {
     return engineConfig;
   }
 
-  @Override
-  public Set<StoragePluginOptimizerRule> getOptimizerRules(OptimizerRulesContext optimizerRulesContext) {
-    return Collections.EMPTY_SET;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduWriter.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduWriter.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduWriter.java
new file mode 100644
index 0000000..03e29d3
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduWriter.java
@@ -0,0 +1,79 @@
+/**
+ * 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.store.kudu;
+
+import java.io.IOException;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.base.AbstractWriter;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class KuduWriter extends AbstractWriter {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduWriter.class);
+
+  private final KuduStoragePlugin plugin;
+  private final String name;
+
+  @JsonCreator
+  public KuduWriter(
+      @JsonProperty("child") PhysicalOperator child,
+      @JsonProperty("name") String name,
+      @JsonProperty("storage") StoragePluginConfig storageConfig,
+      @JacksonInject StoragePluginRegistry engineRegistry) throws IOException, ExecutionSetupException {
+    super(child);
+    this.plugin = (KuduStoragePlugin) engineRegistry.getPlugin(storageConfig);
+    this.name = name;
+  }
+
+
+  KuduWriter(PhysicalOperator child, String name, KuduStoragePlugin plugin) {
+    super(child);
+    this.name = name;
+    this.plugin = plugin;
+  }
+
+  @Override
+  public int getOperatorType() {
+    return 3001;
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new KuduWriter(child, name, plugin);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public StoragePluginConfig getStorage() {
+    return plugin.getConfig();
+  }
+
+  @JsonIgnore
+  public KuduStoragePlugin getPlugin() {
+    return plugin;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduWriterBatchCreator.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduWriterBatchCreator.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduWriterBatchCreator.java
new file mode 100644
index 0000000..c200c17
--- /dev/null
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduWriterBatchCreator.java
@@ -0,0 +1,43 @@
+/**
+ * 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.store.kudu;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.WriterRecordBatch;
+import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+
+public class KuduWriterBatchCreator implements BatchCreator<KuduWriter> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduWriterBatchCreator.class);
+
+  @Override
+  public CloseableRecordBatch getBatch(FragmentContext context, KuduWriter config, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    assert children != null && children.size() == 1;
+
+    return new WriterRecordBatch(config, children.iterator().next(), context, new KuduRecordWriterImpl(
+        null,
+        config.getPlugin().getClient(),
+        config.getName()));
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json
index 3ba12c0..52884a6 100644
--- a/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-kudu/src/main/resources/bootstrap-storage-plugins.json
@@ -2,8 +2,8 @@
   "storage":{
     kudu : {
       type:"kudu",
-      masterAddresses: "172.31.1.99",
-      enabled: true
+      masterAddresses: "1.2.3.4",
+      enabled: false
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/main/resources/checkstyle-config.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/resources/checkstyle-config.xml b/contrib/storage-kudu/src/main/resources/checkstyle-config.xml
deleted file mode 100644
index 74cc856..0000000
--- a/contrib/storage-kudu/src/main/resources/checkstyle-config.xml
+++ /dev/null
@@ -1,42 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!-- 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^M
-  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. -->
-<!DOCTYPE module PUBLIC
-    "-//Puppy Crawl//DTD Check Configuration 1.2//EN"
-    "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
-
-<!--  Checkstyle configuration for Apache Drill -->
-
-<module name="Checker">
-
-  <module name="TreeWalker">
-
-    <module name="IllegalImport">
-      <!-- For "org.apache.commons.lang.*" classes use the corresponding class from "org.apache.commons.lang3.*" -->
-      <property name="illegalPkgs" value="com.google.hive12,com.beust.jcommander.internal,jersey.repackaged,org.apache.commons.lang"/>
-    </module>
-
-    <module name="AvoidStarImport">
-      <property name="allowStaticMemberImports" value="true"/>
-    </module>
-
-    <module name="NeedBraces"/>
-
-  </module>
-
-  <module name="FileTabCharacter"/>
-
-  <module name="RegexpSingleline">
-    <property name="format" value="\s+$"/>
-    <property name="message" value="A line of code cannot contain any trailing whitespace"/>
-  </module>
-
-</module>

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/main/resources/checkstyle-suppressions.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/resources/checkstyle-suppressions.xml b/contrib/storage-kudu/src/main/resources/checkstyle-suppressions.xml
deleted file mode 100644
index 9d4682b..0000000
--- a/contrib/storage-kudu/src/main/resources/checkstyle-suppressions.xml
+++ /dev/null
@@ -1,19 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!-- 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. -->
-<!DOCTYPE suppressions PUBLIC
-    "-//Puppy Crawl//DTD Suppressions 1.1//EN"
-    "suppressions_1_1.dtd">
-
-<!--  Checkstyle Suppressions for Apache Drill -->
-<suppressions>
-  <suppress files="[\\/]generated-sources[\\/]" checks="AvoidStarImport,NeedBraces"/>
-</suppressions>

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
index 5f36f80..0ee0134 100644
--- a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
@@ -20,11 +20,12 @@ package org.apache.drill.store.kudu;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.junit.Ignore;
 import org.junit.Test;
 import org.kududb.ColumnSchema;
 import org.kududb.Schema;
 import org.kududb.Type;
-import org.kududb.client.CreateTableBuilder;
+import org.kududb.client.CreateTableOptions;
 import org.kududb.client.Insert;
 import org.kududb.client.KuduClient;
 import org.kududb.client.KuduScanner;
@@ -36,9 +37,7 @@ import org.kududb.client.RowResult;
 import org.kududb.client.RowResultIterator;
 import org.kududb.client.SessionConfiguration;
 
-import static org.kududb.Type.STRING;
-
-
+@Ignore("requires remote kudu server")
 public class TestKuduConnect {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestKuduConnect.class);
 
@@ -62,7 +61,7 @@ public class TestKuduConnect {
 
       Schema schema = new Schema(columns);
 
-      CreateTableBuilder builder = new CreateTableBuilder();
+      CreateTableOptions builder = new CreateTableOptions();
       builder.setNumReplicas(replicas);
       for (int i = 1; i < tablets; i++) {
         PartialRow splitRow = schema.newPartialRow();

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
index d5e138f..450a1ad 100644
--- a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
@@ -18,8 +18,10 @@
 package org.apache.drill.store.kudu;
 
 import org.apache.drill.BaseTestQuery;
+import org.junit.Ignore;
 import org.junit.Test;
 
+@Ignore("requires a remote kudu server to run.")
 public class TestKuduPlugin extends BaseTestQuery {
 
   @Test
@@ -33,4 +35,12 @@ public class TestKuduPlugin extends BaseTestQuery {
     test("show tables;");
     test("describe demo");
   }
+
+  @Test
+  public void testCreate() throws Exception {
+    test("create table kudu.regions as select 1, * from sys.options limit 1");
+    test("select * from kudu.regions");
+    test("drop table kudu.regions");
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/contrib/storage-kudu/src/test/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/resources/bootstrap-storage-plugins.json b/contrib/storage-kudu/src/test/resources/bootstrap-storage-plugins.json
new file mode 100644
index 0000000..3ba12c0
--- /dev/null
+++ b/contrib/storage-kudu/src/test/resources/bootstrap-storage-plugins.json
@@ -0,0 +1,9 @@
+{
+  "storage":{
+    kudu : {
+      type:"kudu",
+      masterAddresses: "172.31.1.99",
+      enabled: true
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/distribution/pom.xml
----------------------------------------------------------------------
diff --git a/distribution/pom.xml b/distribution/pom.xml
index 5aaf09d..135c974 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -168,6 +168,11 @@
         </dependency>
         <dependency>
           <groupId>org.apache.drill.contrib</groupId>
+          <artifactId>drill-kudu-storage</artifactId>
+          <version>${project.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.drill.contrib</groupId>
           <artifactId>drill-mongo-storage</artifactId>
           <version>${project.version}</version>
         </dependency>

http://git-wip-us.apache.org/repos/asf/drill/blob/392d1f7e/distribution/src/assemble/bin.xml
----------------------------------------------------------------------
diff --git a/distribution/src/assemble/bin.xml b/distribution/src/assemble/bin.xml
index 449ac6c..12682e2 100644
--- a/distribution/src/assemble/bin.xml
+++ b/distribution/src/assemble/bin.xml
@@ -97,6 +97,7 @@
         <include>org.apache.drill.contrib:drill-mongo-storage</include>
         <include>org.apache.drill.contrib:drill-storage-hbase</include>
         <include>org.apache.drill.contrib:drill-jdbc-storage</include>
+        <include>org.apache.drill.contrib:drill-kudu-storage</include>
         <include>org.apache.drill.contrib:drill-gis</include>
       </includes>
       <excludes>


[08/12] drill git commit: DRILL-4241: Add wait stats and estimated row count in RecordReader

Posted by ja...@apache.org.
DRILL-4241: Add wait stats and estimated row count in RecordReader


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

Branch: refs/heads/master
Commit: 54cf72724ab365831f4023b9ef14cd683f7f3561
Parents: 4ba5155
Author: Steven Phillips <sm...@apache.org>
Authored: Thu Nov 19 16:54:41 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 22:54:21 2016 -0800

----------------------------------------------------------------------
 .../drill/exec/store/kudu/KuduGroupScan.java    |  4 +++-
 .../drill/exec/store/kudu/KuduRecordReader.java | 23 +++++++++++++++-----
 2 files changed, 21 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/54cf7272/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
index 8403632..ff4295d 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
@@ -33,6 +33,7 @@ import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.kudu.KuduSubScan.KuduSubScanSpec;
@@ -210,7 +211,8 @@ public class KuduGroupScan extends AbstractGroupScan {
   // List<KuduSubScanSpec> tabletInfoList, List<SchemaPath> columns
   @Override
   public ScanStats getScanStats() {
-    return ScanStats.TRIVIAL_TABLE;
+    long recordCount = 100000 * kuduWorkList.size();
+    return new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, recordCount, 1, recordCount);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/54cf7272/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
index 6b7877d..7f7fc35 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
@@ -77,6 +77,7 @@ public class KuduRecordReader extends AbstractRecordReader {
   private RowResultIterator iterator;
   
   private OutputMutator output;
+  private OperatorContext context;
 
   private static class ProjectedColumnInfo {
     int index;
@@ -97,6 +98,7 @@ public class KuduRecordReader extends AbstractRecordReader {
   @Override
   public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
     this.output = output;
+    this.context = context;
     try {
       KuduTable table = client.openTable(scanSpec.getTableName());
       
@@ -108,10 +110,16 @@ public class KuduRecordReader extends AbstractRecordReader {
         }
         builder.setProjectedColumnNames(colNames);
       }
-      scanner = builder
-              .lowerBoundPartitionKeyRaw(scanSpec.getStartKey())
-              .exclusiveUpperBoundPartitionKeyRaw(scanSpec.getEndKey())
-              .build();
+
+      context.getStats().startWait();
+      try {
+        scanner = builder
+                .lowerBoundPartitionKeyRaw(scanSpec.getStartKey())
+                .exclusiveUpperBoundPartitionKeyRaw(scanSpec.getEndKey())
+                .build();
+      } finally {
+        context.getStats().stopWait();
+      }
     } catch (Exception e) {
       throw new ExecutionSetupException(e);
     }
@@ -143,7 +151,12 @@ public class KuduRecordReader extends AbstractRecordReader {
           iterator = null;
           return 0;
         }
-        iterator = scanner.nextRows();
+        context.getStats().startWait();
+        try {
+          iterator = scanner.nextRows();
+        } finally {
+          context.getStats().stopWait();
+        }
       }
       for (; rowCount < TARGET_RECORD_COUNT && iterator.hasNext(); rowCount++) {
         addRowResult(iterator.next(), rowCount);


[02/12] drill git commit: DRILL-4241: Fixing the build, make RAT and checkstyle happy.

Posted by ja...@apache.org.
DRILL-4241: Fixing the build, make RAT and checkstyle happy.


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

Branch: refs/heads/master
Commit: 805e0416106139e6686672aed2d2cd600bc4e36d
Parents: 5dfb451
Author: Jason Altekruse <al...@gmail.com>
Authored: Sun Jan 10 22:50:05 2016 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 22:53:06 2016 -0800

----------------------------------------------------------------------
 .../src/main/resources/checkstyle-config.xml    | 42 ++++++++++++
 .../main/resources/checkstyle-suppressions.xml  | 19 ++++++
 .../drill/store/kudu/TestKuduConnect.java       | 17 +++++
 .../apache/drill/store/kudu/TestKuduPlugin.java | 17 +++++
 .../drill/store/kudu/TestKuduTableProvider.java | 72 +-------------------
 5 files changed, 97 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/805e0416/contrib/storage-kudu/src/main/resources/checkstyle-config.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/resources/checkstyle-config.xml b/contrib/storage-kudu/src/main/resources/checkstyle-config.xml
new file mode 100644
index 0000000..74cc856
--- /dev/null
+++ b/contrib/storage-kudu/src/main/resources/checkstyle-config.xml
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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^M
+  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. -->
+<!DOCTYPE module PUBLIC
+    "-//Puppy Crawl//DTD Check Configuration 1.2//EN"
+    "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
+
+<!--  Checkstyle configuration for Apache Drill -->
+
+<module name="Checker">
+
+  <module name="TreeWalker">
+
+    <module name="IllegalImport">
+      <!-- For "org.apache.commons.lang.*" classes use the corresponding class from "org.apache.commons.lang3.*" -->
+      <property name="illegalPkgs" value="com.google.hive12,com.beust.jcommander.internal,jersey.repackaged,org.apache.commons.lang"/>
+    </module>
+
+    <module name="AvoidStarImport">
+      <property name="allowStaticMemberImports" value="true"/>
+    </module>
+
+    <module name="NeedBraces"/>
+
+  </module>
+
+  <module name="FileTabCharacter"/>
+
+  <module name="RegexpSingleline">
+    <property name="format" value="\s+$"/>
+    <property name="message" value="A line of code cannot contain any trailing whitespace"/>
+  </module>
+
+</module>

http://git-wip-us.apache.org/repos/asf/drill/blob/805e0416/contrib/storage-kudu/src/main/resources/checkstyle-suppressions.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/resources/checkstyle-suppressions.xml b/contrib/storage-kudu/src/main/resources/checkstyle-suppressions.xml
new file mode 100644
index 0000000..9d4682b
--- /dev/null
+++ b/contrib/storage-kudu/src/main/resources/checkstyle-suppressions.xml
@@ -0,0 +1,19 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<!DOCTYPE suppressions PUBLIC
+    "-//Puppy Crawl//DTD Suppressions 1.1//EN"
+    "suppressions_1_1.dtd">
+
+<!--  Checkstyle Suppressions for Apache Drill -->
+<suppressions>
+  <suppress files="[\\/]generated-sources[\\/]" checks="AvoidStarImport,NeedBraces"/>
+</suppressions>

http://git-wip-us.apache.org/repos/asf/drill/blob/805e0416/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
index a6fc69c..d5c6c51 100644
--- a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
@@ -1,3 +1,20 @@
+/**
+ * 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.store.kudu;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/drill/blob/805e0416/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
index 1a6211f..97b2212 100644
--- a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
@@ -1,3 +1,20 @@
+/**
+ * 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.store.kudu;
 
 import org.apache.drill.BaseTestQuery;

http://git-wip-us.apache.org/repos/asf/drill/blob/805e0416/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java
index ca8f9e1..f5b3ead 100644
--- a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java
@@ -17,75 +17,7 @@
  */
 package org.apache.drill.store.kudu;
 
-import static org.junit.Assert.assertEquals;
+import org.apache.drill.BaseTestQuery;
+public class TestKuduTableProvider extends BaseTestQuery {
 
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.drill.common.config.LogicalPlanPersistence;
-import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
-import org.apache.drill.exec.store.kudu.config.KuduPStoreProvider;
-import org.apache.drill.exec.store.sys.PStore;
-import org.apache.drill.exec.store.sys.PStoreConfig;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class TestKuduTableProvider extends BaseKuduTest {
-
-  private static KuduPStoreProvider provider;
-
-  @BeforeClass // mask Kudu cluster start function
-  public static void setUpBeforeTestKuduTableProvider() throws Exception {
-    provider = new KuduPStoreProvider(storagePluginConfig.getKuduConf(), "drill_store");
-    provider.start();
-  }
-
-  @Test
-  public void testTableProvider() throws IOException {
-    LogicalPlanPersistence lp = PhysicalPlanReaderTestFactory.defaultLogicalPlanPersistence(config);
-    PStore<String> kuduStore = provider.getStore(PStoreConfig.newJacksonBuilder(lp.getMapper(), String.class).name("kudu").build());
-    kuduStore.put("", "v0");
-    kuduStore.put("k1", "v1");
-    kuduStore.put("k2", "v2");
-    kuduStore.put("k3", "v3");
-    kuduStore.put("k4", "v4");
-    kuduStore.put("k5", "v5");
-    kuduStore.put(".test", "testValue");
-
-    assertEquals("v0", kuduStore.get(""));
-    assertEquals("testValue", kuduStore.get(".test"));
-
-    int rowCount = 0;
-    for (Entry<String, String> entry : kuduStore) {
-      rowCount++;
-      System.out.println(entry.getKey() + "=" + entry.getValue());
-    }
-    assertEquals(7, rowCount);
-
-    PStore<String> kuduTestStore = provider.getStore(PStoreConfig.newJacksonBuilder(lp.getMapper(), String.class).name("kudu.test").build());
-    kuduTestStore.put("", "v0");
-    kuduTestStore.put("k1", "v1");
-    kuduTestStore.put("k2", "v2");
-    kuduTestStore.put("k3", "v3");
-    kuduTestStore.put("k4", "v4");
-    kuduTestStore.put(".test", "testValue");
-
-    assertEquals("v0", kuduStore.get(""));
-    assertEquals("testValue", kuduStore.get(".test"));
-
-    rowCount = 0;
-    for (Entry<String, String> entry : kuduTestStore) {
-      rowCount++;
-      System.out.println(entry.getKey() + "=" + entry.getValue());
-    }
-    assertEquals(6, rowCount);
-  }
-
-  @AfterClass
-  public static void tearDownTestKuduTableProvider() {
-    if (provider != null) {
-      provider.close();
-    }
-  }
 }


[07/12] drill git commit: DRILL-4241: Add table metadata and DROP table support

Posted by ja...@apache.org.
DRILL-4241: Add table metadata and DROP table support


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

Branch: refs/heads/master
Commit: 697579baca1755e3c7185d0ed11dd0d3073ca253
Parents: 2c0e53a
Author: Jacques Nadeau <ja...@apache.org>
Authored: Thu Nov 19 14:21:14 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 22:54:21 2016 -0800

----------------------------------------------------------------------
 .../drill/exec/store/kudu/DrillKuduTable.java   | 48 +++++++++++++++++++-
 .../exec/store/kudu/KuduSchemaFactory.java      | 30 +++++++++++-
 .../drill/store/kudu/TestKuduConnect.java       |  8 ++--
 .../apache/drill/store/kudu/TestKuduPlugin.java |  7 +++
 .../drill/store/kudu/TestKuduTableProvider.java | 23 ----------
 5 files changed, 86 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/697579ba/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/DrillKuduTable.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/DrillKuduTable.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/DrillKuduTable.java
index 3b9c757..3fc69c6 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/DrillKuduTable.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/DrillKuduTable.java
@@ -17,20 +17,64 @@
  */
 package org.apache.drill.exec.store.kudu;
 
+import java.util.List;
+
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.drill.exec.planner.logical.DynamicDrillTable;
+import org.kududb.ColumnSchema;
+import org.kududb.Schema;
+import org.kududb.Type;
+
+import com.google.common.collect.Lists;
 
 public class DrillKuduTable extends DynamicDrillTable {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillKuduTable.class);
 
-  public DrillKuduTable(String storageEngineName, KuduStoragePlugin plugin, KuduScanSpec scanSpec) {
+  private final Schema schema;
+
+  public DrillKuduTable(String storageEngineName, KuduStoragePlugin plugin, Schema schema, KuduScanSpec scanSpec) {
     super(plugin, storageEngineName, scanSpec);
+    this.schema = schema;
   }
 
   @Override
   public RelDataType getRowType(RelDataTypeFactory typeFactory) {
-    return super.getRowType(typeFactory);
+
+    List<String> names = Lists.newArrayList();
+    List<RelDataType> types = Lists.newArrayList();
+    for (ColumnSchema column : schema.getColumns()) {
+      names.add(column.getName());
+      RelDataType type = getSqlTypeFromKuduType(typeFactory, column.getType());
+      type = typeFactory.createTypeWithNullability(type, column.isNullable());
+      types.add(type);
+    }
+
+    return typeFactory.createStructType(types, names);
   }
 
+  private RelDataType getSqlTypeFromKuduType(RelDataTypeFactory typeFactory, Type type) {
+    switch (type) {
+    case BINARY:
+      return typeFactory.createSqlType(SqlTypeName.VARBINARY, Integer.MAX_VALUE);
+    case BOOL:
+      return typeFactory.createSqlType(SqlTypeName.BOOLEAN);
+    case DOUBLE:
+      return typeFactory.createSqlType(SqlTypeName.DOUBLE);
+    case FLOAT:
+      return typeFactory.createSqlType(SqlTypeName.FLOAT);
+    case INT16:
+    case INT32:
+    case INT64:
+    case INT8:
+      return typeFactory.createSqlType(SqlTypeName.INTEGER);
+    case STRING:
+      return typeFactory.createSqlType(SqlTypeName.VARCHAR, Integer.MAX_VALUE);
+    case TIMESTAMP:
+      return typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
+    default:
+      throw new UnsupportedOperationException("Unsupported type.");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/697579ba/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
index 294eabe..7ea4f2f 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
@@ -23,9 +23,12 @@ import java.util.Set;
 
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.SchemaFactory;
+import org.kududb.Schema;
+import org.kududb.client.KuduTable;
 import org.kududb.client.ListTablesResponse;
 
 import com.google.common.collect.ImmutableList;
@@ -71,7 +74,15 @@ public class KuduSchemaFactory implements SchemaFactory {
     @Override
     public Table getTable(String name) {
       KuduScanSpec scanSpec = new KuduScanSpec(name);
-      return new DrillKuduTable(schemaName, plugin, scanSpec);
+      try {
+        KuduTable table = plugin.getClient().openTable(name);
+        Schema schema = table.getSchema();
+        return new DrillKuduTable(schemaName, plugin, schema, scanSpec);
+      } catch (Exception e) {
+        logger.warn("Failure while retrieving kudu table {}", name, e);
+        return null;
+      }
+
     }
 
     @Override
@@ -86,6 +97,23 @@ public class KuduSchemaFactory implements SchemaFactory {
     }
 
     @Override
+    public void dropTable(String tableName) {
+      try {
+        plugin.getClient().deleteTable(tableName);
+      } catch (Exception e) {
+        throw UserException.dataWriteError(e)
+            .message("Failure while trying to drop table '%s'.", tableName)
+            .addContext("plugin", name)
+            .build(logger);
+      }
+    }
+
+    @Override
+    public boolean isMutable() {
+      return true;
+    }
+
+    @Override
     public String getTypeName() {
       return KuduStoragePluginConfig.NAME;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/697579ba/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
index d5c6c51..7fac9ad 100644
--- a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduConnect.java
@@ -54,10 +54,10 @@ public class TestKuduConnect {
 
       List<ColumnSchema> columns = new ArrayList<>(5);
       columns.add(new ColumnSchema.ColumnSchemaBuilder("key", Type.INT32).key(true).build());
-      columns.add(new ColumnSchema.ColumnSchemaBuilder("binary", Type.BINARY).build());
-      columns.add(new ColumnSchema.ColumnSchemaBuilder("boolean", Type.BOOL).build());
-      columns.add(new ColumnSchema.ColumnSchemaBuilder("float", Type.FLOAT).build());
-      columns.add(new ColumnSchema.ColumnSchemaBuilder("string", Type.STRING).build());
+      columns.add(new ColumnSchema.ColumnSchemaBuilder("binary", Type.BINARY).nullable(false).build());
+      columns.add(new ColumnSchema.ColumnSchemaBuilder("boolean", Type.BOOL).nullable(true).build());
+      columns.add(new ColumnSchema.ColumnSchemaBuilder("float", Type.FLOAT).nullable(false).build());
+      columns.add(new ColumnSchema.ColumnSchemaBuilder("string", Type.STRING).nullable(true).build());
 
       Schema schema = new Schema(columns);
       client.createTable(KUDU_TABLE, schema);

http://git-wip-us.apache.org/repos/asf/drill/blob/697579ba/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
index 97b2212..d5e138f 100644
--- a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
+++ b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduPlugin.java
@@ -26,4 +26,11 @@ public class TestKuduPlugin extends BaseTestQuery {
   public void testBasicQuery() throws Exception {
     test("select * from kudu.demo;");
   }
+
+  @Test
+  public void testDescribe() throws Exception {
+    test("use kudu;");
+    test("show tables;");
+    test("describe demo");
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/697579ba/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java b/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java
deleted file mode 100644
index f5b3ead..0000000
--- a/contrib/storage-kudu/src/test/java/org/apache/drill/store/kudu/TestKuduTableProvider.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.store.kudu;
-
-import org.apache.drill.BaseTestQuery;
-public class TestKuduTableProvider extends BaseTestQuery {
-
-}


[11/12] drill git commit: DRILL-4241: Timestamps should divide by 1000 in reader

Posted by ja...@apache.org.
DRILL-4241: Timestamps should divide by 1000 in reader


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

Branch: refs/heads/master
Commit: 3694909662d62772ea146c073bdb2d6f10e4150e
Parents: 54cf727
Author: Todd Lipcon <to...@cloudera.com>
Authored: Thu Nov 19 16:19:45 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Jan 10 22:54:21 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/drill/exec/store/kudu/KuduRecordReader.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/36949096/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
index 7f7fc35..a97df77 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduRecordReader.java
@@ -315,10 +315,10 @@ public class KuduRecordReader extends AbstractRecordReader {
       case TIMESTAMP:
         if (pci.kuduColumn.isNullable()) {
           ((NullableTimeStampVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getLong(pci.index));
+            .setSafe(rowIndex, result.getLong(pci.index) / 1000);
         } else {
           ((TimeStampVector.Mutator)pci.vv.getMutator())
-            .setSafe(rowIndex, result.getLong(pci.index));
+            .setSafe(rowIndex, result.getLong(pci.index) / 1000);
         }
         break;
       default: