You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@impala.apache.org by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org> on 2016/09/14 16:47:49 UTC

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Dimitris Tsirogiannis has uploaded a new change for review.

  http://gerrit.cloudera.org:8080/4414

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
  columns in tblproperties.
2) Read table schema from Kudu. When attempting to create an external
  table "foo" in database "bar", Impala will search for a Kudu table
  name "foo.bar" and "bar" (Kudu doesn't have database name spaces
  yet.)
3) The Kudu table is now required to exist at the time of creation in
  Impala.
4) Disallow table properties that could conflict with an existing
  table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
  addresses. The flag is used as the default value for the table
  property kudu_master_addresses.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
  wasn't implemented for Kudu tables and silently ignored. The Kudu
  tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
  Kudu) the existence of the other delegate and the use of delegates in
  general has led to confusion. The Kudu delegate only exists to provide
  functionality missing from Hive. Eventually Hive should have the needed
  functionality and the Kudu delegate (renamed in this patch to KuduCatalogOpExecutor)
  can be removed.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
  standard. When used at the column level, only one column can be
  marked as a key. When used at the table level, multiple columns can
  be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
  The old "kudu.key_columns" table property is no longer accepted
  though it is still used internally. "PRIMARY" is now a keyword.
  "KEY" is expected to be common enough that the ident style
  declaration is used instead to avoid conflicts.
10) Infer a Kudu table name if none was given. The table property
  "kudu.table_name" is now optional. If not given, the Kudu table name
  will be created based on the Hive Metastore database and table name.
  "CREATE TABLE foo.bar (i INT PRIMARY KEY) STORED AS KUDU" will create
  a table in Kudu named "foo.bar". If the database is "default" then
  the Kudu table name will not include the database.
11) Several improvements in the grammar related to the family
  of CREATE TABLE statements.
12) Added new tests and modified existing Kudu test to use the new
  CREATE TABLE syntax.
13) Use Kudu master as the source of truth for table metadata insteads
  of HMS. Table/column metadata are still stored in HMS in order to be
  able to use table and column statistics.

Not included in this commit:
- Additional column properties such as nullability and compression
encodings.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/catalog/catalog.cc
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/com/cloudera/impala/analysis/AnalysisUtils.java
M fe/src/main/java/com/cloudera/impala/analysis/ColumnDef.java
M fe/src/main/java/com/cloudera/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/com/cloudera/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/com/cloudera/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/com/cloudera/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java
M fe/src/main/java/com/cloudera/impala/analysis/DistributeParam.java
A fe/src/main/java/com/cloudera/impala/analysis/TableDataLayout.java
A fe/src/main/java/com/cloudera/impala/analysis/TableDef.java
A fe/src/main/java/com/cloudera/impala/analysis/TableDefOptions.java
M fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java
M fe/src/main/java/com/cloudera/impala/catalog/Catalog.java
M fe/src/main/java/com/cloudera/impala/catalog/Db.java
M fe/src/main/java/com/cloudera/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/com/cloudera/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java
M fe/src/main/java/com/cloudera/impala/catalog/Table.java
D fe/src/main/java/com/cloudera/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/com/cloudera/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/com/cloudera/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/com/cloudera/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/com/cloudera/impala/service/CatalogOpExecutor.java
M fe/src/main/java/com/cloudera/impala/service/Frontend.java
M fe/src/main/java/com/cloudera/impala/service/JniCatalog.java
M fe/src/main/java/com/cloudera/impala/service/JniFrontend.java
A fe/src/main/java/com/cloudera/impala/service/KuduCatalogOpExecutor.java
A fe/src/main/java/com/cloudera/impala/util/KuduClient.java
M fe/src/main/java/com/cloudera/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/com/cloudera/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/com/cloudera/impala/analysis/ParserTest.java
M fe/src/test/java/com/cloudera/impala/service/JdbcTest.java
M fe/src/test/java/com/cloudera/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/query_test/test_kudu.py
56 files changed, 2,651 insertions(+), 1,870 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/1
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 1
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 12:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/4414/12/fe/src/main/java/org/apache/impala/analysis/TableDef.java
File fe/src/main/java/org/apache/impala/analysis/TableDef.java:

