Return-Path: X-Original-To: apmail-drill-commits-archive@www.apache.org Delivered-To: apmail-drill-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 8B162180E9 for ; Mon, 11 Jan 2016 07:53:50 +0000 (UTC) Received: (qmail 88799 invoked by uid 500); 11 Jan 2016 07:53:50 -0000 Delivered-To: apmail-drill-commits-archive@drill.apache.org Received: (qmail 88743 invoked by uid 500); 11 Jan 2016 07:53:50 -0000 Mailing-List: contact commits-help@drill.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: commits@drill.apache.org Delivered-To: mailing list commits@drill.apache.org Received: (qmail 88662 invoked by uid 99); 11 Jan 2016 07:53:50 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 11 Jan 2016 07:53:50 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id EF370E048E; Mon, 11 Jan 2016 07:53:49 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jacques@apache.org To: commits@drill.apache.org Date: Mon, 11 Jan 2016 07:53:55 -0000 Message-Id: <3adf1ddbfaea48c18211f6c0da3d4960@git.apache.org> In-Reply-To: <90208e6d24734ec8a04a65532abf2713@git.apache.org> References: <90208e6d24734ec8a04a65532abf2713@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [07/12] drill git commit: DRILL-4241: Add table metadata and DROP table support 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 Authored: Thu Nov 19 14:21:14 2015 -0800 Committer: Jacques Nadeau 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 names = Lists.newArrayList(); + List 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 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 { - -}