Line 280:     if (options_.fileFormat == THdfsFileFormat.KUDU) {
> This doesn't seem right. We used to be able to change the ROW FORMAT for TE
As we talked offline, I reverted this function to what used to be with the addition of an extra check for Kudu tables that don't support ROW FORMAT. Done


http://gerrit.cloudera.org:8080/#/c/4414/12/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

Line 1209:       org.apache.hadoop.hive.metastore.api.Table msTbl = existingTbl.getMetaStoreTable();
> Should we only do this if existingTbl is not loaded?
As we talked earlier, this handles the most common cases. If an HMS table is dropped externally, then we have no way of deleting the underlying Kudu table as the needed information is stored in the tblproperties of msTbl.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 12
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Michael Brown (Code Review)" <ge...@cloudera.org>.
Michael Brown has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 9: Code-Review+1

for the python tests

-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 9
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 1:

(9 comments)

... a few more things

http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java
File fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java:

PS1, Line 223: Preconditions.checkNotNull(kuduDistributeByParams);
This breaks for tables that were created before this change.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java
File fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java:

PS1, Line 87: 
            :   // Key to specify the number of tablet replicas.
            :   // TODO(KUDU): Allow modification in alter table.
            :   public static final String KEY_TABLET_REPLICAS = "kudu.num_tablet_replicas";
I don't think we need to store this, even if we wanted to modify this later. Further, it doesn't look like the API exposes a way to do that anyway.

Can you add a TODO or remove this?


PS1, Line 92:   public static final String KEY_DISTRIBUTE_BY = "kudu.distribute_by";
I'm not sure we should be storing this. The Kudu client exposes KuduTable.getPartitionSchema(). It looks like it's just for use by SHOW CREATE TABLE, and it'd be good for us to use the PartitionSchema from them which we should probably be able to stringify.

I'm OK with leaving that for later as long as you think it won't be an issue for dev environments to upgrade between state now -> this review as-is -> making the change i suggested


http://gerrit.cloudera.org:8080/#/c/4414/1/testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
File testdata/workloads/functional-query/queries/QueryTest/create_kudu.test:

we might still need to have explicit create table tests.

Specifically we should test that you can create an internal table with a different kudu table name by overriding 'kudu.table_name'. We could basically modify the managed_kudu table test case that's here to do so.

I think we need to find a way to modify the kudu schema (e.g. drop and add columns) and see if we handle gracefully. Seems OK to leave that as a future task, but we should at least know how we handle those cases.


http://gerrit.cloudera.org:8080/#/c/4414/1/testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
File testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test:

well, we should have some coverage of show create table...


http://gerrit.cloudera.org:8080/#/c/4414/1/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test:

PS1, Line 15: 
            : 
            : 
            : 
            : 
            : 
            : 
            : 
            : 
            : 
            : 
I think this kind of error should still be covered. You can still mismatch PK cols and dist by columns.


PS1, Line 37: 
            : 
we should still cover this as well.


PS1, Line 50: 
            : 
same


Some more questions about losing coverage. Fine if this is now covered by FE tests.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 1
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 3:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/4414/3/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

PS3, Line 350: // Parameters needed for hash distribution
             : struct TDistributeByHashParam {
             :   1: required list<string> columns
             :   2: required i32 num_buckets
             : }
             : 
             : struct TRangeLiteral {
             :   1: optional i64 int_literal
             :   2: optional string string_literal
             : }
             : 
             : struct TRangeLiteralList {
             :   1: required list<TRangeLiteral> values
             : }
             : 
             : // A range distribution is identified by a list of columns and a series of split rows.
             : struct TDistributeByRangeParam {
             :   1: required list<string> columns
             :   2: optional list<TRangeLiteralList> split_rows;
             : }
             : 
             : // Parameters for the DISTRIBUTE BY clause. The actual distribution is identified by
             : // the type parameter.
             : struct TDistributeParam {
             :   // Set if type is set to HASH
             :   1: optional TDistributeByHashParam by_hash_param;
             : 
             :   // Set if type is set to RANGE
             :   2: optional TDistributeByRangeParam by_range_param;
             : }
> I don't see why this needs to be in the TKuduTable though. I think we just 
I consider the distribution scheme to be part of table metadata that are loaded in the catalog. Hence, they live in the KuduTable object and need to be serialized before they get sent to all the impalad nodes. Currently, the distribution schemes are only used for the SHOW CREATE TABLE statement. They should always be set in a TKuduTable. The part that is not set if the list of split values because they are not available from the Kudu API.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 3
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 3:

(11 comments)

Not even close to done but want to start giving you feedback in pieces so we can parallelize a bit.

http://gerrit.cloudera.org:8080/#/c/4414/3/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

PS3, Line 344: // Enum listing all possible DISTRIBUTE BY types
             : enum TDistributeType {
             :   HASH,
             :   RANGE,
I don't see this used


PS3, Line 350: // Parameters needed for hash distribution
             : struct TDistributeByHashParam {
             :   1: required list<string> columns
             :   2: required i32 num_buckets
             : }
             : 
             : struct TRangeLiteral {
             :   1: optional i64 int_literal
             :   2: optional string string_literal
             : }
             : 
             : struct TRangeLiteralList {
             :   1: required list<TRangeLiteral> values
             : }
             : 
             : // A range distribution is identified by a list of columns and a series of split rows.
             : struct TDistributeByRangeParam {
             :   1: required list<string> columns
             :   2: optional list<TRangeLiteralList> split_rows;
             : }
             : 
             : // Parameters for the DISTRIBUTE BY clause. The actual distribution is identified by
             : // the type parameter.
             : struct TDistributeParam {
             :   // Set if type is set to HASH
             :   1: optional TDistributeByHashParam by_hash_param;
             : 
             :   // Set if type is set to RANGE
             :   2: optional TDistributeByRangeParam by_range_param;
             : }
I see that these are used in the serialized catalog objects, but given that none of this is stored in the metastore nor do we require it to be stored in the catalog, I don't see why we bother keeping all this in the thrift table object.

Can we just remove it and not serialize this info? Hopefully we can even split the distribute/partition parameters out of CREATE TABLE eventually, anyway.


PS3, Line 390: 
             :   // Distribution schemes
             :   4: required list<TDistributeParam> distribute_by
same for this, can we get rid of it?


http://gerrit.cloudera.org:8080/#/c/4414/3/common/thrift/JniCatalog.thrift
File common/thrift/JniCatalog.thrift:

PS3, Line 398: CatalogObjects.TDistributeParam
wrt my comments in CatalogObjects, I guess we'd need them here, but I don't see why we bother storing them in the serialized catalog table object.


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/ColumnDefOptions.java
File fe/src/main/java/org/apache/impala/analysis/ColumnDefOptions.java:

PS3, Line 21:  * Placeholder for the column definition options of a CREATE TABLE statement.
            :  * Contains the list of column definitions and, optionally, the list of column names
            :  * specified using the PRIMARY KEY keyword.
this feels like a weird abstraction... do we really need this class?


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
File fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java:

PS3, Line 24: class TableDataLayout {
            : 
            :   private final List<ColumnDef> partitionColDefs_;
            :   private final List<DistributeParam> distributeParams_;
Not that we're doing this in this review, but we need to think about how this might change when we do the next steps for Kudu partitioning.


PS3, Line 49:   List<ColumnDef> getPartitionColumnDefs() { return partitionColDefs_; }
            :   List<DistributeParam> getDistributeParams() { return distributeParams_; }
any reason these aren't public?


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/TableDef.java
File fe/src/main/java/org/apache/impala/analysis/TableDef.java:

PS3, Line 1: // Copyright 2016 Cloudera Inc.
           : //
           : // Licensed 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 and maybe others need the new license header:

// 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.


PS3, Line 126: return fullyQualifiedTableName_ != null ? fullyQualifiedTableName_ : tableName_;
Do we need to support calling this before analysis? It'd be nice if we just have 1 possible result here, e.g. throw if fullyQualifiedTableName_ is null (not yet analyzed) and just return fullyQualifiedTableName_


PS3, Line 137:   List<DistributeParam> getDistributeParams() {
             :     return dataLayout_.getDistributeParams();
             :   }
1line?


http://gerrit.cloudera.org:8080/#/c/4414/3/infra/python/deps/requirements.txt
File infra/python/deps/requirements.txt:

PS3, Line 83: The kudu-python
            : # version in download_requirements must be kept in sync with this version.
gotta update download_requirements as well


diff --git a/infra/python/deps/download_requirements b/infra/python/deps/download_requirements
index daa5025..d586104 100755
--- a/infra/python/deps/download_requirements
+++ b/infra/python/deps/download_requirements
@@ -29,5 +29,5 @@ PY26="$(./find_py26.py)"
 "$PY26" pip_download.py virtualenv 13.1.0
 # kudu-python is downloaded separately because pip install attempts to execute a
 # setup.py subcommand for kudu-python that can fail even if the download succeeds.
-"$PY26" pip_download.py kudu-python 0.1.1
+"$PY26" pip_download.py kudu-python 0.2.0


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 3
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Alex Behm (Code Review)" <ge...@cloudera.org>.
Alex Behm has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 9:

(3 comments)

I'll wait for the next change set and then do a final round over everything.

http://gerrit.cloudera.org:8080/#/c/4414/9/fe/src/main/cup/sql-parser.cup
File fe/src/main/cup/sql-parser.cup:

Line 1047: // Used for creating tables where the schema is inferred externally, e.g., from an Avro 
whitespace


http://gerrit.cloudera.org:8080/#/c/4414/9/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java:

Line 1719:     // bucket is partitioned into 4 tables based on the split points of 'y'.
tables == tablets?


http://gerrit.cloudera.org:8080/#/c/4414/9/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_create.test:

Line 82: create table `add`(`analytic` int primary key) distribute by hash (`analytic`)
also cover the range() clause in this same test


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 9
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Alex Behm (Code Review)" <ge...@cloudera.org>.
Alex Behm has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 7:

(26 comments)

Next round over the code.

http://gerrit.cloudera.org:8080/#/c/4414/7/be/src/service/frontend.cc
File be/src/service/frontend.cc:

Line 62:     "value should be a comma separated list of hostnames or IP addresses.");
are ports optional or mandatory?


http://gerrit.cloudera.org:8080/#/c/4414/7/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

Line 358:   1: required list<TRangeLiteral> values
Why not a list of TExpr that are expected to be literals? Seems more future proof.


Line 368: // the type parameter.
which type parameter?


http://gerrit.cloudera.org:8080/#/c/4414/7/common/thrift/JniCatalog.thrift
File common/thrift/JniCatalog.thrift:

Line 398:   14: optional list<CatalogObjects.TDistributeParam> distribute_by;
for consistency let's remove trailing ";"


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/cup/sql-parser.cup
File fe/src/main/cup/sql-parser.cup:

Line 1033: // class doesn't inherit from CreateTableStmt. 
whitespace


Line 1047: // Used for creating external Kudu tables for which the schema is loaded from Kudu.
There seem to be more uses of this production, so this comment could be misleading. Maybe generalize to something like
"Used for creating tables where the schema is inferred externally, e.g., from an Avro schema, Kudu table or query statement."


Line 1112: // or one RANGE clause
typo: clauses


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
File fe/src/main/java/org/apache/impala/analysis/ColumnDef.java:

Line 93:   void setIsPrimaryKey() { isPrimaryKey_ = true; }
do we need this?


Line 191:       Preconditions.checkState(!colDefsByColName.containsKey(colDef.getColName()));
can check return value of put()


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java:

Line 208:    * Analyzes and checks table properties which are common for both managed and external
typo: common to


Line 255:         "PARTITIONED BY cannot be used with an Kudu table.");
typo: a Kudu table

this also needs to be checked for managed tables right?


Line 273:     AnalysisUtils.throwIfNullOrEmpty(getPrimaryKeyColumnDefs(),
Shouldn't this check hasPrimaryKey()?


Line 284:             "zero. Given number of replicas is: " + r.toString() + ".'");
remove trailing .' or add the opening single-quote


Line 318:   private boolean hasPrimaryKey() {
Isn't it enough to check primaryKeyColDefs_ in tableDef_?


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
File fe/src/main/java/org/apache/impala/analysis/DistributeParam.java:

Line 105:     for (String colName: colNames_) {
we could specify the same distribution column multiple times


Line 127:             throw new AnalysisException("Split values cannot be NULL");
do we have a test for this?


Line 223:     colNames_.addAll(colNames);
do we need toLower()?


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/analysis/TableDef.java
File fe/src/main/java/org/apache/impala/analysis/TableDef.java:

Line 69:   // Populated during analysis.
Authoritative list of primary key column definitions populated during analysis.


Line 177:     fqTableName_.analyze();
Do you know if Kudu has more permissive or more restrictive constraints on what strings can be used as table/column names? I'd be surprised if HMS and Kudu were identical in that respect. Better to file a JIRA and leave that investigation/fix out of this patch.


Line 181:     if (analyzer.dbContainsTable(getTblName().getDb(), getTbl(), Privilege.CREATE)
Are we going to check Sentry privs for Kudu tables? Also ok to defer this fix, but let's not forget.


Line 220:    * Analyzes the primary key columns. Primary keys are only supported for Kudu
Replace the second sentence with a brief description what this checks. It does not check the format and succeeds if no primary keys are given, so there is nothing Kudu specific here (that is checked in CreateTableStmt).


Line 234:       StringBuilder columnDefStr = new StringBuilder();
Not used?


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

Line 1117:     if (db != null && params.cascade) dropTablesFromKudu(db);
I think it might be a good idea to do this under the metastoreDdlLock_ as well to ensure that the Kudu and HMS table deletions are atomic.


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java:

Line 47: /**
newline before


Line 231:     } catch (Exception e) {
Did you address my comment more nuanced checking here to distinguish connection issues from "table does not exist"?


http://gerrit.cloudera.org:8080/#/c/4414/7/tests/query_test/test_kudu.py
File tests/query_test/test_kudu.py:

Line 204:         CREATE TABLE {table} (c INT PRIMARY KEY)
add comment to the PK column


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 7
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 1:

(28 comments)

Ok, here's a bit to get started. Just made my way through most of analysis. Still have a lot to get through but figured I'll give you the feedback I have while I do a few other things.

http://gerrit.cloudera.org:8080/#/c/4414/1//COMMIT_MSG
Commit Message:

PS1, Line 21: When attempting to create an external
            :   table "foo" in database "bar", Impala will search for a Kudu table
            :   name "foo.bar" and "bar" (Kudu doesn't have database name spaces
            :   yet.)
I think this sentence is duplicated by #10. Maybe remove this part. This item is about reading the columns, right? The issue of table name resolution is separate IMO, and covered by #10, though consider moving them next together :)


PS1, Line 25:  The Kudu table is now required to exist at the time of creation in
            :   Impala.
for external tables? or if this is more general can you explain a bit more?


PS1, Line 31: default value
can it still be overridden?


PS1, Line 39: Eventually Hive should have the needed
            :   functionality and the Kudu delegate (renamed in this patch to KuduCatalogOpExecutor)
            :   can be removed.
maybe, it's not clear if impala would want to keep the ability to do this on its own... Remove?


PS1, Line 54:  If the database is "default" then
            :   the Kudu table name will not include the database.
hm... I wonder if it'd be easier just to keep it standardized and always have the name, i.e. using default in this case. What do you think? I'll see if I change my mind after reading more code.


PS1, Line 56: 11) Several improvements in the grammar related to the family
            :   of CREATE TABLE statements.
I don't think this bullet adds much value to the commit message. Everything else is more user visible, this is just some necessary code cleanup.


PS1, Line 58: 12) Added new tests and modified existing Kudu test to use the new
            :   CREATE TABLE syntax.
I think this can be removed too, it should be implied there are tests and it takes away from the billion other "features" in this patch :)


PS1, Line 60: source of truth
Does this mean we always hit Kudu for metadata? In the Catalog? In the impalad's Catalog?


PS1, Line 60: insteads
instead


PS1, Line 64: Not included in this commit:
            : - Additional column properties such as nullability and compression
            : encodings.
I don't think you need to include this, we have a separate JIRA for this.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/cup/sql-parser.cup
File fe/src/main/cup/sql-parser.cup:

FYI I'm not looking at this very carefully because it was reviewed previously https://gerrit.cloudera.org/#/c/2865/

Let me know if this has changed much since then.


PS1, Line 405:   view_column_def_list, view_column_defs;
separate nonterminals if they dont fit?


PS1, Line 458: key_ident;
does this not need Boolean like those below?


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java
File fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java:

I'm not in love with the TableDef and TableDefOptions stuff because it feels like that's parser implementation concerns bleeding into analysis code.

Do you think we need both? Can we fold the latter into TableDef?

If we have to keep both, it'd be helpful to at least group the functions in this file based on their source (e.g. TableDef, TDOptions, or on this itself) with comments around the groups? Maybe within those groups they could also be sorted by visibility? There's a lot of stuff going on so I think it'd be helpful to keep it tidy.


PS1, Line 240: getColumnDefs().add(KuduCatalogOpExecutor.REQUIRED_THRIFT_COLUMN_DEF);
is this needed? I thought there was going to be code that fetches the columns from Kudu and stores them.


PS1, Line 242:       if (!getTblProperties().containsKey(KuduTable.KEY_TABLE_NAME)) {
             :         getTblProperties().put(KuduTable.KEY_TABLE_NAME,
             :             KuduUtil.getDefaultCreateKuduTableName(getDb(), getTbl()));
             :       }
hm... can we simplify this and only allow internal tables to be created with the name db_name.table_name? Obviously there could be conflicts and it should fail then. Then it'd be 1 less case to consider.

Also, I wonder if we should prefix the underlying kudu table names with something, e.g. IMPALA_db.table?


PS1, Line 275:         // Kudu's data distribution rules are enforced here. A good reference is
             :         // http://getkudu.io/docs/schema_design.html#data-distribution. As a summary:
             :         // * Only primary key column can be used in distribution definitions
             :         // * Distributions may be either hash or ranged (value intervals)
             :         // * Multiple distributions can be used/mixed
             :         //   * A column can only be used in one hash distribution
             :         //   * A range distribution can only be used once and must be the last definition
             :         //     (both enforced by the grammar).
I don't think we should get in the business of validating the Kudu rules since they should reject it. Is there a reason we have to do this ourselves as well?


PS1, Line 323: throwIfPrimaryKeysWereSpecified
I think the callers can throw, simpler to have a fn to check: bool hasPrimaryKeysSpecified(). Or maybe this isn't necessary? see my commetn below.


PS1, Line 327: isPrimaryKey
when will we have PKs from the col defs but not in the tableDef_.getPrimaryKeyColumnNames()?

Better yet, it looks like the 2 callers of this fn have slightly different use cases so maybe it can be scrapped and handled more specifically. Feel free to disagree...


PS1, Line 327:         if (colDef.isPrimaryKey()) {
             :           hasKey = true;
             :         }
1line


PS1, Line 335: throwIfNotPrimaryKeyType
I only see this called once, please inline it


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/analysis/TableDefOptions.java
File fe/src/main/java/com/cloudera/impala/analysis/TableDefOptions.java:

PS1, Line 32: Represents the end of a CREATE TABLE statement. TableDef represents the beginning of
            :  * the statement.
I find this odd. The 'beginning/end of the statement' feels like a parser implementation detail.

I'm not sure why this needs to be a separate class. TableDef seems OK, and it already has some things that get set after it is constructed (e.g. partitions), so maybe these things can be moved to TableDef and set via addl functions.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java
File fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java:

PS1, Line 207: HdfsFileFormat.KUDU
blegh this looks like an oxymoron. I wonder if we can rename this class to something like StorageFormat. I'll keep an eye out as I read more code.


PS1, Line 212: keyCols != null
why would this be null?


PS1, Line 217: kuduTableName != null
why would this be null?


PS1, Line 217:       if (kuduTableName != null
             :           && kuduTableName.equals(KuduUtil.getDefaultCreateKuduTableName(
             :               table.getDb().getName(), table.getName()))) {
             :         properties.remove(KuduTable.KEY_TABLE_NAME);
             :       }
might just be cleaner to leave this in and not have to have 2 cases to test


http://gerrit.cloudera.org:8080/#/c/4414/1/infra/python/deps/requirements.txt
File infra/python/deps/requirements.txt:

PS1, Line 67: # kudu-python==0.2.0
When this changes I think there may be some steps involved in "upgrading" the environment- have you figured out what that is? We'll need to tell people what command(s) to run after fetching this change.


PS1, Line 70:   # These should eventually be removed  https://issues.apache.org/jira/browse/KUDU-1456
            :   unittest2 == 1.1.0
            :     linecache2 == 1.0.0
            :     traceback2 == 1.4.0
remove


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 1
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Alex Behm (Code Review)" <ge...@cloudera.org>.
Alex Behm has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 12:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/4414/12/fe/src/main/java/org/apache/impala/analysis/TableDef.java
File fe/src/main/java/org/apache/impala/analysis/TableDef.java:

Line 280:     if (options_.fileFormat == THdfsFileFormat.KUDU) {
This doesn't seem right. We used to be able to change the ROW FORMAT for TEXT and SEQUENCE. What was wrong with the previous code?


http://gerrit.cloudera.org:8080/#/c/4414/12/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

Line 1209:       org.apache.hadoop.hive.metastore.api.Table msTbl = existingTbl.getMetaStoreTable();
Should we only do this if existingTbl is not loaded?

Some possible inconsistency issues that we should probably figure out sometime:

The HMS table may have already been dropped, and then we'll not drop the corresponding Kudu table.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 12
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Internal Jenkins (Code Review)" <ge...@cloudera.org>.
Internal Jenkins has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 13: Verified+1

-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 13
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Internal Jenkins
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 4:

(54 comments)

http://gerrit.cloudera.org:8080/#/c/4414/4/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

Line 53: enum THdfsFileFormat {
> rename
This change would touch many places. Would you mind postponing it for a follow up patch?


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/cup/sql-parser.cup
File fe/src/main/cup/sql-parser.cup:

Line 976:   tbl_def_without_col_defs:tbl_def
> a 'create table' without col defs?
This was added for the EXTERNAL Kudu table use case for which no column definitions are specified since we load the schema from the Kudu table. Added a comment.


Line 980:     RESULT = new CreateTableStmt(tbl_def); 
> trailing
Done


Line 1033: // class doesn't inherit from CreateTableStmt.
> should it?
To my opinion yes it should. I actually went down the path of refactoring all the CREATE TABLE* statements but it ended up being too complex to add on top of this big patch. Simplifying the CREATE TABLE statements will also allow us to remove some of the weird table option handling we do in TableDef.java. I will leave a TODO for now.


Line 1065: primary_keys_val ::=
> opt_primary_keys?
Done


Line 1089: tbl_data_layout ::=
> opt_...?
Done


Line 1139:   {: 
> fix spaces and tabs
Done


Line 1370:   KW_PRIMARY key_ident
> what's wrong with KW_KEY?
I don't think we can do that. Don't we use "key" for nested types (map)?


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
File fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java:

Line 30:   static void throwIfNotNullOrNotEmpty(Collection<?> c, String message)
> this is actually 'not null *and* not empty'. you can also phrase that as 'n
Good point. Done


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java:

Line 158:     if (fileFormat_ == THdfsFileFormat.KUDU) {
> check at top
Done


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java:

Line 236:         String.format("PRIMARY KEY must be used instead of the table property '%s'.",
> not good: if you do that on an external table, you get this error message i
I like that idea but it's a bit more complicated for Kudu because different properties are valid depending on whether it's an external or managed table. Moved the check to the function below. Let me know if that works ok.


Line 310:       distributeParam.setPKColumnDefMap(pkColumnDefsByName);
> setPkColumn...
Done


Line 315:   private boolean hasPrimaryKeysSpecified() {
> hasPrimaryKeySpecified (there's only one, which can be a composite key)
Done


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
File fe/src/main/java/org/apache/impala/analysis/DistributeParam.java:

Line 121:           org.apache.impala.catalog.Type colType = colDef.getType();
> does simply Type conflict with something?
Yeah, there is a conflict with the enum Type in this class.


Line 129:           if (colType.isStringType() && !exprType.isStringType()
> this is basically looking for 'assignment compatible', and i'm sure we alre
Done


Line 150:       builder.append(numBuckets_).append(" BUCKETS");
> sprinkle some checkstates in here (on numbuckets and splitrows; or maybe a 
I added checks for numBuckets_. Split rows will go away in a follow up patch with the new range partitioning syntax. I left a TODO to add a validate function then.


Line 200:             literal.setString_literal(expr.getStringValue());
> checkstate that you're getting something valid
Done


Line 211:   void setPKColumnDefMap(Map<String, ColumnDef> pkColumnDefByName) {
> setPkC...
Done


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/TableDef.java
File fe/src/main/java/org/apache/impala/analysis/TableDef.java:

Line 74:   static class TableDefOptions {
> 'Options' is enough
Done


Line 160:     fullyQualifiedTableName_ = analyzer.getFqTableName(getTblName());
> stick with fq abbreviation?
Done


Line 189:     for (ColumnDef colDef: getPartitionColumnDefs()) {
> this is a bit hard to follow. partition cols aren't defined separately, the
These are the columns specified in a PARTITIONED BY clause (non-kudu) and they should be analyzed, no? Sorry, I am not sure I follow your comment.


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
File fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java:

PS4, Line 97: org.apache
> should this have changed?
Hm sed is not that smart :)


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

PS4, Line 89: com.cloudera
> can you add a ref to IMPALA-4271 ?
Done


Line 111:   // Distribution schemes of this Kudu table. Both rang and hash-based distributions are
> range
Done


Line 140:     return msTbl.getParameters().get(KuduTable.KEY_TABLE_NAME);
> i don't think this is worth a function call, it just makes the code harder 
Done


PS4, Line 160: know
> known
Done


PS4, Line 159:   /**
             :    * The number of nodes is not know ahead of time and will be updated during computeStats
             :    * in the scan node.
             :    */
             :   public int getNumNodes() { return -1; }
> I don't see this used
Yeah, removed.


PS4, Line 175: numClusteringCols_ = 0;
> not really related to this change, but it's kind of confusing to have numCl
I like Marcel's suggestion, I changed it to be the number of primary key columns.


PS4, Line 175: numClusteringCols_ = 0;
> those should be the primary key cols
Done


PS4, Line 226:     List<FieldSchema> cols = msTable_.getSd().getCols();
             :     cols.clear();
> why do we get cols from getCols() and then clear() it?
cols is a reference to msTable cols. We clear them here and reload them from Kudu schema in L232. Let me know if it's still not clear or if I should add a comment.


PS4, Line 232: cols.add(new FieldSchema(colName, type.toSql().toLowerCase(), null));
> why do we do this? cols isn't used later
See my comment above. I can add a comment if it's still not clear.


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/catalog/Table.java
File fe/src/main/java/org/apache/impala/catalog/Table.java:

PS4, Line 460: msTbl.getTableType().equals(TableType.EXTERNAL_TABLE.toString());
> we shuold probably compare case insensitive to be safe
Done


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

PS4, Line 1147: occurrs
> occurs
Done


PS4, Line 1482:       } catch (Exception e) {
              :         try {
              :           // Error creating the table in HMS, drop the managed table from Kudu.
              :           if (!Table.isExternalTable(newTable)) {
              :             KuduCatalogOpExecutor.dropTable(newTable, false);
              :           }
              :         } catch (Exception logged) {
              :           String kuduTableName = newTable.getParameters().get(KuduTable.KEY_TABLE_NAME);
              :           LOG.error(String.format("Failed to drop Kudu table '%s'", kuduTableName),
              :               logged);
              :           throw new RuntimeException(String.format("Failed to create the table '%s' in " +
              :               " the Metastore and the newly created Kudu table '%s' could not be " +
              :               " dropped. The log contains more information.", newTable.getTableName(),
              :               kuduTableName), e);
              :         }
              :         if (e instanceof AlreadyExistsException && params.if_not_exists) return false;
              :         throw new ImpalaRuntimeException(
              :             String.format(HMS_RPC_ERROR_FORMAT_STR, "createTable"), e);
> it looks like none of this really needs to be inside the synchronized block
Done


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/service/Frontend.java
File fe/src/main/java/org/apache/impala/service/Frontend.java:

PS4, Line 232:     if (!req.is_delta) {
             :       catalog = new ImpaladCatalog(defaultKuduMasterAddrs_);
             :     }
> 1line
Done


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java:

PS4, Line 135:       if (!hasRangePartitioning) {
             :         tableOpts.setRangePartitionColumns(Collections.<String>emptyList());
             :       }
> I don't think this is necessary
Unfortunately it is. I spoke to Dan (from Kudu team) about it. If the user doesn't specify a range partitioning, Kudu by default creates one with all the primary key columns. So, the distribute params we get from Kudu (and use in the SHOW stmt) is different from the distribute params that the user specified. I added a comment to clarify this. Let me know if this is ok.


PS4, Line 175: erros
> errors
Done


PS4, Line 192: cols.clear();
> can you indicate in the comment that this doesn't just populate msTbl's col
Done


PS4, Line 206: new KuduClient
> I'm not crazy about this wrapper class thing. It's only used in this file.
Done


PS4, Line 212: is accessible
> exists
Done


PS4, Line 215: validateTblProperties
> how about validateKuduTblExists ?
Done


PS4, Line 224: Error accessing table in Kudu " +
             :           "master '%s'
> This could also print the name. Also to avoid confusing with potential futu
Done


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/util/KuduClient.java
File fe/src/main/java/org/apache/impala/util/KuduClient.java:

> as I've said I'd vote to remove this, it's only used by 1 class and adds ex
Done


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java:

Line 1353:         "functional.alltypestiny", "Columns cannot be specified with an external " +
> odd error message. i would expect the 'as select' to be the offending part.
Yeah, you're right. Fixed it.


Line 1720:     AnalyzesOk("create table tab (x int, y int, primary key (X)) " +
> i thought kudu is case-sensitive
We lowercase the pk columns during the analysis. Isn't that ok?


http://gerrit.cloudera.org:8080/#/c/4414/4/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_create.test:

Line 6: as select * from functional.alltypestiny
> shouldn't this be part of an analyzer test?
It used to be that many of these cases were handled during the analysis. Both MJ and Alex suggested we avoid performing checks that are already performed in Kudu (e.g. no boolean primary key columns). Hence, many of these cases are essentially analysis tests that are caught at runtime. Let me know if you prefer to move these back to the analysis. The only issue with this would be keeping these checks consistent with Kudu.


Line 30:   distribute by hash (x) into 3 buckets stored as kudu
> same here, and for the other analysis error test cases in this file
See comment above.


Line 32: NonRecoverableException: Key column may not have type of BOOL, FLOAT, or DOUBLE
> why wouldn't this be an analysis exception?
See comment above.


Line 46: NonRecoverableException: Got out-of-order key column: name: "y" type: INT32 is_key: true is_nullable: false cfile_block_size: 0
> inscrutable error message
This comes from Kudu. I agree it is not user friendly. I'll file the Kudu team to fix this.


Line 53: NonRecoverableException: must have at least two hash buckets
> error message should point out the offending clause
Error message comes from Kudu. That's the drawback of not doing these checks in the analysis. We don't control the error messages :(


Line 60: NonRecoverableException: hash bucket schema components must not contain columns in common
> same here
Same comment as above. I understand this is annoying. The goal is to have the Kudu team fix these error msgs.


http://gerrit.cloudera.org:8080/#/c/4414/4/testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test:

Line 1: ====
> might be a good idea to point out at the top that this test contains test c
Good idea. Done


Line 5:   DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30)) STORED AS KUDU
> analyzer test?
Done


http://gerrit.cloudera.org:8080/#/c/4414/4/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test:

Line 6:   DISTRIBUTE BY RANGE(name) SPLIT ROWS (('abc')) STORED AS KUDU
> analyzer test?
Done


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 4
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Marcel Kornacker (Code Review)" <ge...@cloudera.org>.
Marcel Kornacker has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 5:

(5 comments)

http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
File fe/src/main/java/org/apache/impala/analysis/DistributeParam.java:

Line 129:           if (!org.apache.impala.catalog.Type.areAssignmentCompatibleTypes(colType,
> Done
you want to be able to assign the split value to the column type with just an implicit cast, if any. in other words, for a bigint col it's okay to have a split value of 127, but for a tinyint col, you can't have a split value of maxint.

you can call isImplicitlyCastable(exprType, colType).


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

PS4, Line 226:   private void loadDistributeByParams(org.apache.kudu.client.KuduTable kuduTable) {
             :     Preconditions
> cols is a reference to msTable cols. We clear them here and reload them fro
yes, please leave a comment that you're resetting the mstable cols if there are any. i tripped over that as well.


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

Line 157:   public void load(boolean reuseMetadata /* not used */, IMetaStoreClient msClient,
rename reuseMetadata to dummy


Line 217:       cols.add(new FieldSchema(colName, type.toSql().toLowerCase(), null));
this also updates msTable_

document this side effect in the function comment. this feels pretty convoluted and is hard to follow. is this really necessary or should we simply ignore the column info stored in mstable?


Line 295:         TDistributeByRangeParam rangeParam = param.getBy_range_param();
checkstate(isset...)


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 5
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Michael Brown (Code Review)" <ge...@cloudera.org>.
Michael Brown has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 6:

(10 comments)

http://gerrit.cloudera.org:8080/#/c/4414/6/tests/common/kudu_test_suite.py
File tests/common/kudu_test_suite.py:

PS6, Line 47:     super(KuduTestSuite, cls).setup_class()
            :     if os.environ["KUDU_IS_SUPPORTED"] == "false":
            :       pytest.skip("Kudu is not supported")
Maybe move L47 to L50 (switch the ordering of these)? This isn't just a nit: it avoids running super.setup_class(), which is non-trivial, and then just deciding to skip after all.


PS6, Line 51:   @classmethod
            :   def teardown_class(cls):
            :     pass
Not needed:

1. KuduTestSuite.setup_class() doesn't do anything that needs to be torn down here.

2. This overrides super.teardown_class() and doesn't call super.teardown_class(). super.teardown_class() (ImpalaTestSuite.teardown_class()) does do some work.

So it seems safe to delete this method completely.


http://gerrit.cloudera.org:8080/#/c/4414/6/tests/custom_cluster/test_kudu.py
File tests/custom_cluster/test_kudu.py:

PS6, Line 20: from kudu.client import INT32
ImportError: I think this needs to be kudu.schema


PS6, Line 34:   @classmethod
            :   def add_test_dimensions(cls):
            :     super(CustomClusterTestSuite, cls).add_test_dimensions()
            :     cls.TestMatrix.add_constraint(lambda v:
            :         v.get_value('table_format').file_format == 'parquet' and
            :         v.get_value('table_format').compression_codec == 'none')
I don't think this is needed: no tests here use a vector.


PS6, Line 43:   def test_kudu_master_addrs(self, cursor, kudu):
kudu_client, not kudu. Note this isn't just a rename; it's to make use of the kudu_client fixture. There is no "kudu" fixture.


PS6, Line 45:     with self.temp_kudu_table(kudu, [INT32]) as kudu_table:
kudu_client


PS6, Line 50:             props)
SyntaxError: missing a closing paren here


http://gerrit.cloudera.org:8080/#/c/4414/6/tests/query_test/test_kudu.py
File tests/query_test/test_kudu.py:

PS6, Line 27: from pprint import pprint
Remove: unused import


PS6, Line 30: from copy import copy
            : 
            : from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
            : from tests.common.impala_test_suite import ImpalaTestSuite
            : from tests.common.skip import SkipIf
            : from tests.common.test_dimensions import create_uncompressed_text_dimension
            : from tests.common.test_vector import TestDimension
All imported but not used.


PS6, Line 49:   def test_kudu_scan_node(self, vector):
This test is racy in exhaustive: two tests will be competing for the same database (kududb_test).

You could play it safe and run it serially, but with a little bit of work, this test can be run in parallel if you

1. use the unique_database fixture L49

2. use_db=unique_database L52

3. Modify QueryTest/kudu-scan-node to access dimtbl using the fully qualified path (functional_kudu.dimtbl)

4. Remove all uses of kududb_test here and in kudu-scan-node.test

https://gerrit.cloudera.org/#/c/4169/ shows some of the same principles as an example.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 6
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 12: Code-Review+1

Fixed some test loading issues and modified the dropTable() to properly handle missing Kudu tables. Running exhaustive tests.

-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 12
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has uploaded a new patch set (#4).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   "KEY" is expected to be common enough that the ident style
   declaration is used instead to avoid conflicts.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name. If the database is "default",
   then a Kudu table name will not include the database.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
A fe/src/main/java/org/apache/impala/util/KuduClient.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/catalog/HdfsStorageDescriptorTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/download_requirements
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
M testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/query_test/test_kudu.py
62 files changed, 3,081 insertions(+), 2,107 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/4
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 4
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Alex Behm (Code Review)" <ge...@cloudera.org>.
Alex Behm has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 13: Code-Review+2

-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 13
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has uploaded a new patch set (#3).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   "KEY" is expected to be common enough that the ident style
   declaration is used instead to avoid conflicts.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name. If the database is "default",
   then a Kudu table name will not include the database.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
A fe/src/main/java/org/apache/impala/analysis/ColumnDefOptions.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
A fe/src/main/java/org/apache/impala/util/KuduClient.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/catalog/HdfsStorageDescriptorTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
M testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/query_test/test_kudu.py
62 files changed, 3,119 insertions(+), 2,100 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/3
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 3
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Hello Michael Brown, Alex Behm,

I'd like you to reexamine a change.  Please visit

    http://gerrit.cloudera.org:8080/4414

to look at the new patch set (#12).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   The ident style declaration is used for "KEY" because it is also used
   for nested map types.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/download_requirements
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/query_test/test_kudu.py
62 files changed, 3,122 insertions(+), 2,234 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/12
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 12
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Michael Brown (Code Review)" <ge...@cloudera.org>.
Michael Brown has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 4:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/4414/1/tests/common/kudu_test_suite.py
File tests/common/kudu_test_suite.py:

PS1, Line 54:   def auto_create_db(cls):
            :     return True
            : 
            :   @classmethod
            :   def get_db_name(cls):
            :     # When py.test runs with the xdist plugin, several processes are started and each
            :     # process runs some partition of the tests. It's possible that multiple processes
            :     # will call this method. A random value is generated so the processes won't try
            :     # to use the same database at the same time. The value is cached so within a single
            :     # process the same database name is always used for the class. This doesn't need to
            :     # be thread-safe since multi-threading is never used.
            :     if not cls.__DB_NAME
> Duno. I'll check with Michael.
To answer this: It looks like this provides the same functionality as unique_database, but over a different channel (the conn/cursor fixtures in conftest.py). The design of KuduTestSuite and dependents are so different than most of our test cases. While switching to unique_database can be done, it will be nontrivial, and would require a bit of a change in test design.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 4
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 1:

(31 comments)

Ok, part 3. I have one more batch left...

http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java
File fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java:

PS1, Line 222: 
             : 
             : 
             : 
             : 
             : 
             : 
was this moved somewhere else? Maybe I'll find it as I keep reading...


PS1, Line 238: // Update the HMS
             :     if (reuseMetadata) {
             :       try {
             :         client.alter_table(msTable_.getDbName(), msTable_.getTableName(), msTable_);
             :       } catch (TException e) {
             :         throw new TableLoadingException(e.getMessage());
             :       }
             :     }
Same deal, I'm not sure why reuseMetadata means update the HMS.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/service/CatalogOpExecutor.java
File fe/src/main/java/com/cloudera/impala/service/CatalogOpExecutor.java:

PS1, Line 220: 
             :   private final static KuduCatalogOpExecutor kuduExecutor_
             :       = new KuduCatalogOpExecutor();
all static methods, why instantiate it?


PS1, Line 1122:  // The db == null case isn't handled. The only tables this should matter for are
              :     // Kudu tables. The expectation is Impala will always know about any Kudu tables
              :     // because Hive doesn't support Kudu yet. 
I've always had a problem with this comment :/ even after trimming it down I don't think it makes complete sense. This should mention briefly what this code _is handling_ before it talks about what it's not handling. Can you add 1 sentence at the beginning?


PS1, Line 1124: When Hive supports Kudu the DDL delegates
              :     // can be removed. https://issues.cloudera.org/browse/IMPALA-3424 tracks the removal.
Can you remove these sentences and just leave IMPALA-3424 as a reference? We may not remove our kudu code even if/when Hive supports Kudu.


PS1, Line 1127: ,
nit: extra comma


PS1, Line 1150:         // The operation will be aborted if the Kudu table cannot be dropped. If for
              :         // some reason Kudu is permanently stuck in a non-functional state, the user is
              :         // expected to ALTER TABLE to either set the table to UNMANAGED or set the format
              :         // to something else.
JIRA? Even if we don't fix it we should have something public to point to regarding this behavior. Also to make sure the docs team knows to document it.


PS1, Line 1205: kuduExecutor_
static ref


PS1, Line 1406: if (KuduTable.isKuduTable(tbl)) {
              :       return createKuduTable(tbl, params.if_not_exists, params.getDistribute_by(),
              :           response);
              :     }
              :     return createTable(tbl, params.if_not_exists, params.getCache_op(), response);
              :   }
it's too bad we have to just branch like this and handle everything differently, but I can't think of anything better.


PS1, Line 1420:  createMetaStoreTable
I'd prefer to have this on the prev line and the parameter on the new line


Line 1475:    * creation of a managed Kudu table.
comment on response param


PS1, Line 1480: TDdlExecResponse response)
I think this just fits on the prev line


PS1, Line 1483: kuduExecutor_
static reference KuduCatalogOpExecutor, here elsewhere in this fn


PS1, Line 1509:       // Add the table to the catalog cache
              :       Table newTbl = catalog_.addTable(newTable.getDbName(), newTable.getTableName());
              :       addTableToCatalogUpdate(newTbl, response.result);
While I don't think these will throw, it might be worth wrapping all the logic after the Kudu create table in a try { } catch block that drops the kudu table. That'll future proof this a bit. Feel free to ignore though.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/service/Frontend.java
File fe/src/main/java/com/cloudera/impala/service/Frontend.java:

PS1, Line 230: impaladCatalog_.getDefaultKuduMasterAddrs()
this is fine but a little weird, normally I'm all about removing extra state but in this case it might be better to keep this as a parameter on frontend and pass it in. Unless there's somewhere else besides the old impaladCatalog we can get it?


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/service/JniCatalog.java
File fe/src/main/java/com/cloudera/impala/service/JniCatalog.java:

PS1, Line 83: 
            :       int otherLogLevel, boolean allowAuthToLocal, String kerberosPrincipal)
wrapping


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/com/cloudera/impala/service/KuduCatalogOpExecutor.java:

PS1, Line 29: import org.apache.hadoop.hive.metastore.api.T
since this file references our Table class we should import ours and reference this one w/ namespace. that's just our convention.


PS1, Line 45: yet
remove


PS1, Line 45: When Hive
            :  * functionality is available, that should be preferred to the functionality here.
            :  * https://issues.cloudera.org/browse/IMPALA-3424 tracks this.
remove text since we might not. JIRA can stay but is actually a different issue IMO.


PS1, Line 65: com.cloudera.impala.catalog.Table.
shouldn't need to ref the full namespace


Line 149:    * Reads the schema from a Kudu table and populates 'msTbl' with an equivalent schema.
If an error occurs we may have partially modified the table and leave it in an inconsistent state.


PS1, Line 150: if unable to do so.
if any errors are encountered.


PS1, Line 155:       // Schemas for external tables are not specified by the user in DDL. Instead the
             :       // user provides a Kudu table name (or uses implicit Hive Metastore to Kudu mapping
             :       // rules), then the schema is imported from Kudu.
I'm not sure if this is adding much and I think it's a bit confusing/out of place.


PS1, Line 162: // Start searching....
remove


PS1, Line 166: if (!Strings.isNullOrEmpty(dbName) && !dbName.equalsIgnoreCase("default")) {
             :           // If it exists, the fully qualified name is preferred.
             :           candidateTableNames.add(dbName + "." + tableName);
             :         }
i don't think we should bother handling the default case specially. Can we just always handle it as dbname.tableName? ...and drop the 'candidate' below?


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/util/KuduClient.java
File fe/src/main/java/com/cloudera/impala/util/KuduClient.java:

PS1, Line 30:  * This class wraps an org.apache.kudu.client.KuduClient to transform exceptions into
            :  * ImpalaRuntimeExceptions. No additional functionality is provided. See the Kudu
            :  * documentation for information about the methods.
Let's think about whether or not we can avoid this. Obviously this will have to be updated when kudu client changes... blegh.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/util/KuduUtil.java
File fe/src/main/java/com/cloudera/impala/util/KuduUtil.java:

PS1, Line 41: org.apache.kudu.
reference the ext Type and import ours


PS1, Line 191: com.cloudera.impala.catalog.
import?


PS1, Line 191: isSupportedKeyType
is this their restriction?


PS1, Line 196:  and the user did
             :    * not provide an explicit Kudu table name
... a table, assuming a custom name was not provided.


PS1, Line 201: Catalog.isDefaultDb(metastoreDbName) ?
you know i'm voting for not special casing this...


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 1
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Alex Behm (Code Review)" <ge...@cloudera.org>.
Alex Behm has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 11: Code-Review+2

-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 11
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Hello Michael Brown,

I'd like you to reexamine a change.  Please visit

    http://gerrit.cloudera.org:8080/4414

to look at the new patch set (#10).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   The ident style declaration is used for "KEY" because it is also used
   for nested map types.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/download_requirements
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/query_test/test_kudu.py
62 files changed, 3,077 insertions(+), 2,148 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/10
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 10
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 3:

(14 comments)

next batch... I'm probably about 1/2 way through.

http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java
File fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java:

PS1, Line 275: 
             : 
             : 
             : 
             : 
             : 
             : 
             : 
> Actually, I don't find it such a terrible idea. The sooner we figure out in
It's just more code to support and test...


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/analysis/DistributeParam.java
File fe/src/main/java/com/cloudera/impala/analysis/DistributeParam.java:

PS1, Line 138: 
             : 
             : 
> There is a restriction on the types of primary keys (no bool, float or doub
Thanks, makes sense


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java:

Line 87:   InsertStmt getInsertStmt() { return insertStmt_; }
why change visibility?


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java:

PS3, Line 344:   
nit extra space


PS3, Line 344: getTblPrimaryKeyColumnNames(),  null,
Why would there be PK col names?

Can you add a neg test case to AnalyzeDDLTest.java?

    AnalysisError("create table newkudutbl like kudu '/test-warehouse/schemas/alltypestiny.parquet'")


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java:

PS3, Line 167:     if (getFileFormat() != THdfsFileFormat.KUDU && KuduTable.KUDU_STORAGE_HANDLER.equals(
             :         getTblProperties().get(KuduTable.KEY_STORAGE_HANDLER))) {
             :       throw new AnalysisException(KUDU_STORAGE_HANDLER_ERROR_MESSAGE);
             :     }
             : 
             :     // Avro tables can have empty column defs because they can infer them from the Avro
             :     // schema. Likewise for external Kudu tables, the schema can be read from Kudu.
             :     if (getColumnDefs().isEmpty() && getFileFormat() != THdfsFileFormat.AVRO
             :         && getFileFormat() != THdfsFileFormat.KUDU) {
             :       throw new AnalysisException("Table requires at least 1 column");
             :     }
             : 
             :     if (getFileFormat() == THdfsFileFormat.KUDU) {
             :       analyzeKuduFormat(analyzer);
             :     } else {
             :       AnalysisUtils.throwIfNotNullOrNotEmpty(getDistributeParams(),
             :           "Only Kudu tables can use DISTRIBUTE BY clause.");
             :       throwIfPrimaryKeysWereSpecified("Only Kudu tables can specify a PRIMARY KEY.");
             :     }
             : 
             :     if (getFileFormat() == THdfsFileFormat.AVRO) {
             :       setColumnDefs(analyzeAvroSchema(analyzer));
             :       if (getColumnDefs().isEmpty()) {
             :         throw new AnalysisException(
             :             "An Avro table requires column definitions or an Avro schema.");
             :       }
             :       AvroSchemaUtils.setFromSerdeComment(getColumnDefs());
             :     }
             :   }
what if we were to move everything Kudu related here into analyzeKuduFormat() which first checks if it is Kudu. If it is not, it can make sure it _doesn't_ have the distribute or PK params, then perform the storage handler check, then return. If it is a Kudu table it'd mostly be the same as it is. Basically we could just always call the fn and it handles Kudu or no-Kudu checks appropriately. Then we'd get rid of some of the Kudu mess here.


PS3, Line 279:         throw new AnalysisException(String.format("Column '%s' (type: '%s') cannot " +
             :             "be used as a primary key. Keys must be integer or string types.",
             :             pkColDef.getColName(), pkColDef.getType().toSql()));
I don't see a test case for this error


PS3, Line 277:       ColumnDef pkColDef = getPrimaryKeyColumnDefs().get(i);
             :       if (!KuduUtil.isSupportedKeyType(pkColDef.getType())) {
             :         throw new AnalysisException(String.format("Column '%s' (type: '%s') cannot " +
             :             "be used as a primary key. Keys must be integer or string types.",
             :             pkColDef.getColName(), pkColDef.getType().toSql()));
             :       }
             :       if (!pkColDef.equals(getColumnDefs().get(i))) {
             :         throw new AnalysisException(String.format("Primary key columns in Kudu " +
             :             "tables must be declared first and in-order. Key column '%s' is " +
             :             "out-of-order.", pkColDef.getColName()));
             :       }
If we weren't to do these checks (i.e. that they're supported types and that they come first in the col defs), do you know how we would handle the Kudu failures?

Related to my prev comment in #1 about handling distribution param checking is just that if they start relaxing these requirements then we have to change it and users have to upgrade Impala as well as Kudu. If we can handle Kudu errors on create table nicely, it would help us cut down on the amount of code we have.


PS3, Line 315:    * - Only primary key column can be used in distribution definitions
             :    * - Distributions may be either hash or ranged (value intervals)
             :    * - Multiple distributions can be used/mixed
             :    * - A column can only be used in one hash distribution
             :    * - A range distribution can only be used once and must be the last definition
             :    *   (both enforced by the grammar).
What Kudu supports for partitioning is a bit tricky when we get into nested schemes, so I'm a bit concerned that (a) we're matching their supported functionality perfectly, (b) testing our validation properly, and (c) that we'll necessarily catch any changes that they end up making down the road. It would be nice if we can get a nice error from them and avoid duplication of efforts given the complexity. Let's chat about this in person?


PS3, Line 358: throwIfPrimaryKeysWereSpecified
I do prefer to change this to return a bool and have callers throw: bool hasPrimaryKeysSpecified()


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
File fe/src/main/java/org/apache/impala/analysis/DistributeParam.java:

PS3, Line 156:       if (splitRows_ == null) {
             :         builder.append("...");
can this happen? wouldn't l115 throw?


PS3, Line 191:       if (splitRows_ == null) {
             :         result.setBy_range_param(rangeParam);
             :         return result;
             :       }
this is possible? what does it mean?


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

PS3, Line 259: primarily
remove


PS3, Line 258: Creates a tmp Kudu table. The table is not added to the Kudu storage engine or the
             :    * HMS.
I wasn't sure what this meant. How about:
Creates a temporary KuduTable object populated with the specified properties but has an invalid TableId and is not added to...


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 3
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 9: Code-Review+1

(9 comments)

Carry mikeb's +1 for python tests.

http://gerrit.cloudera.org:8080/#/c/4414/7/be/src/service/frontend.cc
File be/src/service/frontend.cc:

Line 62:     "value should be a comma separated list of hostnames or IP addresses.");
> Add that detail to the description.
Done


http://gerrit.cloudera.org:8080/#/c/4414/9/fe/src/main/cup/sql-parser.cup
File fe/src/main/cup/sql-parser.cup:

Line 1047: // Used for creating tables where the schema is inferred externally, e.g., from an Avro 
> whitespace
Done


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
File fe/src/main/java/org/apache/impala/analysis/DistributeParam.java:

Line 105:     for (String colName: colNames_) {
> I was wondering whether we should allow something like DISTRIBUTED BY(a, a,
Actually this case is rejected by Kudu. Added a test in kudu_create.test.


http://gerrit.cloudera.org:8080/#/c/4414/6/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java:

Line 1350:     // CTAS in an external Kudu table
> I was thinking it would mean the same thing it does for HDFS tables. After 
I see. Let's defer this to a follow up patch if you don't mind. Besides there are other CREATE TABLE statements that we need to revisit for Kudu such as CREATE TABLE LIKE. As is right now, this change would require non-substantial changes to the way we analyze and execute this statement. Filed a JIRA and added a TODO.


http://gerrit.cloudera.org:8080/#/c/4414/9/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java:

Line 1719:     // bucket is partitioned into 4 tables based on the split points of 'y'.
> tables == tablets?
:) yes. Done


http://gerrit.cloudera.org:8080/#/c/4414/6/testdata/bin/generate-schema-statements.py
File testdata/bin/generate-schema-statements.py:

Line 203:   if row_format and file_format == 'text':
> There are a few examples here:
That's fine. I changed it to be on the safe side.


http://gerrit.cloudera.org:8080/#/c/4414/9/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_create.test:

Line 82: create table `add`(`analytic` int primary key) distribute by hash (`analytic`)
> also cover the range() clause in this same test
Done


http://gerrit.cloudera.org:8080/#/c/4414/7/tests/query_test/test_kudu.py
File tests/query_test/test_kudu.py:

Line 204:         """
> File JIRA and defer.
Done KUDU-1711


Line 213:         )
> Thanks for the explanation, seems fine.
Done


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 9
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 4:

(19 comments)

http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
File fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java:

PS4, Line 97: org.apache
should this have changed?


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

PS4, Line 89: com.cloudera
can you add a ref to IMPALA-4271 ?


PS4, Line 160: know
known


PS4, Line 159:   /**
             :    * The number of nodes is not know ahead of time and will be updated during computeStats
             :    * in the scan node.
             :    */
             :   public int getNumNodes() { return -1; }
I don't see this used


PS4, Line 175: numClusteringCols_ = 0;
not really related to this change, but it's kind of confusing to have numClusteringCols_


PS4, Line 226:     List<FieldSchema> cols = msTable_.getSd().getCols();
             :     cols.clear();
why do we get cols from getCols() and then clear() it?


PS4, Line 232: cols.add(new FieldSchema(colName, type.toSql().toLowerCase(), null));
why do we do this? cols isn't used later


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/catalog/Table.java
File fe/src/main/java/org/apache/impala/catalog/Table.java:

PS4, Line 460: msTbl.getTableType().equals(TableType.EXTERNAL_TABLE.toString());
we shuold probably compare case insensitive to be safe


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

PS4, Line 1147: occurrs
occurs


PS4, Line 1482:       } catch (Exception e) {
              :         try {
              :           // Error creating the table in HMS, drop the managed table from Kudu.
              :           if (!Table.isExternalTable(newTable)) {
              :             KuduCatalogOpExecutor.dropTable(newTable, false);
              :           }
              :         } catch (Exception logged) {
              :           String kuduTableName = newTable.getParameters().get(KuduTable.KEY_TABLE_NAME);
              :           LOG.error(String.format("Failed to drop Kudu table '%s'", kuduTableName),
              :               logged);
              :           throw new RuntimeException(String.format("Failed to create the table '%s' in " +
              :               " the Metastore and the newly created Kudu table '%s' could not be " +
              :               " dropped. The log contains more information.", newTable.getTableName(),
              :               kuduTableName), e);
              :         }
              :         if (e instanceof AlreadyExistsException && params.if_not_exists) return false;
              :         throw new ImpalaRuntimeException(
              :             String.format(HMS_RPC_ERROR_FORMAT_STR, "createTable"), e);
it looks like none of this really needs to be inside the synchronized block. Most of it doesn't matter but dropTable creates a KuduClient and makes an RPC. I could imagine this being a problem if Kudu is overwhelmed. You could catch the exception outside of the synchronized block, maybe some extra handling to know if it was created in HMS or not. Though from the looks of this we assume no exceptions from l1502 and 1503.


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/service/Frontend.java
File fe/src/main/java/org/apache/impala/service/Frontend.java:

PS4, Line 232:     if (!req.is_delta) {
             :       catalog = new ImpaladCatalog(defaultKuduMasterAddrs_);
             :     }
1line


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java:

PS4, Line 135:       if (!hasRangePartitioning) {
             :         tableOpts.setRangePartitionColumns(Collections.<String>emptyList());
             :       }
I don't think this is necessary


PS4, Line 175: erros
errors


PS4, Line 192: cols.clear();
can you indicate in the comment that this doesn't just populate msTbl's cols, it replaces whatever was there.


PS4, Line 206: new KuduClient
I'm not crazy about this wrapper class thing. It's only used in this file.


PS4, Line 212: is accessible
exists


PS4, Line 215: validateTblProperties
how about validateKuduTblExists ?


PS4, Line 224: Error accessing table in Kudu " +
             :           "master '%s'
This could also print the name. Also to avoid confusing with potential future auth rules, can this say something like:

"Kudu table TABLENAME on master MASTERADDRS does not exist."


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/util/KuduClient.java
File fe/src/main/java/org/apache/impala/util/KuduClient.java:

as I've said I'd vote to remove this, it's only used by 1 class and adds extra maintenance when we use new API methods or they change any.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 4
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 3:

(23 comments)

http://gerrit.cloudera.org:8080/#/c/4414/3/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

PS3, Line 344: // Enum listing all possible DISTRIBUTE BY types
             : enum TDistributeType {
             :   HASH,
             :   RANGE,
> I don't see this used
Correct. Removed.


PS3, Line 350: // Parameters needed for hash distribution
             : struct TDistributeByHashParam {
             :   1: required list<string> columns
             :   2: required i32 num_buckets
             : }
             : 
             : struct TRangeLiteral {
             :   1: optional i64 int_literal
             :   2: optional string string_literal
             : }
             : 
             : struct TRangeLiteralList {
             :   1: required list<TRangeLiteral> values
             : }
             : 
             : // A range distribution is identified by a list of columns and a series of split rows.
             : struct TDistributeByRangeParam {
             :   1: required list<string> columns
             :   2: optional list<TRangeLiteralList> split_rows;
             : }
             : 
             : // Parameters for the DISTRIBUTE BY clause. The actual distribution is identified by
             : // the type parameter.
             : struct TDistributeParam {
             :   // Set if type is set to HASH
             :   1: optional TDistributeByHashParam by_hash_param;
             : 
             :   // Set if type is set to RANGE
             :   2: optional TDistributeByRangeParam by_range_param;
             : }
> I see that these are used in the serialized catalog objects, but given that
We store the distribute by params in the KuduTable class which is then serialized and sent to al the impalads. So, I think we do need them in this case, unless I misunderstood your comment.


PS3, Line 390: 
             :   // Distribution schemes
             :   4: required list<TDistributeParam> distribute_by
> same for this, can we get rid of it?
See my comment above. We do load and serialize the distribute by params in the KuduTable class.


http://gerrit.cloudera.org:8080/#/c/4414/3/common/thrift/JniCatalog.thrift
File common/thrift/JniCatalog.thrift:

PS3, Line 398: CatalogObjects.TDistributeParam
> wrt my comments in CatalogObjects, I guess we'd need them here, but I don't
See my responses to your comments and let me know if it's still not clear.


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/ColumnDefOptions.java
File fe/src/main/java/org/apache/impala/analysis/ColumnDefOptions.java:

PS3, Line 21:  * Placeholder for the column definition options of a CREATE TABLE statement.
            :  * Contains the list of column definitions and, optionally, the list of column names
            :  * specified using the PRIMARY KEY keyword.
> this feels like a weird abstraction... do we really need this class?
This was used only during parsing to store the "(col type, col type, PRIMARY KEY (col, ..))" part of the create table stmt. I modified the parser so that this isn't needed anymore. Done


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java:

Line 87:   InsertStmt getInsertStmt() { return insertStmt_; }
> why change visibility?
Done


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java:

PS3, Line 344: getTblPrimaryKeyColumnNames(),  null,
> Why would there be PK col names?
Done


PS3, Line 344:   
> nit extra space
Done


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java:

PS3, Line 167:     if (getFileFormat() != THdfsFileFormat.KUDU && KuduTable.KUDU_STORAGE_HANDLER.equals(
             :         getTblProperties().get(KuduTable.KEY_STORAGE_HANDLER))) {
             :       throw new AnalysisException(KUDU_STORAGE_HANDLER_ERROR_MESSAGE);
             :     }
             : 
             :     // Avro tables can have empty column defs because they can infer them from the Avro
             :     // schema. Likewise for external Kudu tables, the schema can be read from Kudu.
             :     if (getColumnDefs().isEmpty() && getFileFormat() != THdfsFileFormat.AVRO
             :         && getFileFormat() != THdfsFileFormat.KUDU) {
             :       throw new AnalysisException("Table requires at least 1 column");
             :     }
             : 
             :     if (getFileFormat() == THdfsFileFormat.KUDU) {
             :       analyzeKuduFormat(analyzer);
             :     } else {
             :       AnalysisUtils.throwIfNotNullOrNotEmpty(getDistributeParams(),
             :           "Only Kudu tables can use DISTRIBUTE BY clause.");
             :       throwIfPrimaryKeysWereSpecified("Only Kudu tables can specify a PRIMARY KEY.");
             :     }
             : 
             :     if (getFileFormat() == THdfsFileFormat.AVRO) {
             :       setColumnDefs(analyzeAvroSchema(analyzer));
             :       if (getColumnDefs().isEmpty()) {
             :         throw new AnalysisException(
             :             "An Avro table requires column definitions or an Avro schema.");
             :       }
             :       AvroSchemaUtils.setFromSerdeComment(getColumnDefs());
             :     }
             :   }
> what if we were to move everything Kudu related here into analyzeKuduFormat
Done. Let me know if that looks any better now.


PS3, Line 279:         throw new AnalysisException(String.format("Column '%s' (type: '%s') cannot " +
             :             "be used as a primary key. Keys must be integer or string types.",
             :             pkColDef.getColName(), pkColDef.getType().toSql()));
> I don't see a test case for this error
There are a couple in AnalyzeDDLTest.java (L1738, L1743).


PS3, Line 277:       ColumnDef pkColDef = getPrimaryKeyColumnDefs().get(i);
             :       if (!KuduUtil.isSupportedKeyType(pkColDef.getType())) {
             :         throw new AnalysisException(String.format("Column '%s' (type: '%s') cannot " +
             :             "be used as a primary key. Keys must be integer or string types.",
             :             pkColDef.getColName(), pkColDef.getType().toSql()));
             :       }
             :       if (!pkColDef.equals(getColumnDefs().get(i))) {
             :         throw new AnalysisException(String.format("Primary key columns in Kudu " +
             :             "tables must be declared first and in-order. Key column '%s' is " +
             :             "out-of-order.", pkColDef.getColName()));
             :       }
> If we weren't to do these checks (i.e. that they're supported types and tha
Done


PS3, Line 315:    * - Only primary key column can be used in distribution definitions
             :    * - Distributions may be either hash or ranged (value intervals)
             :    * - Multiple distributions can be used/mixed
             :    * - A column can only be used in one hash distribution
             :    * - A range distribution can only be used once and must be the last definition
             :    *   (both enforced by the grammar).
> What Kudu supports for partitioning is a bit tricky when we get into nested
Done


PS3, Line 358: throwIfPrimaryKeysWereSpecified
> I do prefer to change this to return a bool and have callers throw: bool ha
Done


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
File fe/src/main/java/org/apache/impala/analysis/DistributeParam.java:

PS3, Line 156:       if (splitRows_ == null) {
             :         builder.append("...");
> can this happen? wouldn't l115 throw?
This is possible because an object of DistributeParam class is created in two cases: 
1. During the analysis of a create table stmt for Kudu. In this case, split rows for range partitioning are guaranteed to be set so L115 is ok. 
2. When we load the schema and distribution schemes in the catalog for a Kudu table. In this case, Kudu's API doesn't currently return any information about the split points so this field could be null. 

With the advent of proper range partitioning syntax this should go away. Kudu will have to provide information about the range partitions including split points or ranges. Would it be ok to add a TODO and reference the pending JIRA? Or let me know if you have another preference.


PS3, Line 191:       if (splitRows_ == null) {
             :         result.setBy_range_param(rangeParam);
             :         return result;
             :       }
> this is possible? what does it mean?
See explanation above.


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
File fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java:

PS3, Line 24: class TableDataLayout {
            : 
            :   private final List<ColumnDef> partitionColDefs_;
            :   private final List<DistributeParam> distributeParams_;
> Not that we're doing this in this review, but we need to think about how th
Agreed. Left a TODO.


PS3, Line 49:   List<ColumnDef> getPartitionColumnDefs() { return partitionColDefs_; }
            :   List<DistributeParam> getDistributeParams() { return distributeParams_; }
> any reason these aren't public?
No need to be public. The default access method gives public access to classes in the same package and that's all we need here. Let me know if it is confusing.


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/TableDef.java
File fe/src/main/java/org/apache/impala/analysis/TableDef.java:

PS3, Line 1: // Copyright 2016 Cloudera Inc.
           : //
           : // Licensed 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 and maybe others need the new license header:
arrrgh. Thanks. Done


PS3, Line 126: return fullyQualifiedTableName_ != null ? fullyQualifiedTableName_ : tableName_;
> Do we need to support calling this before analysis? It'd be nice if we just
We do call this function before tableName_ has been analyzed. One example is during the parsing of CREATE TABLE LIKE  statements where we access the specified table name from a TableDef object.


PS3, Line 137:   List<DistributeParam> getDistributeParams() {
             :     return dataLayout_.getDistributeParams();
             :   }
> 1line?
1 char off :)


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

PS3, Line 258: Creates a tmp Kudu table. The table is not added to the Kudu storage engine or the
             :    * HMS.
> I wasn't sure what this meant. How about:
Done


PS3, Line 259: primarily
> remove
Done


http://gerrit.cloudera.org:8080/#/c/4414/3/infra/python/deps/requirements.txt
File infra/python/deps/requirements.txt:

PS3, Line 83: The kudu-python
            : # version in download_requirements must be kept in sync with this version.
> gotta update download_requirements as well
Done


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 3
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Hello Michael Brown,

I'd like you to reexamine a change.  Please visit

    http://gerrit.cloudera.org:8080/4414

to look at the new patch set (#11).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   The ident style declaration is used for "KEY" because it is also used
   for nested map types.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/download_requirements
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/query_test/test_kudu.py
62 files changed, 3,078 insertions(+), 2,148 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/11
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 11
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 5:

(15 comments)

http://gerrit.cloudera.org:8080/#/c/4414/5/bin/start-impala-cluster.py
File bin/start-impala-cluster.py:

PS5, Line 73: parser.add_option("--kudu_masters", default="127.0.0.1",
> Could we import the default value from tests.common instead of hardcoding i
Good point. Done


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
File fe/src/main/java/org/apache/impala/analysis/DistributeParam.java:

Line 129:           if (!org.apache.impala.catalog.Type.areAssignmentCompatibleTypes(colType,
> you want to be able to assign the split value to the column type with just 
Done


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

PS4, Line 226:   private void loadDistributeByParams(org.apache.kudu.client.KuduTable kuduTable) {
             :     Preconditions
> yes, please leave a comment that you're resetting the mstable cols if there
Done


PS4, Line 226:   private void loadDistributeByParams(org.apache.kudu.client.KuduTable kuduTable) {
             :     Preconditions
> I see. yes, a brief comment would be helpful.
Done


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

Line 157:   public void load(boolean reuseMetadata /* not used */, IMetaStoreClient msClient,
> rename reuseMetadata to dummy
Good point. Done


PS5, Line 193:       msTable_.putToParameters(StatsSetupConst.DO_NOT_UPDATE_STATS,
             :           StatsSetupConst.TRUE);
> This also needs to be stripped out later for SHOW CREATE TABLE.
Done


Line 217:       cols.add(new FieldSchema(colName, type.toSql().toLowerCase(), null));
> this also updates msTable_
Added a comment to clarify it. Currently, we need to keep the schema in HMS consistent with the one in Kudu in order to use the column statistics. Once we move away from this and Kudu provides us with proper statistics API we will get rid of this annoying dance around msTable_.


Line 295:         TDistributeByRangeParam rangeParam = param.getBy_range_param();
> checkstate(isset...)
Done


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

PS5, Line 1484:         // Add the table to the catalog cache
              :         Table newTbl = catalog_.addTable(newTable.getDbName(), newTable.getTableName());
              :         addTableToCatalogUpdate(newTbl, response.result);
> If something here throws (e.g. addTable() looks like it could throw), we pr
Actually, addTable should not be declared to throw because none of its operations throw an exception. addTableToCatalogUpdate also doesn't throw, so I believe it's safe to assume that we don't need to rollback the hms change here. Modified the addTable() function in CatalogServiceCatalog.


http://gerrit.cloudera.org:8080/#/c/4414/5/tests/common/kudu_test_suite.py
File tests/common/kudu_test_suite.py:

PS5, Line 38: @SkipIf.kudu_not_supported
> This is extremely risky due to bugs in our version of old pytest. See IMPAL
Done. Let me know is that's ok now.


http://gerrit.cloudera.org:8080/#/c/4414/5/tests/custom_cluster/test_kudu.py
File tests/custom_cluster/test_kudu.py:

PS5, Line 1: # Copyright (c) 2016 Cloudera, Inc. All rights reserved.
> Remove Cloudera copyright and make sure the license text is correct otherwi
Done


http://gerrit.cloudera.org:8080/#/c/4414/5/tests/metadata/test_ddl.py
File tests/metadata/test_ddl.py:

PS5, Line 220: 
> Why remove this skip?
Hm, that wasn't intentional. I put it back.


PS5, Line 221: 
> Is this removal safe?
Same here.


PS5, Line 221:     self.expected_exceptions = 2
> I can't see where this is used; delete?
Done


http://gerrit.cloudera.org:8080/#/c/4414/5/tests/query_test/test_kudu.py
File tests/query_test/test_kudu.py:

PS5, Line 87:         print("Describe formatted output:")
            :         pprint(table_desc)
> Use LOG.info()
Done


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 5
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has uploaded a new patch set (#8).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   The ident style declaration is used for "KEY" because it is also used
   for nested map types.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/download_requirements
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/query_test/test_kudu.py
62 files changed, 3,059 insertions(+), 2,148 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/8
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 8
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 1:

There are definitely some rough edges but I am posting the CR so that we can iterate through it sooner than later. I'll keep refining it but all comments are welcomed.

-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 1
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Marcel Kornacker (Code Review)" <ge...@cloudera.org>.
Marcel Kornacker has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 4:

(37 comments)

http://gerrit.cloudera.org:8080/#/c/4414/4/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

Line 53: enum THdfsFileFormat {
rename


Line 63: // rename this enum to not be Hdfs specific.
resolve


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/cup/sql-parser.cup
File fe/src/main/cup/sql-parser.cup:

Line 976:   tbl_def_without_col_defs:tbl_def
a 'create table' without col defs?


Line 980:     RESULT = new CreateTableStmt(tbl_def); 
trailing


Line 1033: // class doesn't inherit from CreateTableStmt.
should it?


Line 1065: primary_keys_val ::=
opt_primary_keys?


Line 1089: tbl_data_layout ::=
opt_...?


Line 1139:   {: 
fix spaces and tabs


Line 1370:   KW_PRIMARY key_ident
what's wrong with KW_KEY?


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
File fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java:

Line 30:   static void throwIfNotNullOrNotEmpty(Collection<?> c, String message)
this is actually 'not null *and* not empty'. you can also phrase that as 'not empty'.


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java:

Line 158:     if (fileFormat_ == THdfsFileFormat.KUDU) {
check at top


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java:

Line 236:         String.format("PRIMARY KEY must be used instead of the table property '%s'.",
not good: if you do that on an external table, you get this error message instead of 'primary key not allowed'. move this check into the next function.

alternatively, why not have a list of valid table properties per storage format, and then flag everything else as invalid?


Line 310:       distributeParam.setPKColumnDefMap(pkColumnDefsByName);
setPkColumn...


Line 315:   private boolean hasPrimaryKeysSpecified() {
hasPrimaryKeySpecified (there's only one, which can be a composite key)

hasPrimaryKey would express the same

when is it legal to call this (before/after analyze()?)


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
File fe/src/main/java/org/apache/impala/analysis/DistributeParam.java:

Line 121:           org.apache.impala.catalog.Type colType = colDef.getType();
does simply Type conflict with something?


Line 129:           if (colType.isStringType() && !exprType.isStringType()
this is basically looking for 'assignment compatible', and i'm sure we already have code somewhere to express that more succinctly.


Line 150:       builder.append(numBuckets_).append(" BUCKETS");
sprinkle some checkstates in here (on numbuckets and splitrows; or maybe a validate() function that does those checks, toThrift() would also benefit from it)


Line 200:             literal.setString_literal(expr.getStringValue());
checkstate that you're getting something valid


Line 211:   void setPKColumnDefMap(Map<String, ColumnDef> pkColumnDefByName) {
setPkC...


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/analysis/TableDef.java
File fe/src/main/java/org/apache/impala/analysis/TableDef.java:

Line 74:   static class TableDefOptions {
'Options' is enough


Line 84:     // File format of the table
since this is clearly not a file format anymore, TStorageFormat?


Line 160:     fullyQualifiedTableName_ = analyzer.getFqTableName(getTblName());
stick with fq abbreviation?


Line 189:     for (ColumnDef colDef: getPartitionColumnDefs()) {
this is a bit hard to follow. partition cols aren't defined separately, they're declared. so then why do you need to call colDef.analyze()?


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

Line 111:   // Distribution schemes of this Kudu table. Both rang and hash-based distributions are
range


Line 140:     return msTbl.getParameters().get(KuduTable.KEY_TABLE_NAME);
i don't think this is worth a function call, it just makes the code harder to follow (extra level of indirection)


PS4, Line 175: numClusteringCols_ = 0;
> not really related to this change, but it's kind of confusing to have numCl
those should be the primary key cols


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java:

Line 1353:         "functional.alltypestiny", "Columns cannot be specified with an external " +
odd error message. i would expect the 'as select' to be the offending part.


Line 1720:     AnalyzesOk("create table tab (x int, y int, primary key (X)) " +
i thought kudu is case-sensitive


http://gerrit.cloudera.org:8080/#/c/4414/4/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_create.test:

Line 6: as select * from functional.alltypestiny
shouldn't this be part of an analyzer test?


Line 30:   distribute by hash (x) into 3 buckets stored as kudu
same here, and for the other analysis error test cases in this file


Line 32: NonRecoverableException: Key column may not have type of BOOL, FLOAT, or DOUBLE
why wouldn't this be an analysis exception?


Line 46: NonRecoverableException: Got out-of-order key column: name: "y" type: INT32 is_key: true is_nullable: false cfile_block_size: 0
inscrutable error message


Line 53: NonRecoverableException: must have at least two hash buckets
error message should point out the offending clause


Line 60: NonRecoverableException: hash bucket schema components must not contain columns in common
same here


http://gerrit.cloudera.org:8080/#/c/4414/4/testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test:

Line 1: ====
might be a good idea to point out at the top that this test contains test cases for what basically amount to analysis errors, but they only show up at runtime.


Line 5:   DISTRIBUTE BY RANGE SPLIT ROWS ((10), (30)) STORED AS KUDU
analyzer test?


http://gerrit.cloudera.org:8080/#/c/4414/4/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test:

Line 6:   DISTRIBUTE BY RANGE(name) SPLIT ROWS (('abc')) STORED AS KUDU
analyzer test?


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 4
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has uploaded a new patch set (#9).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   The ident style declaration is used for "KEY" because it is also used
   for nested map types.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/download_requirements
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/query_test/test_kudu.py
62 files changed, 3,065 insertions(+), 2,148 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/9
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 9
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 1:

(94 comments)

http://gerrit.cloudera.org:8080/#/c/4414/1//COMMIT_MSG
Commit Message:

PS1, Line 21: When attempting to create an external
            :   table "foo" in database "bar", Impala will search for a Kudu table
            :   name "foo.bar" and "bar" (Kudu doesn't have database name spaces
            :   yet.)
> I think this sentence is duplicated by #10. Maybe remove this part. This it
Done


PS1, Line 25:  The Kudu table is now required to exist at the time of creation in
            :   Impala.
> for external tables? or if this is more general can you explain a bit more?
Done


PS1, Line 31: default value
> can it still be overridden?
Done


PS1, Line 39: Eventually Hive should have the needed
            :   functionality and the Kudu delegate (renamed in this patch to KuduCatalogOpExecutor)
            :   can be removed.
> maybe, it's not clear if impala would want to keep the ability to do this o
Done


PS1, Line 54:  If the database is "default" then
            :   the Kudu table name will not include the database.
> hm... I wonder if it'd be easier just to keep it standardized and always ha
Done


PS1, Line 56: 11) Several improvements in the grammar related to the family
            :   of CREATE TABLE statements.
> I don't think this bullet adds much value to the commit message. Everything
Done


PS1, Line 58: 12) Added new tests and modified existing Kudu test to use the new
            :   CREATE TABLE syntax.
> I think this can be removed too, it should be implied there are tests and i
Done


PS1, Line 60: insteads
> instead
Done


PS1, Line 60: source of truth
> Does this mean we always hit Kudu for metadata? In the Catalog? In the impa
Done


PS1, Line 64: Not included in this commit:
            : - Additional column properties such as nullability and compression
            : encodings.
> I don't think you need to include this, we have a separate JIRA for this.
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/be/src/catalog/catalog.cc
File be/src/catalog/catalog.cc:

PS1, Line 47:     {"<init>", "(ZILjava/lang/String;IIZLjava/lang/String;)V",
            :       &catalog_ctor_},
> 1 line
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/be/src/service/frontend.cc
File be/src/service/frontend.cc:

PS1, Line 37: // XXX: This flag doesn't seem to be used anywhere. Maybe remove it?
            : DEFINE_bool(load_catalog_at_startup, false, "if true, load all catalog data at startup");
> agreed. doesn't look like it's used by JniFrontend. Let's remove it.
Done


PS1, Line 63: IPs
> IP addresses.
Done


PS1, Line 63:  
> no space
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

PS1, Line 52: THdfsFileFormat
> Maybe not for this patch since it's already huge, but it'd be great to gene
I agree we need to revisit this. Left a TODO for now.


PS1, Line 61: THdfsCompression
> similarly, this seems unnecessarily specific to Hdfs. Not necessarily somet
Left TODO.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/cup/sql-parser.cup
File fe/src/main/cup/sql-parser.cup:

> FYI I'm not looking at this very carefully because it was reviewed previous
Only few things changed mostly wrt specifying the PRIMARY KEY keyword  inside the column definition list.


PS1, Line 31: ColumnDefOptions
> this class doesn't exist, please add the missing file
Done


PS1, Line 405:   view_column_def_list, view_column_defs;
> separate nonterminals if they dont fit?
Done


PS1, Line 458: key_ident;
> does this not need Boolean like those below?
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java
File fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java:

> I'm not in love with the TableDef and TableDefOptions stuff because it feel
As we spoke offline, the initial attempt to refactor this logic didn't succeed. I will not change these classes but I'll make an effort to separate the Kudu-specific logic at the function level.


PS1, Line 240: getColumnDefs().add(KuduCatalogOpExecutor.REQUIRED_THRIFT_COLUMN_DEF);
> is this needed? I thought there was going to be code that fetches the colum
We do, but that happens in the catalog. This was added to satisfy the thrift spec.


PS1, Line 242:       if (!getTblProperties().containsKey(KuduTable.KEY_TABLE_NAME)) {
             :         getTblProperties().put(KuduTable.KEY_TABLE_NAME,
             :             KuduUtil.getDefaultCreateKuduTableName(getDb(), getTbl()));
             :       }
> hm... can we simplify this and only allow internal tables to be created wit
This is executed only for managed tables. Is this what you meant? Also, I changed the prefix to be __IMPALA__db.table;


PS1, Line 275:         // Kudu's data distribution rules are enforced here. A good reference is
             :         // http://getkudu.io/docs/schema_design.html#data-distribution. As a summary:
             :         // * Only primary key column can be used in distribution definitions
             :         // * Distributions may be either hash or ranged (value intervals)
             :         // * Multiple distributions can be used/mixed
             :         //   * A column can only be used in one hash distribution
             :         //   * A range distribution can only be used once and must be the last definition
             :         //     (both enforced by the grammar).
> I don't think we should get in the business of validating the Kudu rules si
Actually, I don't find it such a terrible idea. The sooner we figure out invalid params the better it is for the entire system, i.e. we don't have to make an rpc to the catalog to make an rpc to Kudu in order to figure out that a non-primary key column was used in a distribution. Thoughts?


PS1, Line 323: throwIfPrimaryKeysWereSpecified
> I think the callers can throw, simpler to have a fn to check: bool hasPrima
I responded below. If you don't like it we can change to check + throw in the caller. I don't mind either options.


PS1, Line 327: isPrimaryKey
> when will we have PKs from the col defs but not in the tableDef_.getPrimary
We can have PKs from the col defs in the following case:
CREATE TABLE foo (a int PRIMARY KEY)... Even though the callers of this function correspond to different use cases (external vs non-kudu) the check is similar, i.e. that no PK was specified anywhere in the CREATE TABLE stmt. I am ok with this but if you don't like it I can change it.


PS1, Line 327:         if (colDef.isPrimaryKey()) {
             :           hasKey = true;
             :         }
> 1line
Done


PS1, Line 335: throwIfNotPrimaryKeyType
> I only see this called once, please inline it
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/analysis/DistributeParam.java
File fe/src/main/java/com/cloudera/impala/analysis/DistributeParam.java:

PS1, Line 84: BigDecimal
> why BigDecimal? Ultimately this has to resolve to some int for kudu's API. 
I believe because that's what the Parser generates. I changed it to int.


PS1, Line 110: <= 1)
> is 1 bucket actually not valid?
Yes, I verified it using the Kudu client.


PS1, Line 138: colType.isStringType() && !exprType.isStringType()
             :               || colType.isIntegerType() && (!exprType.isIntegerType()
             :                   || exprType.getPrecision() > colType.getPrecision())
> 1. I don't see anything in the Kudu client that explicitly says you can't p
There is a restriction on the types of primary keys (no bool, float or double) which I verified using the Kudu client; it is also mentioned in Kudu docs (see http://kudu.apache.org/docs/schema_design.html#primary-keys). Consequently, you can't partition on these types of columns.

In any case, these checks here are for ensuring that the types of split value literals can be mapped to the types of the partition columns. Do you think we should change the way we validate this?


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/analysis/TableDefOptions.java
File fe/src/main/java/com/cloudera/impala/analysis/TableDefOptions.java:

PS1, Line 32: Represents the end of a CREATE TABLE statement. TableDef represents the beginning of
            :  * the statement.
> I find this odd. The 'beginning/end of the statement' feels like a parser i
I merged these two classes into TableDef and updated the parser. Let me know what you think.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java
File fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java:

PS1, Line 207: HdfsFileFormat.KUDU
> blegh this looks like an oxymoron. I wonder if we can rename this class to 
Yeah, I agree. I left a TODO in the thrift file to change it.


PS1, Line 212: keyCols != null
> why would this be null?
Done


PS1, Line 217: kuduTableName != null
> why would this be null?
Done


PS1, Line 217:       if (kuduTableName != null
             :           && kuduTableName.equals(KuduUtil.getDefaultCreateKuduTableName(
             :               table.getDb().getName(), table.getName()))) {
             :         properties.remove(KuduTable.KEY_TABLE_NAME);
             :       }
> might just be cleaner to leave this in and not have to have 2 cases to test
Done


PS1, Line 223: Preconditions.checkNotNull(kuduDistributeByParams);
> This breaks for tables that were created before this change.
Good point. Removed.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/catalog/HdfsFileFormat.java
File fe/src/main/java/com/cloudera/impala/catalog/HdfsFileFormat.java:

PS1, Line 39: HdfsFileFormat
> Can you open a JIRA (and leave it in the comment) to refactor this later?
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java
File fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java:

PS1, Line 222: 
             : 
             : 
             : 
             : 
             : 
             : 
> was this moved somewhere else? Maybe I'll find it as I keep reading...
Added necessary checks in the load() function.


PS1, Line 79: 
            :   // TODO we should have something like KuduConfig.getDefaultConfig()
> do you know what this means / can it be removed now that we're adding defau
Not clear to me. I removed it.


PS1, Line 87: 
            :   // Key to specify the number of tablet replicas.
            :   // TODO(KUDU): Allow modification in alter table.
            :   public static final String KEY_TABLET_REPLICAS = "kudu.num_tablet_replicas";
> After looking a bit more, I see that this is the way the user specifies the
Yeah, as you pointed out, it would be weird from a usability point of view to strip something that the user specified in tblProperties. I removed stuff that should be in tblProperties such as the primary key columns. Also removed the TODO. Wrt CREATE TABLE LIKE semantics, let's talk offline.


PS1, Line 87: 
            :   // Key to specify the number of tablet replicas.
            :   // TODO(KUDU): Allow modification in alter table.
            :   public static final String KEY_TABLET_REPLICAS = "kudu.num_tablet_replicas";
> I don't think we need to store this, even if we wanted to modify this later
Done


Line 92:   public static final String KEY_DISTRIBUTE_BY = "kudu.distribute_by";
> comment
Done


PS1, Line 92:   public static final String KEY_DISTRIBUTE_BY = "kudu.distribute_by";
> I'm not sure we should be storing this. The Kudu client exposes KuduTable.g
Agree. I wasn't aware of the getPartitionSchema() API. I changed the code to use that instead.


Line 153:    * Load the columns from the schema list
> can you add a comment about error handling in this function?
Done


PS1, Line 158:       LOG.error(String.format("Kudu tables must have at least one"
             :           + "key column (had %d), and no more key columns than there are table columns "
             :           + "(had %d).", keyColumns.size(), schema.size()));
> shouldn't this still fail? if not, can you add a comment why this continues
Done


PS1, Line 184:       LOG.error(String.format("Some key columns were not found in"
             :               + " the set of columns. List of column names: %s, List of key column names:"
             :               + " %s", Iterables.toString(columnNames), Iterables.toString(keyColumns)));
> why do we continue?
Done


PS1, Line 199:     // Get the table metadata from Kudu
             :     if (reuseMetadata) {
> I'm confused about this. It's not clear to me from the name 'reuseMetadata'
Done


PS1, Line 238: // Update the HMS
             :     if (reuseMetadata) {
             :       try {
             :         client.alter_table(msTable_.getDbName(), msTable_.getTableName(), msTable_);
             :       } catch (TException e) {
             :         throw new TableLoadingException(e.getMessage());
             :       }
             :     }
> Same deal, I'm not sure why reuseMetadata means update the HMS.
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/service/CatalogOpExecutor.java
File fe/src/main/java/com/cloudera/impala/service/CatalogOpExecutor.java:

PS1, Line 220: 
             :   private final static KuduCatalogOpExecutor kuduExecutor_
             :       = new KuduCatalogOpExecutor();
> all static methods, why instantiate it?
Correct. Removed.


PS1, Line 1122:  // The db == null case isn't handled. The only tables this should matter for are
              :     // Kudu tables. The expectation is Impala will always know about any Kudu tables
              :     // because Hive doesn't support Kudu yet. 
> I've always had a problem with this comment :/ even after trimming it down 
Done


PS1, Line 1124: When Hive supports Kudu the DDL delegates
              :     // can be removed. https://issues.cloudera.org/browse/IMPALA-3424 tracks the removal.
> Can you remove these sentences and just leave IMPALA-3424 as a reference? W
Done


PS1, Line 1127: ,
> nit: extra comma
Done


PS1, Line 1150:         // The operation will be aborted if the Kudu table cannot be dropped. If for
              :         // some reason Kudu is permanently stuck in a non-functional state, the user is
              :         // expected to ALTER TABLE to either set the table to UNMANAGED or set the format
              :         // to something else.
> JIRA? Even if we don't fix it we should have something public to point to r
Not sure what the JIRA should say in this case. I'll look into the tests and see if I can create a more specific scenario which can be documented.


PS1, Line 1205: kuduExecutor_
> static ref
Done


PS1, Line 1406: if (KuduTable.isKuduTable(tbl)) {
              :       return createKuduTable(tbl, params.if_not_exists, params.getDistribute_by(),
              :           response);
              :     }
              :     return createTable(tbl, params.if_not_exists, params.getCache_op(), response);
              :   }
> it's too bad we have to just branch like this and handle everything differe
Done


PS1, Line 1420:  createMetaStoreTable
> I'd prefer to have this on the prev line and the parameter on the new line
Done


Line 1475:    * creation of a managed Kudu table.
> comment on response param
Done


PS1, Line 1480: TDdlExecResponse response)
> I think this just fits on the prev line
Done


PS1, Line 1483: kuduExecutor_
> static reference KuduCatalogOpExecutor, here elsewhere in this fn
Done


PS1, Line 1509:       // Add the table to the catalog cache
              :       Table newTbl = catalog_.addTable(newTable.getDbName(), newTable.getTableName());
              :       addTableToCatalogUpdate(newTbl, response.result);
> While I don't think these will throw, it might be worth wrapping all the lo
Let me think about this a bit more. In theory, in this case the user should be able to recover by doing a REFRESH on the table.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/service/Frontend.java
File fe/src/main/java/com/cloudera/impala/service/Frontend.java:

PS1, Line 230: impaladCatalog_.getDefaultKuduMasterAddrs()
> this is fine but a little weird, normally I'm all about removing extra stat
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/service/JniCatalog.java
File fe/src/main/java/com/cloudera/impala/service/JniCatalog.java:

PS1, Line 83: 
            :       int otherLogLevel, boolean allowAuthToLocal, String kerberosPrincipal)
> wrapping
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/com/cloudera/impala/service/KuduCatalogOpExecutor.java:

PS1, Line 29: import org.apache.hadoop.hive.metastore.api.T
> since this file references our Table class we should import ours and refere
Done


PS1, Line 45: yet
> remove
Done


PS1, Line 45: When Hive
            :  * functionality is available, that should be preferred to the functionality here.
            :  * https://issues.cloudera.org/browse/IMPALA-3424 tracks this.
> remove text since we might not. JIRA can stay but is actually a different i
Done


PS1, Line 65: com.cloudera.impala.catalog.Table.
> shouldn't need to ref the full namespace
Done


Line 149:    * Reads the schema from a Kudu table and populates 'msTbl' with an equivalent schema.
> If an error occurs we may have partially modified the table and leave it in
Done


PS1, Line 150: if unable to do so.
> if any errors are encountered.
Done


PS1, Line 155:       // Schemas for external tables are not specified by the user in DDL. Instead the
             :       // user provides a Kudu table name (or uses implicit Hive Metastore to Kudu mapping
             :       // rules), then the schema is imported from Kudu.
> I'm not sure if this is adding much and I think it's a bit confusing/out of
Done


PS1, Line 162: // Start searching....
> remove
Done


PS1, Line 166: if (!Strings.isNullOrEmpty(dbName) && !dbName.equalsIgnoreCase("default")) {
             :           // If it exists, the fully qualified name is preferred.
             :           candidateTableNames.add(dbName + "." + tableName);
             :         }
> i don't think we should bother handling the default case specially. Can we 
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/util/KuduClient.java
File fe/src/main/java/com/cloudera/impala/util/KuduClient.java:

PS1, Line 30:  * This class wraps an org.apache.kudu.client.KuduClient to transform exceptions into
            :  * ImpalaRuntimeExceptions. No additional functionality is provided. See the Kudu
            :  * documentation for information about the methods.
> Let's think about whether or not we can avoid this. Obviously this will hav
Yeah, not a big fan of this either. Let's talk about it.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/util/KuduUtil.java
File fe/src/main/java/com/cloudera/impala/util/KuduUtil.java:

PS1, Line 41: org.apache.kudu.
> reference the ext Type and import ours
Done


PS1, Line 191: isSupportedKeyType
> is this their restriction?
yes. Done


PS1, Line 191: com.cloudera.impala.catalog.
> import?
Done


PS1, Line 196:  and the user did
             :    * not provide an explicit Kudu table name
> ... a table, assuming a custom name was not provided.
Done


PS1, Line 201: Catalog.isDefaultDb(metastoreDbName) ?
> you know i'm voting for not special casing this...
I've changed the default behavior for external tables. For managed it may make sense to generate a default name. No strong opinion though. If you feel it's too confusing, we can change it.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/test/java/com/cloudera/impala/analysis/AnalyzeDDLTest.java
File fe/src/test/java/com/cloudera/impala/analysis/AnalyzeDDLTest.java:

PS1, Line 28: import com.cloudera.impala.catalog.KuduTable;
            : import junit.framework.Assert;
> nit: organize imports
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/infra/python/deps/requirements.txt
File infra/python/deps/requirements.txt:

PS1, Line 67: # kudu-python==0.2.0
> When this changes I think there may be some steps involved in "upgrading" t
I didn't have to do anything. bin/boostrap_toolchain.py checks the versions and downloads/builds the new version automatically.


PS1, Line 70:   # These should eventually be removed  https://issues.apache.org/jira/browse/KUDU-1456
            :   unittest2 == 1.1.0
            :     linecache2 == 1.0.0
            :     traceback2 == 1.4.0
> remove
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/testdata/bin/generate-schema-statements.py
File testdata/bin/generate-schema-statements.py:

PS1, Line 196: primay
> primary
Done


http://gerrit.cloudera.org:8080/#/c/4414/1/testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
File testdata/workloads/functional-query/queries/QueryTest/create_kudu.test:

> Also, I don't see any tests for CTAS here or anywhere.
Most of these have been moved elsewhere. I'll add CTAS tests.


> we might still need to have explicit create table tests.
I agree. We do lack integration tests with Kudu. I'll add them in a followup patch.


http://gerrit.cloudera.org:8080/#/c/4414/1/testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
File testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test:

> well, we should have some coverage of show create table...
We do in the test_kudu.py (TestShowCreateTable class)


http://gerrit.cloudera.org:8080/#/c/4414/1/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test:

PS1, Line 15: 
            : 
            : 
            : 
            : 
            : 
            : 
            : 
            : 
            : 
            : 
> I think this kind of error should still be covered. You can still mismatch 
Covered in analysis tests.


PS1, Line 37: 
            : 
> we should still cover this as well.
Covered in analysis tests.


PS1, Line 50: 
            : 
> same
Covered in analysis tests.


> Some more questions about losing coverage. Fine if this is now covered by F
Yes, all these cases are covered in the AnalyzeDDLTest.java (analysis tests).


http://gerrit.cloudera.org:8080/#/c/4414/1/tests/common/__init__.py
File tests/common/__init__.py:

> does this need to be its own file?
Maybe a more python-savvy person can argue if this is best practice. I assume __init__.py  is where you put any initialization code/global vars.


http://gerrit.cloudera.org:8080/#/c/4414/1/tests/common/kudu_test_suite.py
File tests/common/kudu_test_suite.py:

PS1, Line 54:   @classmethod
            :   def get_db_name(cls):
            :     # When py.test runs with the xdist plugin, several processes are started and each
            :     # process runs some partition of the tests. It's possible that multiple processes
            :     # will call this method. A random value is generated so the processes won't try
            :     # to use the same database at the same time. The value is cached so within a single
            :     # process the same database name is always used for the class. This doesn't need to
            :     # be thread-safe since multi-threading is never used.
            :     if not cls.__DB_NAME:
            :       cls.__DB_NAME = \
            :           choice(ascii_lowercase) + "".join(sample(ascii_lowercase + digits, 5))
            :     return cls.__DB_NAME
> I've always disliked this function... I think we should try to use the db f
Duno. I'll check with Michael.


PS1, Line 127:     mapping = {BOOL: "BOOLEAN",
             :         DOUBLE: "DOUBLE",
             :         FLOAT: "FLOAT",
             :         INT16: "SMALLINT",
             :         INT32: "INT",
             :         INT64: "BIGINT",
             :         INT8: "TINYINT",
             :         STRING: "STRING"}
> this gets defined as a local every function invocation, right? how about cr
Hm, well it's only defined once and it is used only in the context of this fn. If you insist I'll move it outside of this fn.


http://gerrit.cloudera.org:8080/#/c/4414/1/tests/custom_cluster/test_kudu.py
File tests/custom_cluster/test_kudu.py:

PS1, Line 25: TestRedaction
> ?
:) Done


PS1, Line 42: temp_kudu_table
> I'd think we could have the test fn take the database fixture and then pass
I'll check with Michael to see what the issue with get _db_name() and if it is ok to replace it with db fixture, I'll change it here.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 1
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 7:

(63 comments)

Still a few minor asks to go through (e.g. add JIRA numbers and few clarifications) but I am sending the next patch.

http://gerrit.cloudera.org:8080/#/c/4414/7/be/src/service/frontend.cc
File be/src/service/frontend.cc:

Line 62:     "value should be a comma separated list of hostnames or IP addresses.");
> are ports optional or mandatory?
Ports are optional.


http://gerrit.cloudera.org:8080/#/c/4414/7/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

Line 358:   1: required list<TRangeLiteral> values
> Why not a list of TExpr that are expected to be literals? Seems more future
Hm, TExpr? That sounds a bit too much for simple literal values unless we envision using exprs, udfs, etc. Thoughts?


Line 368: // the type parameter.
> which type parameter?
Sorry, there was an enum that got removed and I forgot to update the comments. Done


http://gerrit.cloudera.org:8080/#/c/4414/7/common/thrift/JniCatalog.thrift
File common/thrift/JniCatalog.thrift:

Line 398:   14: optional list<CatalogObjects.TDistributeParam> distribute_by;
> for consistency let's remove trailing ";"
Done


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/cup/sql-parser.cup
File fe/src/main/cup/sql-parser.cup:

Line 1033: // class doesn't inherit from CreateTableStmt. 
> whitespace
Done


Line 1047: // Used for creating external Kudu tables for which the schema is loaded from Kudu.
> There seem to be more uses of this production, so this comment could be mis
Done


Line 1112: // or one RANGE clause
> typo: clauses
Done


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
File fe/src/main/java/org/apache/impala/analysis/ColumnDef.java:

Line 93:   void setIsPrimaryKey() { isPrimaryKey_ = true; }
> do we need this?
No, removed.


Line 191:       Preconditions.checkState(!colDefsByColName.containsKey(colDef.getColName()));
> can check return value of put()
Good point. Keep forgetting this. Done


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java:

Line 208:    * Analyzes and checks table properties which are common for both managed and external
> typo: common to
Done


Line 255:         "PARTITIONED BY cannot be used with an Kudu table.");
> typo: a Kudu table
Done


Line 273:     AnalysisUtils.throwIfNullOrEmpty(getPrimaryKeyColumnDefs(),
> Shouldn't this check hasPrimaryKey()?
Done


Line 284:             "zero. Given number of replicas is: " + r.toString() + ".'");
> remove trailing .' or add the opening single-quote
Done


Line 318:   private boolean hasPrimaryKey() {
> Isn't it enough to check primaryKeyColDefs_ in tableDef_?
You're right, we can do that since tableDef_ is analyzed.


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
File fe/src/main/java/org/apache/impala/analysis/DistributeParam.java:

Line 105:     for (String colName: colNames_) {
> we could specify the same distribution column multiple times
Not sure I follow. Are we doing something to prevent this?


Line 127:             throw new AnalysisException("Split values cannot be NULL");
> do we have a test for this?
Hm no, added one. Thanks


Line 223:     colNames_.addAll(colNames);
> do we need toLower()?
We've already lowered the column names before the call to setColumnNames().


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/analysis/TableDef.java
File fe/src/main/java/org/apache/impala/analysis/TableDef.java:

Line 69:   // Populated during analysis.
> Authoritative list of primary key column definitions populated during analy
Done


Line 177:     fqTableName_.analyze();
> Do you know if Kudu has more permissive or more restrictive constraints on 
Agreed. Will do.


Line 181:     if (analyzer.dbContainsTable(getTblName().getDb(), getTbl(), Privilege.CREATE)
> Are we going to check Sentry privs for Kudu tables? Also ok to defer this f
It's still being discussed :). Lineage as well.


Line 220:    * Analyzes the primary key columns. Primary keys are only supported for Kudu
> Replace the second sentence with a brief description what this checks. It d
Done


Line 234:       StringBuilder columnDefStr = new StringBuilder();
> Not used?
Done


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

Line 1117:     if (db != null && params.cascade) dropTablesFromKudu(db);
> I think it might be a good idea to do this under the metastoreDdlLock_ as w
Done


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java:

Line 47: /**
> newline before
Done


Line 231:     } catch (Exception e) {
> Did you address my comment more nuanced checking here to distinguish connec
I don't think we can differentiate these two given the way the Kudu client works (i.e. it doesn't give you the ability to check if it is successfully connected to the master). I added 'e' in the construction of ImpalaRuntimeException.


http://gerrit.cloudera.org:8080/#/c/4414/6/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java:

Line 1350:     // CTAS in an external Kudu table
> What's the rationale for not allowing this?
Hm, what does it really mean to allow CTAS on external Kudu table? We have the Kudu table schema and the inferred schema from the select stmt. Do you think we should try to see if they match and then create the table and perform the insert? It felt kind of weird to allow something like this. If something fails the user will have to get the schema from Kudu and figure out what the insert should look like. Instead the user can 1. create an external table, 2. describe to see the table schema, 3. do an insert select. Thoughts?


Line 1716:     AnalyzesOk("create table tab (x int, y string, primary key(x, y)) " +
> Can you add a brief comment explaining the table layout for this one?
Done. Let me know if that's better.


Line 1717:         "distribute by hash(x) into 3 buckets, range(y) split rows " +
> can we distribute by range, hash?
No, range needs to be the last one. Fails during parsing. Added another parser test.


Line 1723:     AnalyzesOk("create table tab (a int, b int, c int, d int, primary key (a, b, c))" +
> What happens if I specify PARTITION BY for a Kudu table?
You get an error but the error was saying that no DISTRIBUTE BY is used. Fixed it to say that PARTITIONED BY is not allowed for Kudu tables. Also added test. Done


Line 1739:     AnalysisError(String.format("create table tab (x int) distribute by hash (x) " +
> What about specifying the other params in tblproperties like the distribute
Well these were never specified in TBLPROPERTIES so I am not sure if we should be checking for any weird behavior wrt to properties set there. Besides they will be ignored and/or an error message will be thrown if some crucial clause is missing. Let me know if you have some specific example in mind.


Line 1748:     AnalysisError("create table tab (x int primary key, primary key(x)) stored " +
> Add negative test where two ColumnDefs are declared as PK.
Done


Line 1762:     AnalysisError("create table tab (a int, b int, c int, d int, primary key(a, b, c)) " +
> Do we have tests somewhere for checking which types are supported with Kudu
We do in kudu_create.test and in test_kudu.py. Added a few more in kudu_create.test. Let me know if there is anything else missing.


Line 1772:     // No float split keys
> Even if the column type is float? If so, might want to test that.
float is not allowed as a pk column. We have test for that in kudu_create.test.


Line 1810:     // DISTRIBUTE BY is required for managed tables.
> Primary key is also required, do we have a test?
Ha, interestingly not :). Done


Line 1812:         "Table partitioning must be specified for managed Kudu tables.");
> Let's rephrase this as "Table distribution must be specified ..."
Done


PS6, Line 1828: AnalysisError("create external table t tblproperties (" +
> Not that it makes any sense, but hat happens with:
That results in parsing error. Let me know if you want me to add a test in ParserTest.


Line 1863:   public void TestCreateAvroTest() {
> Add a test with some of the Kudu-specific clauses and STORED AS AVRO
Done


Line 2822:   public void TestShowFiles() throws AnalysisException {
> DO we have a TODO/JIRA somewhere to go through all DDL/SHOW commands and ma
We have some tests for SHOW CREATE TABLE in test_kudu.py but we need a JIRA for show partitions.


http://gerrit.cloudera.org:8080/#/c/4414/6/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
File fe/src/test/java/org/apache/impala/analysis/ParserTest.java:

Line 1632
> Why remove this? It will break my setup :)
Sorry, I had issues with this on when running parser tests only. Done


Line 2235:     ParsesOk("CREATE TABLE foo (i INT PRIMARY KEY, PRIMARY KEY(i)) STORED AS KUDU");
> Add case like this:
Done


Line 2541:     ParsesOk("CREATE TABLE Foo TBLPROPERTIES ('a'='b', 'c'='d') AS SELECT * from bar");
> nit: let's be consistent with how we chop up string literals across lines (
Done


http://gerrit.cloudera.org:8080/#/c/4414/6/fe/src/test/java/org/apache/impala/catalog/HdfsStorageDescriptorTest.java
File fe/src/test/java/org/apache/impala/catalog/HdfsStorageDescriptorTest.java:

Line 58:         "org.apache.impala.hive.serde.ParquetInputFormat",
> revert
Done


http://gerrit.cloudera.org:8080/#/c/4414/6/testdata/bin/generate-schema-statements.py
File testdata/bin/generate-schema-statements.py:

Line 203:   if row_format and file_format == 'text':
> I think row_format is also valid for sequence files (maybe we're not using 
Hm, I couldn't find anything in the docs or in our tests.


http://gerrit.cloudera.org:8080/#/c/4414/6/testdata/datasets/functional/functional_schema_template.sql
File testdata/datasets/functional/functional_schema_template.sql:

Line 823: distribute by range(id) split rows ((1003), (1007)) stored as kudu;
> ah so much better
Done


http://gerrit.cloudera.org:8080/#/c/4414/6/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
File testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test:

Line 218: create table kudu_table_clone like functional_kudu.alltypes
> Why can't we check this in analysis?
Done


http://gerrit.cloudera.org:8080/#/c/4414/6/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_create.test:

Line 3: # This test file contains several cases for what basically amount to analysis errors,
> Should mention that some of this behavior is intentional and why.
Done


Line 11: ImpalaRuntimeException: Type TIMESTAMP is not supported in Kudu
> Do we get the same message for DECIMAL and complex types? (No need to add a
Yes, added tests. Done


Line 14: create table t primary key (id) distribute by hash (id) into 3 buckets
> Add test for creating and querying a table that has Impala keywords as the 
Yeah you were right about using getIdentSql(). This doesn't work as expected. Fixed it and added a test.


http://gerrit.cloudera.org:8080/#/c/4414/6/tests/common/kudu_test_suite.py
File tests/common/kudu_test_suite.py:

PS6, Line 47:     super(KuduTestSuite, cls).setup_class()
            :     if os.environ["KUDU_IS_SUPPORTED"] == "false":
            :       pytest.skip("Kudu is not supported")
> Maybe move L47 to L50 (switch the ordering of these)? This isn't just a nit
Done


PS6, Line 51:   @classmethod
            :   def teardown_class(cls):
            :     pass
> Not needed:
Done


http://gerrit.cloudera.org:8080/#/c/4414/6/tests/custom_cluster/test_kudu.py
File tests/custom_cluster/test_kudu.py:

PS6, Line 20: from kudu.client import INT32
> ImportError: I think this needs to be kudu.schema
Done


PS6, Line 34:   @classmethod
            :   def add_test_dimensions(cls):
            :     super(CustomClusterTestSuite, cls).add_test_dimensions()
            :     cls.TestMatrix.add_constraint(lambda v:
            :         v.get_value('table_format').file_format == 'parquet' and
            :         v.get_value('table_format').compression_codec == 'none')
> I don't think this is needed: no tests here use a vector.
Done


PS6, Line 43:   def test_kudu_master_hosts(self, cursor, kudu):
> kudu_client, not kudu. Note this isn't just a rename; it's to make use of t
Done


PS6, Line 45:     with self.temp_kudu_table(kudu, [INT32]) as kudu_table:
> kudu_client
Done


PS6, Line 50:             props)
> SyntaxError: missing a closing paren here
Done


http://gerrit.cloudera.org:8080/#/c/4414/6/tests/query_test/test_kudu.py
File tests/query_test/test_kudu.py:

PS6, Line 27: from pprint import pprint
> Remove: unused import
Done


PS6, Line 30: from copy import copy
            : 
            : from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
            : from tests.common.impala_test_suite import ImpalaTestSuite
            : from tests.common.skip import SkipIf
            : from tests.common.test_dimensions import create_uncompressed_text_dimension
            : from tests.common.test_vector import TestDimension
> All imported but not used.
Done


PS6, Line 49:   def test_kudu_scan_node(self, vector):
> This test is racy in exhaustive: two tests will be competing for the same d
Thanks. Done


Line 57:   def test_insert_update_delete(self, vector, unique_database):
> Can we keep the .test file name and the python test function consistent? i.
Done


http://gerrit.cloudera.org:8080/#/c/4414/7/tests/query_test/test_kudu.py
File tests/query_test/test_kudu.py:

Line 192:        'show_create_sql' can be templates that can be used with str.format(). format()
> In our other SHOW CREATE TABLE tests we also try to run execute the output 
See comment below for SPLIT ROWS. Unfortunately, the output is not always executable. To be fixed in Kudu.


Line 204:         CREATE TABLE {table} (c INT PRIMARY KEY)
> add comment to the PK column
That's interesting. Actually, currently don't work because we can't persist them in Kudu and the first time the schema is loaded from Kudu we override the HMS column defs, so the comments are lost. So, option 1 is to store them in tblproperies and option 2 to ignore and/or throw error if specified for Kudu tables. Thoughts?


Line 213:         TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}')""".format(
> Why include these TBLPROPERTIES?
Good question. If the user doesn't specify the kudu master_addresses we populate tblproperties with the known values. The issue is that we can't tell if the user added this property or it was generated. I chose to always display this property but I can change it if you don't like it.


Line 226:         DISTRIBUTE BY HASH (c) INTO 3 BUCKETS, RANGE (c) SPLIT ROWS (...)
> is the SPLIT ROWS (...) legal syntax?
No, but Kudu's API doesn't currently return any information about range partitions except the column names used.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 7
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has uploaded a new patch set (#7).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   The ident style declaration is used for "KEY" because it is also used
   for nested map types.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/catalog/HdfsStorageDescriptorTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/download_requirements
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
M testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/query_test/test_kudu.py
64 files changed, 3,025 insertions(+), 2,138 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/7
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 7
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Alex Behm (Code Review)" <ge...@cloudera.org>.
Alex Behm has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 5:

(41 comments)

Sending out the first wave.

http://gerrit.cloudera.org:8080/#/c/4414/5//COMMIT_MSG
Commit Message:

Line 45:    "KEY" is expected to be common enough that the ident style
Might also want to mention that "key" is used for nested map types, so it's not desirable to make it a keyword.


Line 51:    on the HMS database and table name. If the database is "default",
Does this last regarding "default" make the code easier or more complicated? What's the motivation for special casing it, i.e., what's the harm in including the "default" database name?


http://gerrit.cloudera.org:8080/#/c/4414/5/be/src/service/frontend.cc
File be/src/service/frontend.cc:

Line 61: DEFINE_string(kudu_master_addrs, "", "Specifies the default Kudu master(s). The given "
Can we make this more consistent with out existing options? For example:

-catalog_service_host
-state_store_host


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
File fe/src/main/java/org/apache/impala/analysis/ColumnDef.java:

Line 176:   static List<String> toColumnNames(Collection<ColumnDef> columnDefs) {
colDefs


Line 184:   static Map<String, ColumnDef> mapByColumnNames(Collection<ColumnDef> colDefs) {
comment that colDefs is assumed have no duplicate names, or alternatively deal with that case


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java:

Line 99:           "does not support (%s) file format. Supported formats are: (%s)",
the (%s) file format.


Line 100:           createStmt_.getFileFormat().toString().replace("_", ""),
what's with this weird "_" replacement?


Line 101:           "PARQUET, TEXTFILE, KUDU"));
use SUPPORTED_INSERT_FORMATS instead of hardcoding


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java:

Line 80:   private void setColumnDefs(List<ColumnDef> columnDefs) {
nit: colDefs everywhere


Line 95:   Map<String, String> getTblProperties() { return tableDef_.getTblProperties(); }
is there any significance to not adding public/private to some of these methods?


Line 192:           "Only Kudu tables can use DISTRIBUTE BY clause.");
the DISTRIBUTE BY clause


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/analysis/TableDef.java
File fe/src/main/java/org/apache/impala/analysis/TableDef.java:

Line 43:  * Represents the table parameters in a CREATE TABLE statement.
Let's be more precise and list the clauses that this captures.


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

Line 97:   // Table name in Kudu storage engine. It may not neccessarily be the same as the
the Kudu storage engine


Line 101:   // 1. For managed tables, 'kuduTableName_' is prefixed with '__IMPALA__<db_name>' to
Can we make the prefix 'impala::<db_name>' instead?

Better to keep the "impala" part lowercase for consistency.

It might be a minor detail, but keeping the name readable seems important. I'd imagine that Kudu tables will be accessed from several tools.


Line 154:    * are loaded from HMS. The function also updates the table schema in HMS.
Add a comment about why we also update the HMS. My understanding is that we want to propagate alterations made to the Kudu table to the HMS.


Line 195:       msClient.alter_table(msTable_.getDbName(), msTable_.getTableName(), msTable_);
This means every invalidate/refresh will do an extra alter to HMS. Are we concerned about the extra load?


Line 202:    * Loads the schema from the Kudu table including column definitions and primary key
mention that it loads the schema into this table as well as the HMS table


Line 216:       String colName = ToSqlUtils.getIdentSql(colSchema.getName());
getIdentSql() may add backticks, I don't think we want that here


Line 223:     numClusteringCols_ = primaryKeyColumnNames_.size();
Check that there is at least one PK column


Line 228:     PartitionSchema partitionSchema = kuduTable.getPartitionSchema();
Preconditions.checkState(!colsByPos_.isEmpty()); ?


Line 254:   public static KuduTable createTmpTable(Db db,
createCtasTarget()? Seems more explicit since there really is only one use.


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

Line 1117:     if (db != null) dropTablesFromKudu(db);
Won't this drop all Kudu tables even if CASCADE is not specified?


Line 1151:     // the metadata must be fetched from the Hive Metastore.
Why do we need to load from HMS? See comments below.


Line 1167:         throw new ImpalaRuntimeException(
This seems like weird behavior. The user issued a cascading drop db, but gets an error because one of the tables could not be fetched from the HMS. It seems fine to ignore the error because the user wanted to drop everything anyway.


Line 1172:       if (!KuduTable.isKuduTable(msTable)) continue;
Why do we need this check? Won't Kudu tell us if the table does not exist? Seems fine to ignore that error.


Line 1438:     StorageDescriptor sd = HiveStorageDescriptorFactory.createSd(
should we consolidate all the SD stuff into one createSd() call?


Line 1453:     if (params.getPartition_columns() != null) {
for my understanding: these are empty for Kudu tables right?


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java:

Line 41: //import org.apache.hadoop.hive.metastore.api.Table;
?


Line 53:  * such as creating and droping tables from Kudu.
typo: dropping


Line 58:   public final static ColumnDef REQUIRED_THRIFT_COLUMN_DEF;
Isn't it enough to set an empty list or mark the thrift field as set?


Line 70:   static void createTable(org.apache.hadoop.hive.metastore.api.Table msTbl,
createManagedTable()? and then mention that msTbl must be a managed table


Line 85:       kudu.createTable(kuduTableName, schema, tableOpts);
May this throw if the table already exists? If so it might be cleaner to rely on that exception for checking whether the table already exists. Right now there's a possible race because tableExists() and createTable() are not atomic (i.e., kudu could still say the table already exists here)


Line 97:       throws ImpalaRuntimeException {
remove throws?


Line 121:       TCreateTableParams params, Schema schema) throws ImpalaRuntimeException {
remove throws?


Line 155:       int r = Integer.parseInt(replication);
catch NumberFormatException?


Line 170:     if (Table.isExternalTable(msTbl)) return;
Can we make this a Precondition and have the caller check it?


Line 178:       if (kudu.tableExists(tableName)) {
same comment about non-atomic exists() + drop() calls


Line 214:         String colName = ToSqlUtils.getIdentSql(colSchema.getName());
I don't think we want getIdentSql() here


Line 239:     } catch (Exception e) {
I think we should be more nuanced in our exception handling here, in particular, also include the original exception as the cause. For example, if for some reason we cannot connect to Kudu we will return a "table does not exist" without even including the original cause (presumably an IOE).


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/util/KuduUtil.java
File fe/src/main/java/org/apache/impala/util/KuduUtil.java:

Line 198:         metastoreTableName : "__IMPALA__" + metastoreDbName + "." + metastoreTableName;
make the prefix a static constant (and change it to "impalad::" please)


Line 240:       case BOOL: return Type.BOOLEAN;
no DECIMAL?


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 5
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 7:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/4414/7/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

Line 358:   1: required list<TRangeLiteral> values
> Why add a new special purpose structure that is redundant with TExpr? To me
If that's ok, I'll defer that for now. There are a bunch of stuff in KuduUtil related to split rows that make use of TRangeLiteral which will btw change in the range partitioning patch. So, I'll fix it there for good. Added a TODO.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 7
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 1:

(17 comments)

another batch of comments, still a lot of files i haven't touched yet...

http://gerrit.cloudera.org:8080/#/c/4414/1/be/src/catalog/catalog.cc
File be/src/catalog/catalog.cc:

PS1, Line 47:     {"<init>", "(ZILjava/lang/String;IIZLjava/lang/String;)V",
            :       &catalog_ctor_},
1 line


http://gerrit.cloudera.org:8080/#/c/4414/1/be/src/service/frontend.cc
File be/src/service/frontend.cc:

PS1, Line 37: // XXX: This flag doesn't seem to be used anywhere. Maybe remove it?
            : DEFINE_bool(load_catalog_at_startup, false, "if true, load all catalog data at startup");
agreed. doesn't look like it's used by JniFrontend. Let's remove it.


PS1, Line 63: IPs
IP addresses.


PS1, Line 63:  
no space


http://gerrit.cloudera.org:8080/#/c/4414/1/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

PS1, Line 52: THdfsFileFormat
Maybe not for this patch since it's already huge, but it'd be great to generalize this if we can. I can think of two improvements:
1) Maybe we should model the storage layer, e.g. have a TStorageEngine, then make this TFileFormat (perhaps). This is probably a big change.
2) Rename this to be TStorageFormat, which kind of addresses #1 but doesn't separate out storage engines and file formats.


PS1, Line 61: THdfsCompression
similarly, this seems unnecessarily specific to Hdfs. Not necessarily something to change now but maybe we can create a follow-up JIRA to clean this up.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/cup/sql-parser.cup
File fe/src/main/cup/sql-parser.cup:

PS1, Line 31: ColumnDefOptions
this class doesn't exist, please add the missing file


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/analysis/DistributeParam.java
File fe/src/main/java/com/cloudera/impala/analysis/DistributeParam.java:

PS1, Line 84: BigDecimal
why BigDecimal? Ultimately this has to resolve to some int for kudu's API. (We can check if it's 32 or 64bit).


PS1, Line 110: <= 1)
is 1 bucket actually not valid?


PS1, Line 138: colType.isStringType() && !exprType.isStringType()
             :               || colType.isIntegerType() && (!exprType.isIntegerType()
             :                   || exprType.getPrecision() > colType.getPrecision())
1. I don't see anything in the Kudu client that explicitly says you can't partition on any particular types. This code will exclude boolean and floating pt types, which is maybe unnecessary. 
2. Esp if we can address #1, is there a cleaner way to make sure the types are valid rather than enumerating the kinds of types to consider? I'm not sure, but maybe one of the frontend gurus can think of something.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/catalog/HdfsFileFormat.java
File fe/src/main/java/com/cloudera/impala/catalog/HdfsFileFormat.java:

PS1, Line 39: HdfsFileFormat
Can you open a JIRA (and leave it in the comment) to refactor this later?


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java
File fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java:

PS1, Line 79: 
            :   // TODO we should have something like KuduConfig.getDefaultConfig()
do you know what this means / can it be removed now that we're adding default master addrs?


Line 92:   public static final String KEY_DISTRIBUTE_BY = "kudu.distribute_by";
comment


Line 153:    * Load the columns from the schema list
can you add a comment about error handling in this function?


PS1, Line 158:       LOG.error(String.format("Kudu tables must have at least one"
             :           + "key column (had %d), and no more key columns than there are table columns "
             :           + "(had %d).", keyColumns.size(), schema.size()));
shouldn't this still fail? if not, can you add a comment why this continues?


PS1, Line 184:       LOG.error(String.format("Some key columns were not found in"
             :               + " the set of columns. List of column names: %s, List of key column names:"
             :               + " %s", Iterables.toString(columnNames), Iterables.toString(keyColumns)));
why do we continue?


PS1, Line 199:     // Get the table metadata from Kudu
             :     if (reuseMetadata) {
I'm confused about this. It's not clear to me from the name 'reuseMetadata' why this means we should populate the metadata from Kudu. If anything, it sounds like it would be the opposite.

The base class comment just says "If 'reuseMetadata' is true, reuse valid existing metadata.".


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 1
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Internal Jenkins (Code Review)" <ge...@cloudera.org>.
Internal Jenkins has submitted this change and it was merged.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   The ident style declaration is used for "KEY" because it is also used
   for nested map types.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Reviewed-on: http://gerrit.cloudera.org:8080/4414
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Internal Jenkins
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/download_requirements
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
M testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/metadata/test_show_create_table.py
M tests/query_test/test_kudu.py
64 files changed, 3,138 insertions(+), 2,255 deletions(-)

Approvals:
  Internal Jenkins: Verified
  Alex Behm: Looks good to me, approved



-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 14
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Internal Jenkins
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Alex Behm (Code Review)" <ge...@cloudera.org>.
Alex Behm has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 7:

(29 comments)

http://gerrit.cloudera.org:8080/#/c/4414/6/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java:

Line 1350:     // CTAS in an external Kudu table
What's the rationale for not allowing this?


Line 1716:     AnalyzesOk("create table tab (x int, y string, primary key(x, y)) " +
Can you add a brief comment explaining the table layout for this one?


Line 1717:         "distribute by hash(x) into 3 buckets, range(y) split rows " +
can we distribute by range, hash?


Line 1723:     AnalyzesOk("create table tab (a int, b int, c int, d int, primary key (a, b, c))" +
What happens if I specify PARTITION BY for a Kudu table?


Line 1739:     AnalysisError(String.format("create table tab (x int) distribute by hash (x) " +
What about specifying the other params in tblproperties like the distribute by and split rows.


Line 1748:     AnalysisError("create table tab (x int primary key, primary key(x)) stored " +
Add negative test where two ColumnDefs are declared as PK.


Line 1762:     AnalysisError("create table tab (a int, b int, c int, d int, primary key(a, b, c)) " +
Do we have tests somewhere for checking which types are supported with Kudu? We should make sure that:
* you can create a table with all supported types (and same for the specific clauses like primary key, distributed by, etc.)
* you cannot create tables with unsupported types like TIMESTAMP/DECIMAL and nested types (should fail gracefully with "not supported")
* or alternatively, if we want to defer the type checks to Kudu (and not bake it into Impala analysis), then we should document that somewhere


Line 1772:     // No float split keys
Even if the column type is float? If so, might want to test that.


Line 1810:     // DISTRIBUTE BY is required for managed tables.
Primary key is also required, do we have a test?


Line 1812:         "Table partitioning must be specified for managed Kudu tables.");
Let's rephrase this as "Table distribution must be specified ..."


PS6, Line 1828: AnalysisError("create external table t tblproperties (" +
Not that it makes any sense, but hat happens with:

create external table t (primary key()) ...


Line 1863:   public void TestCreateAvroTest() {
Add a test with some of the Kudu-specific clauses and STORED AS AVRO


Line 2822:   public void TestShowFiles() throws AnalysisException {
DO we have a TODO/JIRA somewhere to go through all DDL/SHOW commands and make them behave properly for Kudu?


http://gerrit.cloudera.org:8080/#/c/4414/6/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
File fe/src/test/java/org/apache/impala/analysis/ParserTest.java:

Line 1632
Why remove this? It will break my setup :)


Line 2235:     ParsesOk("CREATE TABLE foo (i INT PRIMARY KEY, PRIMARY KEY(i)) STORED AS KUDU");
Add case like this:
CREATE TABLE foo (i INT PRIMARY KEY, j PRIMARY KEY) STORED AS KUDU


Line 2541:     ParsesOk("CREATE TABLE Foo TBLPROPERTIES ('a'='b', 'c'='d') AS SELECT * from bar");
nit: let's be consistent with how we chop up string literals across lines (space at end of previous line xor beginning of next line)


http://gerrit.cloudera.org:8080/#/c/4414/6/fe/src/test/java/org/apache/impala/catalog/HdfsStorageDescriptorTest.java
File fe/src/test/java/org/apache/impala/catalog/HdfsStorageDescriptorTest.java:

Line 58:         "org.apache.impala.hive.serde.ParquetInputFormat",
revert


http://gerrit.cloudera.org:8080/#/c/4414/6/testdata/bin/generate-schema-statements.py
File testdata/bin/generate-schema-statements.py:

Line 203:   if row_format and file_format == 'text':
I think row_format is also valid for sequence files (maybe we're not using it though)


http://gerrit.cloudera.org:8080/#/c/4414/6/testdata/datasets/functional/functional_schema_template.sql
File testdata/datasets/functional/functional_schema_template.sql:

Line 823: distribute by range(id) split rows ((1003), (1007)) stored as kudu;
ah so much better


http://gerrit.cloudera.org:8080/#/c/4414/6/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
File testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test:

Line 218: create table kudu_table_clone like functional_kudu.alltypes
Why can't we check this in analysis?


http://gerrit.cloudera.org:8080/#/c/4414/6/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
File testdata/workloads/functional-query/queries/QueryTest/kudu_create.test:

Line 3: # This test file contains several cases for what basically amount to analysis errors,
Should mention that some of this behavior is intentional and why.


Line 11: ImpalaRuntimeException: Type TIMESTAMP is not supported in Kudu
Do we get the same message for DECIMAL and complex types? (No need to add another test, just asking whether you've checked).


Line 14: create table t primary key (id) distribute by hash (id) into 3 buckets
Add test for creating and querying a table that has Impala keywords as the table name and some column names.


http://gerrit.cloudera.org:8080/#/c/4414/6/tests/common/kudu_test_suite.py
File tests/common/kudu_test_suite.py:

Line 69:   def get_db_name(cls):
Why not use the unique_database fixture? Sure, we don't run in parallel but unique_database seems saner (no need to explain all these framework intricacies)


Line 94:        name will be used. If 'prepend_db_name' is True, the table name will be prepended
what does the db_name parameter do then?


http://gerrit.cloudera.org:8080/#/c/4414/6/tests/query_test/test_kudu.py
File tests/query_test/test_kudu.py:

Line 57:   def test_insert_update_delete(self, vector, unique_database):
Can we keep the .test file name and the python test function consistent? i.e. rename with to test_kudu_crud


http://gerrit.cloudera.org:8080/#/c/4414/7/tests/query_test/test_kudu.py
File tests/query_test/test_kudu.py:

Line 192:        'show_create_sql' can be templates that can be used with str.format(). format()
In our other SHOW CREATE TABLE tests we also try to run execute the output of SHOW CREATE TABLE


Line 213:         TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}')""".format(
Why include these TBLPROPERTIES?


Line 226:         DISTRIBUTE BY HASH (c) INTO 3 BUCKETS, RANGE (c) SPLIT ROWS (...)
is the SPLIT ROWS (...) legal syntax?


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 7
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Alex Behm (Code Review)" <ge...@cloudera.org>.
Alex Behm has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 7:

(10 comments)

Responses to comments. Doing another round now.

http://gerrit.cloudera.org:8080/#/c/4414/7/be/src/service/frontend.cc
File be/src/service/frontend.cc:

Line 62:     "value should be a comma separated list of hostnames or IP addresses.");
> Ports are optional.
Add that detail to the description.


http://gerrit.cloudera.org:8080/#/c/4414/7/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

Line 358:   1: required list<TRangeLiteral> values
> Hm, TExpr? That sounds a bit too much for simple literal values unless we e
Why add a new special purpose structure that is redundant with TExpr? To me a TExpr is pretty clear, but without looking at the definition, I don't really know what a TRangeLiteral is.


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
File fe/src/main/java/org/apache/impala/analysis/DistributeParam.java:

Line 105:     for (String colName: colNames_) {
> Not sure I follow. Are we doing something to prevent this?
I was wondering whether we should allow something like DISTRIBUTED BY(a, a, a, a, a) or whether we should reject that because 'a' is mentioned multiple times. We reject such cases for PKs.


http://gerrit.cloudera.org:8080/#/c/4414/7/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java:

Line 231:     } catch (Exception e) {
> I don't think we can differentiate these two given the way the Kudu client 
Got it, thanks!


http://gerrit.cloudera.org:8080/#/c/4414/6/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
File fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java:

Line 1350:     // CTAS in an external Kudu table
> Hm, what does it really mean to allow CTAS on external Kudu table? We have 
I was thinking it would mean the same thing it does for HDFS tables. After the CTAS we have a new external table with a schema according to the query statement. The fact that it's a Kudu and not an HDFS table seems like an insignificant detail from a user's point of view. In other words, at a high-level it's not very clear to me why Kudu would behave any different than HDFS.

I get that the implementation may be weird. Let's revisit in a follow-on change, if necessary.


Line 1739:     AnalysisError(String.format("create table tab (x int) distribute by hash (x) " +
> Well these were never specified in TBLPROPERTIES so I am not sure if we sho
I see. Most of the metadata is in Kudu now. Nice!


http://gerrit.cloudera.org:8080/#/c/4414/6/testdata/bin/generate-schema-statements.py
File testdata/bin/generate-schema-statements.py:

Line 203:   if row_format and file_format == 'text':
> Hm, I couldn't find anything in the docs or in our tests.
There are a few examples here:
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL


No idea whether we'll actually do the right thing


http://gerrit.cloudera.org:8080/#/c/4414/7/tests/query_test/test_kudu.py
File tests/query_test/test_kudu.py:

Line 204:         CREATE TABLE {table} (c INT PRIMARY KEY)
> That's interesting. Actually, currently don't work because we can't persist
File JIRA and defer.


Line 213:         TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}')""".format(
> Good question. If the user doesn't specify the kudu master_addresses we pop
Thanks for the explanation, seems fine.


Line 226:         DISTRIBUTE BY HASH (c) INTO 3 BUCKETS, RANGE (c) SPLIT ROWS (...)
> No, but Kudu's API doesn't currently return any information about range par
Got it, thanks.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 7
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 1:

(3 comments)

sorry, more comments

I keep finding new things as I'm adding some new test cases myself.

http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java
File fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java:

PS1, Line 87: 
            :   // Key to specify the number of tablet replicas.
            :   // TODO(KUDU): Allow modification in alter table.
            :   public static final String KEY_TABLET_REPLICAS = "kudu.num_tablet_replicas";
> I don't think we need to store this, even if we wanted to modify this later
After looking a bit more, I see that this is the way the user specifies the # of replicas and there is no other way. I guess it's weird to strip out table properties added by the user, so I suppose we should keep it. Instead, it'd be worth us considering separating out options that need to be persisted (i.e. we need the mapping to kudu tbl name) vs things that just get passed through and we shouldn't get in the business of trying to replicate in our own metadata. Probably not P1 but worth a JIRA since this is user facing and will sit in metadata.

Also:
1) the TODO(KUDU) that is here could be removed.
2) would be worth testing how this behaves with CREATE TABLE LIKE. I'd expect it to inherit this value but it'd be worth checking.


http://gerrit.cloudera.org:8080/#/c/4414/1/fe/src/test/java/com/cloudera/impala/analysis/AnalyzeDDLTest.java
File fe/src/test/java/com/cloudera/impala/analysis/AnalyzeDDLTest.java:

PS1, Line 28: import com.cloudera.impala.catalog.KuduTable;
            : import junit.framework.Assert;
nit: organize imports


http://gerrit.cloudera.org:8080/#/c/4414/1/testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
File testdata/workloads/functional-query/queries/QueryTest/create_kudu.test:

> we might still need to have explicit create table tests.
Also, I don't see any tests for CTAS here or anywhere.

Can you add a few positive (e.g. from kudu_functional.alltypes) and negative tests (e.g. involving datatypes that kudu doesn't support, e.g. functional.alltypes and functional.decimaltbl ).


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 1
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Alex Behm (Code Review)" <ge...@cloudera.org>.
Alex Behm has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 5:

(8 comments)

Responses to comments. Starting next round on code.

http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

Line 195:       msClient.alter_table(msTable_.getDbName(), msTable_.getTableName(), msTable_);
> Good point. We can potentially avoid this by checking if there has been som
Right, most of the time there will be no changes. I'm not only worried about the perf, but all the extra weird states we can be in when doing this extra RPC to modify the HMS.


Line 216:       String colName = ToSqlUtils.getIdentSql(colSchema.getName());
> Hm, can you explain why this is the case given that we will use this name t
We don't parse the column names in the HMS backend table and neither does Hive. The backticks are only needed to tell our parser to interpret a token as an identifier, but the parser is never invoked for column names from the HMS.

I recommend trying this out with some Impala keywords to see what happens.


Line 223:     numClusteringCols_ = primaryKeyColumnNames_.size();
> Actually, I removed this and left a TODO. There are a few places that inter
Yea, there's definitely some inconsistency here.


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

Line 1151:     // the metadata must be fetched from the Hive Metastore.
> In order to drop a table from Kudu we need the Kudu table name which is sto
Makes sense now that you've explained, please leave a brief comment.


Line 1172:       if (!KuduTable.isKuduTable(msTable)) continue;
> Well, it's an easy check we can do to avoid RPC calls to Kudu for dropping 
Got it, thanks!


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java:

Line 85:       kudu.createTable(kuduTableName, schema, tableOpts);
> I see what you're saying but I don't think this will work. We need to be ab
Agree. Can you file a JIRA against Kudu? Thanks!


Line 178:       if (kudu.tableExists(tableName)) {
> I get it. See my comment above and let me know what you think.
I agree with you.


Line 214:         String colName = ToSqlUtils.getIdentSql(colSchema.getName());
> I'd appreciate it if you could explain why is it wrong to use this here and
Explained in other comment, let me know if still unclear :)


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 5
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 5:

(41 comments)

http://gerrit.cloudera.org:8080/#/c/4414/5//COMMIT_MSG
Commit Message:

Line 45:    "KEY" is expected to be common enough that the ident style
> Might also want to mention that "key" is used for nested map types, so it's
Done


Line 51:    on the HMS database and table name. If the database is "default",
> Does this last regarding "default" make the code easier or more complicated
Not sure why the special case. Removing it unless someone objects.


http://gerrit.cloudera.org:8080/#/c/4414/5/be/src/service/frontend.cc
File be/src/service/frontend.cc:

Line 61: DEFINE_string(kudu_master_addrs, "", "Specifies the default Kudu master(s). The given "
> Can we make this more consistent with out existing options? For example:
Done


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
File fe/src/main/java/org/apache/impala/analysis/ColumnDef.java:

Line 176:   static List<String> toColumnNames(Collection<ColumnDef> columnDefs) {
> colDefs
Done


Line 184:   static Map<String, ColumnDef> mapByColumnNames(Collection<ColumnDef> colDefs) {
> comment that colDefs is assumed have no duplicate names, or alternatively d
Done


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java:

Line 99:           "does not support (%s) file format. Supported formats are: (%s)",
> the (%s) file format.
Done


Line 100:           createStmt_.getFileFormat().toString().replace("_", ""),
> what's with this weird "_" replacement?
I believe the intention was to make the file format name from THdfsFileFormat consistent with the file format name as specified in a "STORED AS" clause. RC_FILE is RCFILE, TEXT is TEXTFILE. So, as you can tell we aren't very consistent here. That's also the reason for hardcoding the values in L101. Left a TODO in the CatalogObjects.thrift for now if that's ok with you.


Line 101:           "PARQUET, TEXTFILE, KUDU"));
> use SUPPORTED_INSERT_FORMATS instead of hardcoding
See comment above.


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
File fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java:

Line 80:   private void setColumnDefs(List<ColumnDef> columnDefs) {
> nit: colDefs everywhere
Done


Line 95:   Map<String, String> getTblProperties() { return tableDef_.getTblProperties(); }
> is there any significance to not adding public/private to some of these met
Casey started using the default access modifier that essentially gives public access only to classes in the same package. In theory, we should be using the most restrictive access modifier, so I kind of like this. For sure the codebase is not consistent with using this rule, so if you think we shouldn't be doing this I'll make them public.


Line 192:           "Only Kudu tables can use DISTRIBUTE BY clause.");
> the DISTRIBUTE BY clause
Done


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/analysis/TableDef.java
File fe/src/main/java/org/apache/impala/analysis/TableDef.java:

Line 43:  * Represents the table parameters in a CREATE TABLE statement.
> Let's be more precise and list the clauses that this captures.
Done


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

Line 97:   // Table name in Kudu storage engine. It may not neccessarily be the same as the
> the Kudu storage engine
Done


Line 101:   // 1. For managed tables, 'kuduTableName_' is prefixed with '__IMPALA__<db_name>' to
> Can we make the prefix 'impala::<db_name>' instead?
Sure no prob. Done


Line 154:    * are loaded from HMS. The function also updates the table schema in HMS.
> Add a comment about why we also update the HMS. My understanding is that we
Done


Line 195:       msClient.alter_table(msTable_.getDbName(), msTable_.getTableName(), msTable_);
> This means every invalidate/refresh will do an extra alter to HMS. Are we c
Good point. We can potentially avoid this by checking if there has been some change in the schema. At this point, I am not so terribly worried about this. I'll keep a note to actually profile it and see if it's worth optimizing that path.


Line 202:    * Loads the schema from the Kudu table including column definitions and primary key
> mention that it loads the schema into this table as well as the HMS table
Done


Line 216:       String colName = ToSqlUtils.getIdentSql(colSchema.getName());
> getIdentSql() may add backticks, I don't think we want that here
Hm, can you explain why this is the case given that we will use this name to populate the columns in the HMS table?


Line 223:     numClusteringCols_ = primaryKeyColumnNames_.size();
> Check that there is at least one PK column
Actually, I removed this and left a TODO. There are a few places that interpret numClusteringCols_ in a way that is not consistent with Kudu tables. For example, the analysis in InsertStmt is using this value to determine if there is a result expr in the associated select stmt for every non-partitioned and partitioned column of the target table.


Line 228:     PartitionSchema partitionSchema = kuduTable.getPartitionSchema();
> Preconditions.checkState(!colsByPos_.isEmpty()); ?
Done


Line 254:   public static KuduTable createTmpTable(Db db,
> createCtasTarget()? Seems more explicit since there really is only one use.
Done


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

Line 1117:     if (db != null) dropTablesFromKudu(db);
> Won't this drop all Kudu tables even if CASCADE is not specified?
Oops. You're right. Done


Line 1151:     // the metadata must be fetched from the Hive Metastore.
> Why do we need to load from HMS? See comments below.
In order to drop a table from Kudu we need the Kudu table name which is stored in tblproperties. Let me know if it's clear or if I should leave a comment.


Line 1167:         throw new ImpalaRuntimeException(
> This seems like weird behavior. The user issued a cascading drop db, but ge
Makes sense. Changed it to log the error and continue.


Line 1172:       if (!KuduTable.isKuduTable(msTable)) continue;
> Why do we need this check? Won't Kudu tell us if the table does not exist? 
Well, it's an easy check we can do to avoid RPC calls to Kudu for dropping non-Kudu tables. msTables contains all the tables of a database.


Line 1438:     StorageDescriptor sd = HiveStorageDescriptorFactory.createSd(
> should we consolidate all the SD stuff into one createSd() call?
Done


Line 1453:     if (params.getPartition_columns() != null) {
> for my understanding: these are empty for Kudu tables right?
Yes


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java:

Line 41: //import org.apache.hadoop.hive.metastore.api.Table;
> ?
Done


Line 53:  * such as creating and droping tables from Kudu.
> typo: dropping
Done


Line 58:   public final static ColumnDef REQUIRED_THRIFT_COLUMN_DEF;
> Isn't it enough to set an empty list or mark the thrift field as set?
Yeah, I don't know why this was added in the first place. Removed.


Line 70:   static void createTable(org.apache.hadoop.hive.metastore.api.Table msTbl,
> createManagedTable()? and then mention that msTbl must be a managed table
Done


Line 85:       kudu.createTable(kuduTableName, schema, tableOpts);
> May this throw if the table already exists? If so it might be cleaner to re
I see what you're saying but I don't think this will work. We need to be able to handle the IF NOT EXISTS clause and there doesn't seem to exist a subclass of KuduException to indicate that createTable() failed because the table already exists. So, I can't find a clean way to differentiate between the error cases. What's needed here is Kudu API to take IF NOT EXISTS as parameter.


Line 97:       throws ImpalaRuntimeException {
> remove throws?
Hm, fromImpalaType throws so I don't think we can remove it here.


Line 121:       TCreateTableParams params, Schema schema) throws ImpalaRuntimeException {
> remove throws?
Same here, KuduUtil.parseSplits throws ImpalaRuntimeException.


Line 155:       int r = Integer.parseInt(replication);
> catch NumberFormatException?
Done


Line 170:     if (Table.isExternalTable(msTbl)) return;
> Can we make this a Precondition and have the caller check it?
Done


Line 178:       if (kudu.tableExists(tableName)) {
> same comment about non-atomic exists() + drop() calls
I get it. See my comment above and let me know what you think.


Line 214:         String colName = ToSqlUtils.getIdentSql(colSchema.getName());
> I don't think we want getIdentSql() here
I'd appreciate it if you could explain why is it wrong to use this here and/or what is the desired behavior in terms of column name.


Line 239:     } catch (Exception e) {
> I think we should be more nuanced in our exception handling here, in partic
Good point. Done


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/util/KuduUtil.java
File fe/src/main/java/org/apache/impala/util/KuduUtil.java:

Line 198:         metastoreTableName : "__IMPALA__" + metastoreDbName + "." + metastoreTableName;
> make the prefix a static constant (and change it to "impalad::" please)
Done


Line 240:       case BOOL: return Type.BOOLEAN;
> no DECIMAL?
noop :)


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 5
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 5:

(4 comments)

A few minor clarifications.

http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

Line 195:       msClient.alter_table(msTable_.getDbName(), msTable_.getTableName(), msTable_);
> Right, most of the time there will be no changes. I'm not only worried abou
In theory, it should be ok given that there is a single write path to HMS for Kudu table schema. If that changes, I agree we may have to deal with some weird inconsistent states.


Line 216:       String colName = ToSqlUtils.getIdentSql(colSchema.getName());
> We don't parse the column names in the HMS backend table and neither does H
Yeah, you're right. Fixed it in a follow up patch.


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

Line 1151:     // the metadata must be fetched from the Hive Metastore.
> Makes sense now that you've explained, please leave a brief comment.
Done


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java:

Line 85:       kudu.createTable(kuduTableName, schema, tableOpts);
> Agree. Can you file a JIRA against Kudu? Thanks!
Done


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 5
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has uploaded a new patch set (#5).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   "KEY" is expected to be common enough that the ident style
   declaration is used instead to avoid conflicts.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name. If the database is "default",
   then a Kudu table name will not include the database.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/catalog/HdfsStorageDescriptorTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/download_requirements
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
M testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/query_test/test_kudu.py
62 files changed, 2,972 insertions(+), 2,119 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/5
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 5
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 1:

(6 comments)

http://gerrit.cloudera.org:8080/#/c/4414/1/testdata/bin/generate-schema-statements.py
File testdata/bin/generate-schema-statements.py:

PS1, Line 196: primay
primary


http://gerrit.cloudera.org:8080/#/c/4414/1/tests/common/__init__.py
File tests/common/__init__.py:

does this need to be its own file?


http://gerrit.cloudera.org:8080/#/c/4414/1/tests/common/kudu_test_suite.py
File tests/common/kudu_test_suite.py:

PS1, Line 54:   @classmethod
            :   def get_db_name(cls):
            :     # When py.test runs with the xdist plugin, several processes are started and each
            :     # process runs some partition of the tests. It's possible that multiple processes
            :     # will call this method. A random value is generated so the processes won't try
            :     # to use the same database at the same time. The value is cached so within a single
            :     # process the same database name is always used for the class. This doesn't need to
            :     # be thread-safe since multi-threading is never used.
            :     if not cls.__DB_NAME:
            :       cls.__DB_NAME = \
            :           choice(ascii_lowercase) + "".join(sample(ascii_lowercase + digits, 5))
            :     return cls.__DB_NAME
I've always disliked this function... I think we should try to use the db fixture that Michael Brown added. Maybe I'm missing something though. It'd be good to get Michael's input here.


PS1, Line 127:     mapping = {BOOL: "BOOLEAN",
             :         DOUBLE: "DOUBLE",
             :         FLOAT: "FLOAT",
             :         INT16: "SMALLINT",
             :         INT32: "INT",
             :         INT64: "BIGINT",
             :         INT8: "TINYINT",
             :         STRING: "STRING"}
this gets defined as a local every function invocation, right? how about creating the map outside the fn so its static.


http://gerrit.cloudera.org:8080/#/c/4414/1/tests/custom_cluster/test_kudu.py
File tests/custom_cluster/test_kudu.py:

PS1, Line 25: TestRedaction
?


PS1, Line 42: temp_kudu_table
I'd think we could have the test fn take the database fixture and then pass it to temp_kudu_table


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 1
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 5:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

PS5, Line 193:       msTable_.putToParameters(StatsSetupConst.DO_NOT_UPDATE_STATS,
             :           StatsSetupConst.TRUE);
This also needs to be stripped out later for SHOW CREATE TABLE.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 5
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has uploaded a new patch set (#6).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   "KEY" is expected to be common enough that the ident style
   declaration is used instead to avoid conflicts.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name. If the database is "default",
   then a Kudu table name will not include the database.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/catalog/HdfsStorageDescriptorTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/download_requirements
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
M testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/query_test/test_kudu.py
63 files changed, 2,993 insertions(+), 2,124 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/6
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 6
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Hello Michael Brown, Alex Behm,

I'd like you to reexamine a change.  Please visit

    http://gerrit.cloudera.org:8080/4414

to look at the new patch set (#13).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   The ident style declaration is used for "KEY" because it is also used
   for nested map types.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
M fe/src/main/java/org/apache/impala/analysis/ColumnDef.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
M fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
M fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
A fe/src/main/java/org/apache/impala/analysis/TableDataLayout.java
A fe/src/main/java/org/apache/impala/analysis/TableDef.java
M fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
M fe/src/main/java/org/apache/impala/catalog/Catalog.java
M fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
M fe/src/main/java/org/apache/impala/catalog/Db.java
M fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/org/apache/impala/catalog/KuduTable.java
M fe/src/main/java/org/apache/impala/catalog/Table.java
M fe/src/main/java/org/apache/impala/catalog/TableLoader.java
M fe/src/main/java/org/apache/impala/catalog/Type.java
D fe/src/main/java/org/apache/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/org/apache/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/org/apache/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
M fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/service/Frontend.java
M fe/src/main/java/org/apache/impala/service/JniCatalog.java
M fe/src/main/java/org/apache/impala/service/JniFrontend.java
A fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
M fe/src/main/java/org/apache/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/org/apache/impala/analysis/ParserTest.java
M fe/src/test/java/org/apache/impala/service/JdbcTest.java
M fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/download_requirements
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
M testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/metadata/test_show_create_table.py
M tests/query_test/test_kudu.py
64 files changed, 3,138 insertions(+), 2,255 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/13
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 13
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 3:

(6 comments)

http://gerrit.cloudera.org:8080/#/c/4414/3/common/thrift/CatalogObjects.thrift
File common/thrift/CatalogObjects.thrift:

PS3, Line 350: // Parameters needed for hash distribution
             : struct TDistributeByHashParam {
             :   1: required list<string> columns
             :   2: required i32 num_buckets
             : }
             : 
             : struct TRangeLiteral {
             :   1: optional i64 int_literal
             :   2: optional string string_literal
             : }
             : 
             : struct TRangeLiteralList {
             :   1: required list<TRangeLiteral> values
             : }
             : 
             : // A range distribution is identified by a list of columns and a series of split rows.
             : struct TDistributeByRangeParam {
             :   1: required list<string> columns
             :   2: optional list<TRangeLiteralList> split_rows;
             : }
             : 
             : // Parameters for the DISTRIBUTE BY clause. The actual distribution is identified by
             : // the type parameter.
             : struct TDistributeParam {
             :   // Set if type is set to HASH
             :   1: optional TDistributeByHashParam by_hash_param;
             : 
             :   // Set if type is set to RANGE
             :   2: optional TDistributeByRangeParam by_range_param;
             : }
> We store the distribute by params in the KuduTable class which is then seri
I don't see why this needs to be in the TKuduTable though. I think we just need this in the TCreateTableParams. I wouldn't mind except it's more work to reason about when it's set/not set in a TKuduTable.


PS3, Line 390: 
             :   // Distribution schemes
             :   4: required list<TDistributeParam> distribute_by
> See my comment above. We do load and serialize the distribute by params in 
I don't see why this needs to be in TKuduTable, only TCreateTableParams.


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/DistributeParam.java
File fe/src/main/java/org/apache/impala/analysis/DistributeParam.java:

PS3, Line 156:       if (splitRows_ == null) {
             :         builder.append("...");
> This is possible because an object of DistributeParam class is created in t
In case #2 why do we even bother creating a DistributeParams? 

The partitioning/distribution info only really makes sense at the time of table creation I think, right? This goes back to our conversation (in person) about how it'd probably make sense to separate the partitioning part of the command from CREATE TABLE, and do partitioning more like how we would for non-Kudu tables, i.e. separate commands. Obviously that hasn't been worked through and realistically we probably won't get to that in time.

If we don't get to something like that, I think the only reason we'd want #2 (in terms of loading DistributeParams) is SHOW CREATE TABLE, so this class only needs to support toSql() in case #2. Is that right? Can you add a comment in the class header or the relevant constructor where we might get case #2 so it's easier to reason about?


A TODO / JIRA ref about resolving the '...' would be good. It might be good to add a test case with this output if there isn't one already to make sure we fix it properly and/or it doesn't change in some other unexpected way.


PS3, Line 191:       if (splitRows_ == null) {
             :         result.setBy_range_param(rangeParam);
             :         return result;
             :       }
> See explanation above.
I guess it's confusing to me because TDistributeParam doesn't seem to be necessary in case #2.

In case #1 it gets added to a TCreateTableStmt object. In case #2 it gets added to a TKuduTable object, but it seems unnecessary given that no other backend or even the catalog care about the distribute params - right? If that's true, I think we could avoid storing this in KuduTable, then avoid calling this in KuduTable.getTKuduTable(), and then throwing here if splitRows_ is null since I don't think this should be called in this case.


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/TableDef.java
File fe/src/main/java/org/apache/impala/analysis/TableDef.java:

PS3, Line 126: return fullyQualifiedTableName_ != null ? fullyQualifiedTableName_ : tableName_;
> We do call this function before tableName_ has been analyzed. One example i
ah ok, can you add a comment?


http://gerrit.cloudera.org:8080/#/c/4414/3/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
File fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java:

PS3, Line 243: List<String> primaryKeysSql, String kuduDistributeByParams,
can you update the comment, especially around which of these should/can be null vs. empty.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 3
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Dimitris Tsirogiannis (Code Review)" <ge...@cloudera.org>.
Dimitris Tsirogiannis has uploaded a new patch set (#2).

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................

IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

With this commit we simplify the syntax and handling of CREATE TABLE
statements for both managed and external Kudu tables.

Syntax example:
CREATE TABLE foo(a INT, b STRING, PRIMARY KEY (a, b))
DISTRIBUTE BY HASH (a) INTO 3 BUCKETS,
RANGE (b) SPLIT ROWS (('abc', 'def'))
STORED AS KUDU

Changes:
1) Remove the requirement to specify table properties such as key
   columns in tblproperties.
2) Read table schema (column definitions, primary keys, and distribution
   schemes) from Kudu instead of the HMS.
3) For external tables, the Kudu table is now required to exist at the
   time of creation in Impala.
4) Disallow table properties that could conflict with an existing
   table. Ex: key_columns cannot be specified.
5) Add KUDU as a file format.
6) Add a startup flag to impalad to specify the default Kudu master
   addresses. The flag is used as the default value for the table
   property kudu_master_addresses but it can still be overriden
   using TBLPROPERTIES.
7) Fix a post merge issue (IMPALA-3178) where DROP DATABASE CASCADE
   wasn't implemented for Kudu tables and silently ignored. The Kudu
   tables wouldn't be removed in Kudu.
8) Remove DDL delegates. There was only one functional delegate (for
   Kudu) the existence of the other delegate and the use of delegates in
   general has led to confusion. The Kudu delegate only exists to provide
   functionality missing from Hive.
9) Add PRIMARY KEY at the column and table level. This syntax is fairly
   standard. When used at the column level, only one column can be
   marked as a key. When used at the table level, multiple columns can
   be used as a key. Only Kudu tables are allowed to use PRIMARY KEY.
   The old "kudu.key_columns" table property is no longer accepted
   though it is still used internally. "PRIMARY" is now a keyword.
   "KEY" is expected to be common enough that the ident style
   declaration is used instead to avoid conflicts.
10) For managed tables, infer a Kudu table name if none was given.
   The table property "kudu.table_name" is optional for managed tables
   and is required for external tables. If for a managed table a Kudu
   table name is not provided, a table name will be generated based
   on the HMS database and table name. If the database is "default",
   then a Kudu table name will not include the database.
11) Use Kudu master as the source of truth for table metadata instead
   of HMS when a table is loaded or refreshed. Table/column metadata
   are cached in the catalog and are stored in HMS in order to be
   able to use table and column statistics.

Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
---
M be/src/service/frontend.cc
M bin/start-catalogd.sh
M bin/start-impala-cluster.py
M common/thrift/CatalogObjects.thrift
M common/thrift/JniCatalog.thrift
M fe/src/main/cup/sql-parser.cup
A fe/src/main/java/com/cloudera/impala/analysis/AnalysisUtils.java
M fe/src/main/java/com/cloudera/impala/analysis/ColumnDef.java
A fe/src/main/java/com/cloudera/impala/analysis/ColumnDefOptions.java
M fe/src/main/java/com/cloudera/impala/analysis/CreateTableAsSelectStmt.java
M fe/src/main/java/com/cloudera/impala/analysis/CreateTableDataSrcStmt.java
M fe/src/main/java/com/cloudera/impala/analysis/CreateTableLikeFileStmt.java
M fe/src/main/java/com/cloudera/impala/analysis/CreateTableLikeStmt.java
M fe/src/main/java/com/cloudera/impala/analysis/CreateTableStmt.java
M fe/src/main/java/com/cloudera/impala/analysis/DistributeParam.java
M fe/src/main/java/com/cloudera/impala/analysis/ModifyStmt.java
A fe/src/main/java/com/cloudera/impala/analysis/TableDataLayout.java
A fe/src/main/java/com/cloudera/impala/analysis/TableDef.java
M fe/src/main/java/com/cloudera/impala/analysis/ToSqlUtils.java
M fe/src/main/java/com/cloudera/impala/catalog/Catalog.java
M fe/src/main/java/com/cloudera/impala/catalog/Db.java
M fe/src/main/java/com/cloudera/impala/catalog/HdfsFileFormat.java
M fe/src/main/java/com/cloudera/impala/catalog/ImpaladCatalog.java
M fe/src/main/java/com/cloudera/impala/catalog/KuduTable.java
M fe/src/main/java/com/cloudera/impala/catalog/Table.java
M fe/src/main/java/com/cloudera/impala/catalog/TableLoader.java
D fe/src/main/java/com/cloudera/impala/catalog/delegates/DdlDelegate.java
D fe/src/main/java/com/cloudera/impala/catalog/delegates/KuduDdlDelegate.java
D fe/src/main/java/com/cloudera/impala/catalog/delegates/UnsupportedOpDelegate.java
M fe/src/main/java/com/cloudera/impala/planner/HdfsPartitionFilter.java
M fe/src/main/java/com/cloudera/impala/service/CatalogOpExecutor.java
M fe/src/main/java/com/cloudera/impala/service/Frontend.java
M fe/src/main/java/com/cloudera/impala/service/JniCatalog.java
M fe/src/main/java/com/cloudera/impala/service/JniFrontend.java
A fe/src/main/java/com/cloudera/impala/service/KuduCatalogOpExecutor.java
A fe/src/main/java/com/cloudera/impala/util/KuduClient.java
M fe/src/main/java/com/cloudera/impala/util/KuduUtil.java
M fe/src/main/jflex/sql-scanner.flex
M fe/src/test/java/com/cloudera/impala/analysis/AnalyzeDDLTest.java
M fe/src/test/java/com/cloudera/impala/analysis/ParserTest.java
M fe/src/test/java/com/cloudera/impala/service/JdbcTest.java
M fe/src/test/java/com/cloudera/impala/testutil/ImpaladTestCatalog.java
M infra/python/deps/requirements.txt
M testdata/bin/generate-schema-statements.py
M testdata/datasets/functional/functional_schema_template.sql
M testdata/datasets/tpch/tpch_schema_template.sql
D testdata/workloads/functional-query/queries/QueryTest/create_kudu.test
M testdata/workloads/functional-query/queries/QueryTest/kudu-scan-node.test
D testdata/workloads/functional-query/queries/QueryTest/kudu-show-create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
A testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_crud.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
M testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
M tests/common/__init__.py
A tests/common/kudu_test_suite.py
M tests/conftest.py
A tests/custom_cluster/test_kudu.py
M tests/metadata/test_ddl.py
M tests/query_test/test_kudu.py
60 files changed, 2,966 insertions(+), 2,075 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/14/4414/2
-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 2
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Michael Brown (Code Review)" <ge...@cloudera.org>.
Michael Brown has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 5:

(7 comments)

http://gerrit.cloudera.org:8080/#/c/4414/5/bin/start-impala-cluster.py
File bin/start-impala-cluster.py:

PS5, Line 73: parser.add_option("--kudu_masters", default="127.0.0.1",
Could we import the default value from tests.common instead of hardcoding it?


http://gerrit.cloudera.org:8080/#/c/4414/5/tests/common/kudu_test_suite.py
File tests/common/kudu_test_suite.py:

PS5, Line 38: @SkipIf.kudu_not_supported
This is extremely risky due to bugs in our version of old pytest. See IMPALA-3614 for an example of how this is risky. I think you will have to work around this by skipping in setup_class.


http://gerrit.cloudera.org:8080/#/c/4414/5/tests/custom_cluster/test_kudu.py
File tests/custom_cluster/test_kudu.py:

PS5, Line 1: # Copyright (c) 2016 Cloudera, Inc. All rights reserved.
Remove Cloudera copyright and make sure the license text is correct otherwise.


http://gerrit.cloudera.org:8080/#/c/4414/5/tests/metadata/test_ddl.py
File tests/metadata/test_ddl.py:

PS5, Line 220: 
Why remove this skip?


PS5, Line 221: 
Is this removal safe?


PS5, Line 221:     self.expected_exceptions = 2
I can't see where this is used; delete?


http://gerrit.cloudera.org:8080/#/c/4414/5/tests/query_test/test_kudu.py
File tests/query_test/test_kudu.py:

PS5, Line 87:         print("Describe formatted output:")
            :         pprint(table_desc)
Use LOG.info()


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 5
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables

Posted by "Matthew Jacobs (Code Review)" <ge...@cloudera.org>.
Matthew Jacobs has posted comments on this change.

Change subject: IMPALA-3719: Simplify CREATE TABLE statements with Kudu tables
......................................................................


Patch Set 5:

(4 comments)

http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/catalog/KuduTable.java
File fe/src/main/java/org/apache/impala/catalog/KuduTable.java:

PS4, Line 175: }
> Done
While it does seem related, does it really mean the same thing?

Either way, we have to be sure we're handling it correctly everywhere. E.g. does  Table.getClusteringColumns(), getNonClusteringColumns(), work as expected?
They also get stored in the thrift TTable differently, see Table.java:


      // Clustering columns come first.
      if (i < numClusteringCols_) {
        table.addToClustering_columns(colDesc);
      } else {
        table.addToColumns(colDesc);
      }


Further, Table.getNumClusteringCols() exposes this and it is used in a lot of places.

E.g. TupleDescriptor.java:
 
  public boolean hasClusteringColsOnly() {
    Table table = getTable();
    if (!(table instanceof HdfsTable) || table.getNumClusteringCols() == 0) return false;
    ...


There are a ton of other places to check.


PS4, Line 226:   private void loadDistributeByParams(org.apache.kudu.client.KuduTable kuduTable) {
             :     Preconditions
> cols is a reference to msTable cols. We clear them here and reload them fro
I see. yes, a brief comment would be helpful.


http://gerrit.cloudera.org:8080/#/c/4414/5/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java:

PS5, Line 1484:         // Add the table to the catalog cache
              :         Table newTbl = catalog_.addTable(newTable.getDbName(), newTable.getTableName());
              :         addTableToCatalogUpdate(newTbl, response.result);
If something here throws (e.g. addTable() looks like it could throw), we probably need to roll back the hiveClient.createTable() call, right?


http://gerrit.cloudera.org:8080/#/c/4414/4/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
File fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java:

PS4, Line 135:           tableOpts.setRangePartitionColumns(
             :               distParam.getBy_range_param().getColumns());
             :        
> Unfortunately it is. I spoke to Dan (from Kudu team) about it. If the user 
Thanks, I followed up with an e-mail to Dan to make sure I understand the Kudu behavior.


-- 
To view, visit http://gerrit.cloudera.org:8080/4414
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I7b9d51b2720ab57649abdb7d5c710ea04ff50dc1
Gerrit-PatchSet: 5
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Alex Behm <al...@cloudera.com>
Gerrit-Reviewer: Dimitris Tsirogiannis <dt...@cloudera.com>
Gerrit-Reviewer: Marcel Kornacker <ma...@cloudera.com>
Gerrit-Reviewer: Matthew Jacobs <mj...@cloudera.com>
Gerrit-Reviewer: Michael Brown <mi...@cloudera.com>
Gerrit-HasComments: Yes