You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ph...@apache.org on 2018/01/23 00:43:55 UTC

[6/6] impala git commit: IMPALA-5654: Disallow setting Kudu table name in CREATE TABLE

IMPALA-5654: Disallow setting Kudu table name in CREATE TABLE

This change disallows explicitly setting the Kudu table name property
for managed Kudu tables in a CREATE TABLE statement. The Kudu table
name property gets a generated value as the following:
'impala::db_name.table_name' where table_name is the one given in
the CREATE TABLE statement.
Providing the Kudu table name property when creating a managed Kudu
table results in an error without creating the table. E.g.:
CREATE TABLE t (i INT) STORED AS KUDU
  TBLPROPERTIES('kudu.table_name'='some_name');

Alongside the CREATE TABLE statement also the ALTER TABLE statement
is changed not to allow the modification of Kudu.table_name of
managed Kudu tables.

Change-Id: Ieca037498abf8f5fde67b77e824b720482cdbe6f
Reviewed-on: http://gerrit.cloudera.org:8080/8820
Reviewed-by: Dimitris Tsirogiannis <dt...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: 7f652ce659142abf1cd908196be1b56eed30bcbd
Parents: c959355
Author: Gabor Kaszab <ga...@cloudera.com>
Authored: Tue Dec 12 09:46:21 2017 +0100
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Mon Jan 22 22:34:30 2018 +0000

----------------------------------------------------------------------
 .../analysis/AlterTableSetTblProperties.java    | 36 ++++++---
 .../apache/impala/analysis/CreateTableStmt.java | 34 ++++++--
 .../org/apache/impala/analysis/TableDef.java    |  9 +++
 .../apache/impala/analysis/AnalyzeDDLTest.java  | 22 ++++--
 .../org/apache/impala/analysis/ToSqlTest.java   |  6 +-
 .../queries/QueryTest/kudu_alter.test           | 73 ++++++++---------
 tests/query_test/test_kudu.py                   | 83 ++++++++++----------
 7 files changed, 157 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/7f652ce6/fe/src/main/java/org/apache/impala/analysis/AlterTableSetTblProperties.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterTableSetTblProperties.java b/fe/src/main/java/org/apache/impala/analysis/AlterTableSetTblProperties.java
index fb83f24..732f5f6 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AlterTableSetTblProperties.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterTableSetTblProperties.java
@@ -91,17 +91,7 @@ public class AlterTableSetTblProperties extends AlterTableSetStmt {
           hive_metastoreConstants.META_TABLE_STORAGE));
     }
 
-    if (getTargetTable() instanceof KuduTable && analyzer.getAuthzConfig().isEnabled()) {
-      // Checking for 'EXTERNAL' is case-insensitive, see IMPALA-5637.
-      boolean setsExternal =
-          MetaStoreUtil.findTblPropKeyCaseInsensitive(tblProperties_, "EXTERNAL") != null;
-      if (setsExternal || tblProperties_.containsKey(KuduTable.KEY_MASTER_HOSTS)) {
-        String authzServer = analyzer.getAuthzConfig().getServerName();
-        Preconditions.checkNotNull(authzServer);
-        analyzer.registerPrivReq(new PrivilegeRequestBuilder().onServer(
-            authzServer).all().toRequest());
-      }
-    }
+    if (getTargetTable() instanceof KuduTable) analyzeKuduTable(analyzer);
 
     // Check avro schema when it is set in avro.schema.url or avro.schema.literal to
     // avoid potential metadata corruption (see IMPALA-2042).
@@ -121,6 +111,30 @@ public class AlterTableSetTblProperties extends AlterTableSetStmt {
     analyzeSortColumns(getTargetTable(), tblProperties_);
   }
 
+  private void analyzeKuduTable(Analyzer analyzer) throws AnalysisException {
+    // Checking for 'EXTERNAL' is case-insensitive, see IMPALA-5637.
+    String keyForExternalProperty =
+        MetaStoreUtil.findTblPropKeyCaseInsensitive(tblProperties_, "EXTERNAL");
+
+    // Throw error if kudu.table_name is provided for managed Kudu tables
+    // TODO IMPALA-6375: Allow setting kudu.table_name for managed Kudu tables
+    // if the 'EXTERNAL' property is set to TRUE in the same step.
+    if (!Table.isExternalTable(table_.getMetaStoreTable())) {
+      AnalysisUtils.throwIfNotNull(tblProperties_.get(KuduTable.KEY_TABLE_NAME),
+          String.format("Not allowed to set '%s' manually for managed Kudu tables .",
+              KuduTable.KEY_TABLE_NAME));
+    }
+    if (analyzer.getAuthzConfig().isEnabled()) {
+      if (keyForExternalProperty != null ||
+          tblProperties_.containsKey(KuduTable.KEY_MASTER_HOSTS)) {
+        String authzServer = analyzer.getAuthzConfig().getServerName();
+        Preconditions.checkNotNull(authzServer);
+        analyzer.registerPrivReq(new PrivilegeRequestBuilder().onServer(
+            authzServer).all().toRequest());
+      }
+    }
+  }
+
   /**
    * Check that Avro schema provided in avro.schema.url or avro.schema.literal is valid
    * Json and contains only supported Impala types. If both properties are set, then

http://git-wip-us.apache.org/repos/asf/impala/blob/7f652ce6/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
index 5810a40..b4d59e8 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
@@ -41,6 +41,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import com.google.common.primitives.Ints;
+import com.google.common.collect.Maps;
 
 /**
  * Represents a CREATE TABLE statement.
@@ -111,6 +112,12 @@ public class CreateTableStmt extends StatementBase {
   Map<String, String> getSerdeProperties() { return tableDef_.getSerdeProperties(); }
   public THdfsFileFormat getFileFormat() { return tableDef_.getFileFormat(); }
   RowFormat getRowFormat() { return tableDef_.getRowFormat(); }
+  private String getGeneratedKuduTableName() {
+    return tableDef_.getGeneratedKuduTableName();
+  }
+  private void setGeneratedKuduTableName(String tableName) {
+    tableDef_.setGeneratedKuduTableName(tableName);
+  }
 
   // Only exposed for ToSqlUtils. Returns the list of primary keys declared by the user
   // at the table level. Note that primary keys may also be declared in column
@@ -158,7 +165,11 @@ public class CreateTableStmt extends StatementBase {
     params.setFile_format(getFileFormat());
     params.setIf_not_exists(getIfNotExists());
     params.setSort_columns(getSortColumns());
-    params.setTable_properties(getTblProperties());
+    params.setTable_properties(Maps.newHashMap(getTblProperties()));
+    if (!getGeneratedKuduTableName().isEmpty()) {
+      params.getTable_properties().put(KuduTable.KEY_TABLE_NAME,
+          getGeneratedKuduTableName());
+    }
     params.setSerde_properties(getSerdeProperties());
     for (KuduPartitionParam d: getKuduPartitionParams()) {
       params.addToPartition_by(d.toThrift());
@@ -294,13 +305,8 @@ public class CreateTableStmt extends StatementBase {
    * Analyzes and checks parameters specified for managed Kudu tables.
    */
   private void analyzeManagedKuduTableParams(Analyzer analyzer) throws AnalysisException {
-    // If no Kudu table name is specified in tblproperties, generate one using the
-    // current database as a prefix to avoid conflicts in Kudu.
-    // TODO: Disallow setting this manually for managed tables (IMPALA-5654).
-    if (!getTblProperties().containsKey(KuduTable.KEY_TABLE_NAME)) {
-      getTblProperties().put(KuduTable.KEY_TABLE_NAME,
-          KuduUtil.getDefaultCreateKuduTableName(getDb(), getTbl()));
-    }
+    analyzeManagedKuduTableName();
+
     // Check column types are valid Kudu types
     for (ColumnDef col: getColumnDefs()) {
       try {
@@ -338,6 +344,18 @@ public class CreateTableStmt extends StatementBase {
   }
 
   /**
+   * Generates a Kudu table name based on the target database and table and stores
+   * it in TableDef.generatedKuduTableName_. Throws if the Kudu table
+   * name was given manually via TBLPROPERTIES.
+   */
+  private void analyzeManagedKuduTableName() throws AnalysisException {
+    AnalysisUtils.throwIfNotNull(getTblProperties().get(KuduTable.KEY_TABLE_NAME),
+        String.format("Not allowed to set '%s' manually for managed Kudu tables .",
+            KuduTable.KEY_TABLE_NAME));
+    setGeneratedKuduTableName(KuduUtil.getDefaultCreateKuduTableName(getDb(), getTbl()));
+  }
+
+  /**
    * Analyzes the partitioning schemes specified in the CREATE TABLE statement.
    */
   private void analyzeKuduPartitionParams(Analyzer analyzer) throws AnalysisException {

http://git-wip-us.apache.org/repos/asf/impala/blob/7f652ce6/fe/src/main/java/org/apache/impala/analysis/TableDef.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/TableDef.java b/fe/src/main/java/org/apache/impala/analysis/TableDef.java
index 178a976..915bbba 100644
--- a/fe/src/main/java/org/apache/impala/analysis/TableDef.java
+++ b/fe/src/main/java/org/apache/impala/analysis/TableDef.java
@@ -88,6 +88,9 @@ class TableDef {
   // True if analyze() has been called.
   private boolean isAnalyzed_ = false;
 
+  //Kudu table name generated during analysis for managed Kudu tables
+  private String generatedKuduTableName_ = "";
+
   // END: Members that need to be reset()
   /////////////////////////////////////////
 
@@ -160,6 +163,7 @@ class TableDef {
     dataLayout_.reset();
     columnDefs_.clear();
     isAnalyzed_ = false;
+    generatedKuduTableName_ = "";
   }
 
   public TableName getTblName() {
@@ -183,6 +187,11 @@ class TableDef {
   List<ColumnDef> getPrimaryKeyColumnDefs() { return primaryKeyColDefs_; }
   boolean isExternal() { return isExternal_; }
   boolean getIfNotExists() { return ifNotExists_; }
+  String getGeneratedKuduTableName() { return generatedKuduTableName_; }
+  void setGeneratedKuduTableName(String tableName) {
+    Preconditions.checkNotNull(tableName);
+    generatedKuduTableName_ = tableName;
+  }
   List<KuduPartitionParam> getKuduPartitionParams() {
     return dataLayout_.getKuduPartitionParams();
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/7f652ce6/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
index 72b7f58..895d8c5 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -2079,9 +2079,16 @@ public class AnalyzeDDLTest extends FrontendTestBase {
         "Cannot change the type of a Kudu column using an ALTER TABLE CHANGE COLUMN " +
         "statement: (INT vs BIGINT)");
 
-    // Rename the underlying Kudu table
-    AnalyzesOk("ALTER TABLE functional_kudu.testtbl SET " +
-        "TBLPROPERTIES ('kudu.table_name' = 'Hans')");
+    // Rename the underlying Kudu table is not supported for managed Kudu tables
+    AnalysisError("ALTER TABLE functional_kudu.testtbl SET " +
+        "TBLPROPERTIES ('kudu.table_name' = 'Hans')",
+        "Not allowed to set 'kudu.table_name' manually for managed Kudu tables");
+
+    // TODO IMPALA-6375: Allow setting kudu.table_name for managed Kudu tables
+    // if the 'EXTERNAL' property is set to TRUE in the same step.
+    AnalysisError("ALTER TABLE functional_kudu.testtbl SET " +
+        "TBLPROPERTIES ('EXTERNAL' = 'TRUE','kudu.table_name' = 'Hans')",
+        "Not allowed to set 'kudu.table_name' manually for managed Kudu tables");
 
     // ALTER TABLE RENAME TO
     AnalyzesOk("ALTER TABLE functional_kudu.testtbl RENAME TO new_testtbl");
@@ -2242,11 +2249,14 @@ public class AnalyzeDDLTest extends FrontendTestBase {
         "partition by range(unknown_column) (partition value = 'abc') stored as kudu",
         "Column 'unknown_column' in 'RANGE (unknown_column) (PARTITION VALUE = 'abc')' " +
         "is not a key column. Only key columns can be used in PARTITION BY");
-    // Kudu table name is specified in tblproperties
+    // Kudu num_tablet_replicas is specified in tblproperties
     AnalyzesOk("create table tab (x int primary key) partition by hash (x) " +
-        "partitions 8 stored as kudu tblproperties ('kudu.table_name'='tab_1'," +
-        "'kudu.num_tablet_replicas'='1'," +
+        "partitions 8 stored as kudu tblproperties ('kudu.num_tablet_replicas'='1'," +
         "'kudu.master_addresses' = '127.0.0.1:8080, 127.0.0.1:8081')");
+    // Kudu table name is specified in tblproperties resulting in an error
+    AnalysisError("create table tab (x int primary key) partition by hash (x) " +
+        "partitions 8 stored as kudu tblproperties ('kudu.table_name'='tab')",
+        "Not allowed to set 'kudu.table_name' manually for managed Kudu tables");
     // No port is specified in kudu master address
     AnalyzesOk("create table tdata_no_port (id int primary key, name string, " +
         "valf float, vali bigint) partition by range(id) (partition values <= 10, " +

http://git-wip-us.apache.org/repos/asf/impala/blob/7f652ce6/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
index 6105925..bb6a39b 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
@@ -92,12 +92,12 @@ public class ToSqlTest extends FrontendTestBase {
   }
 
   private void testToSql(String query, String defaultDb, String expected,
-      boolean ignore_whitespace) {
+      boolean ignoreWhitespace) {
     String actual = null;
     try {
       ParseNode node = AnalyzesOk(query, createAnalyzer(defaultDb));
       actual = node.toSql();
-      if (ignore_whitespace) {
+      if (ignoreWhitespace) {
         // Transform whitespace to single space.
         actual = actual.replace('\n', ' ').replaceAll(" +", " ").trim();
       }
@@ -315,7 +315,6 @@ public class ToSqlTest extends FrontendTestBase {
         "CREATE TABLE default.p ( a BIGINT PRIMARY KEY, b TIMESTAMP " +
         "DEFAULT '1987-05-19' ) PARTITION BY HASH (a) PARTITIONS 3 " +
         "STORED AS KUDU TBLPROPERTIES ('kudu.master_addresses'='foo', " +
-        "'kudu.table_name'='impala::default.p', " +
         "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler')", true);
   }
 
@@ -348,7 +347,6 @@ public class ToSqlTest extends FrontendTestBase {
         "CREATE TABLE default.p PRIMARY KEY (a, b) PARTITION BY HASH (a) PARTITIONS 3, " +
         "RANGE (b) (PARTITION VALUE = 1) STORED AS KUDU TBLPROPERTIES " +
         "('kudu.master_addresses'='foo', " +
-        "'kudu.table_name'='impala::default.p', " +
         "'storage_handler'='com.cloudera.kudu.hive.KuduStorageHandler') AS " +
         "SELECT int_col a, bigint_col b FROM functional.alltypes", true);
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/7f652ce6/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
index 04f104f..305ccf1 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
@@ -344,20 +344,6 @@ select id, last_name from tbl_to_alter
 INT,STRING
 ====
 ---- QUERY
-# Rename the underlying Kudu table
-alter table tbl_to_alter set tblproperties('kudu.table_name'='kudu_tbl_to_alter')
----- RESULTS
-'Updated table.'
-====
----- QUERY
-# Create a new table and try to rename to an existing kudu table
-create table copy_of_tbl (a int primary key) partition by hash (a) partitions 3
-  stored as kudu tblproperties('kudu.table_name'='copy_of_tbl');
-alter table copy_of_tbl set tblproperties('kudu.table_name'='kudu_tbl_to_alter')
----- CATCH
-ImpalaRuntimeException: Error renaming Kudu table copy_of_tbl to kudu_tbl_to_alter
-====
----- QUERY
 # Ensure the Kudu table is accessible
 select count(*) from tbl_to_alter
 ---- RESULTS
@@ -379,34 +365,49 @@ select count(*) from kudu_tbl_to_alter
 BIGINT
 ====
 ---- QUERY
-# Rename Kudu table and insert a number of rows
-alter table copy_of_tbl set tblproperties('kudu.table_name'='shared_kudu_tbl');
-insert into copy_of_tbl values (1), (2), (3)
+# Create an external Kudu table pointing to an existing Kudu table
+create external table external_tbl stored as kudu
+  tblproperties('kudu.table_name'='impala::$DATABASE.tbl_to_alter');
+select count(*) from external_tbl
+---- RESULTS
+5
+---- TYPES
+BIGINT
+====
+---- QUERY
+# Insert an item into the table pointed by the external Kudu table
+insert into kudu_tbl_to_alter (id, name, new_col1, new_col2)
+  values (7, 'test', 4, 400)
 ---- RUNTIME_PROFILE
-NumModifiedRows: 3
+NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-A
----- DML_RESULTS: copy_of_tbl
-1
-2
-3
+ID, NAME, NEW_COL1, NEW_COL2, LAST_NAME, NEW_COL4
+---- DML_RESULTS: kudu_tbl_to_alter
+2,'test',1,100,'NULL',-1
+3,'test',10,1000,'NULL',-1
+4,'test',1,100,'NULL',NULL
+5,'test',2,200,'names',1
+6,'test',3,300,'NULL',-1
+7,'test',4,400,'NULL',-1
 ---- TYPES
-INT
+INT,STRING,INT,BIGINT,STRING,INT
 ====
 ---- QUERY
-# Create an external table
-create external table external_tbl stored as kudu
-tblproperties('kudu.table_name'='kudu_tbl_to_alter');
-select count(*) from external_tbl
+# After an insert into the underlying table check if the row count of the
+# external table pointing to it also increased.
+select count(*) from external_tbl;
 ---- RESULTS
-5
+6
 ---- TYPES
 BIGINT
 ====
 ---- QUERY
 # Change the external table to point to a different Kudu table
-alter table external_tbl set tblproperties('kudu.table_name'='shared_kudu_tbl');
+create table temp_kudu_table (i int primary key) stored as kudu;
+insert into temp_kudu_table values (1), (2), (3);
+alter table external_tbl set
+  tblproperties('kudu.table_name'='impala::$DATABASE.temp_kudu_table');
 select count(*) from external_tbl
 ---- RESULTS
 3
@@ -464,7 +465,7 @@ alter table kudu_tbl_to_alter alter column new_col1 set default 10 + 5;
 ---- QUERY
 # check that the above defaults are applied
 insert into kudu_tbl_to_alter (id, last_name, new_col4)
-values (7, 'test', 8)
+values (8, 'test', 8)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
@@ -474,7 +475,8 @@ NumRowErrors: 0
 4,'test',1,100,'NULL',NULL
 5,'test',2,200,'names',1
 6,'test',3,300,'NULL',-1
-7,'name_default',15,1000,'test',8
+7,'test',4,400,'NULL',-1
+8,'name_default',15,1000,'test',8
 ---- LABELS
 ID,NAME,NEW_COL1,NEW_COL2,LAST_NAME,NEW_COL4
 ---- TYPES
@@ -530,7 +532,7 @@ STRING,STRING,STRING,STRING,STRING,STRING,STRING,STRING,STRING
 ---- QUERY
 # check that we can insert and scan after the storage attribute changes
 insert into kudu_tbl_to_alter (id, name, new_col1, new_col2, last_name, new_col4)
-values (8, 'nine', 10, 11, 'twelve', 13)
+values (9, 'nine', 10, 11, 'twelve', 13)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
@@ -540,8 +542,9 @@ NumRowErrors: 0
 4,'test',1,100,'NULL',NULL
 5,'test',2,200,'names',1
 6,'test',3,300,'NULL',-1
-7,'name_default',15,1000,'test',8
-8,'nine',10,11,'twelve',13
+7,'test',4,400,'NULL',-1
+8,'name_default',15,1000,'test',8
+9,'nine',10,11,'twelve',13
 ---- LABELS
 ID,NAME,NEW_COL1,NEW_COL2,LAST_NAME,NEW_COL4
 ---- TYPES

http://git-wip-us.apache.org/repos/asf/impala/blob/7f652ce6/tests/query_test/test_kudu.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_kudu.py b/tests/query_test/test_kudu.py
index f8a761a..ddf59f2 100644
--- a/tests/query_test/test_kudu.py
+++ b/tests/query_test/test_kudu.py
@@ -318,18 +318,6 @@ class TestKuduOperations(KuduTestSuite):
     cursor.execute("SELECT * FROM %s.foo" % (unique_database))
     assert cursor.fetchall() == [(0, )]
 
-  def test_kudu_rename_table(self, cursor, kudu_client, unique_database):
-    """Test Kudu table rename"""
-    cursor.execute("""CREATE TABLE %s.foo (a INT PRIMARY KEY) PARTITION BY HASH(a)
-        PARTITIONS 3 STORED AS KUDU""" % unique_database)
-    kudu_tbl_name = KuduTestSuite.to_kudu_table_name(unique_database, "foo")
-    assert kudu_client.table_exists(kudu_tbl_name)
-    new_kudu_tbl_name = "blah"
-    cursor.execute("ALTER TABLE %s.foo SET TBLPROPERTIES('kudu.table_name'='%s')" % (
-        unique_database, new_kudu_tbl_name))
-    assert kudu_client.table_exists(new_kudu_tbl_name)
-    assert not kudu_client.table_exists(kudu_tbl_name)
-
   def test_kudu_show_unbounded_range_partition(self, cursor, kudu_client,
                                                unique_database):
     """Check that a single unbounded range partition gets printed correctly."""
@@ -854,36 +842,48 @@ class TestShowCreateTable(KuduTestSuite):
       create_sql_fmt % ("2009-01-01 00:00:00.000000999"),
       show_create_sql_fmt % ("1230768000000001"))
 
-  def test_properties(self, cursor):
-    # If an explicit table name is used for the Kudu table and it differs from what
-    # would be the default Kudu table name, the name should be shown as a table property.
-    kudu_table = self.random_table_name()
-    props = "'kudu.table_name'='%s'" % kudu_table
-    self.assert_show_create_equals(cursor,
-        """
-        CREATE TABLE {{table}} (c INT PRIMARY KEY)
-        PARTITION BY HASH (c) PARTITIONS 3
-        STORED AS KUDU
-        TBLPROPERTIES ({props})""".format(props=props),
-        """
-        CREATE TABLE {db}.{{table}} (
-          c INT NOT NULL ENCODING AUTO_ENCODING COMPRESSION DEFAULT_COMPRESSION,
-          PRIMARY KEY (c)
-        )
-        PARTITION BY HASH (c) PARTITIONS 3
-        STORED AS KUDU
-        TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}', {props})""".format(
-            db=cursor.conn.db_name, kudu_addr=KUDU_MASTER_HOSTS, props=props))
+  def test_external_kudu_table_name_with_show_create(self, cursor, kudu_client,
+      unique_database):
+    """Check that the generated kudu.table_name tblproperty is present with
+       show create table with external Kudu tables.
+    """
+    schema_builder = SchemaBuilder()
+    column_spec = schema_builder.add_column("id", INT64)
+    column_spec.nullable(False)
+    schema_builder.set_primary_keys(["id"])
+    partitioning = Partitioning().set_range_partition_columns(["id"])
+    schema = schema_builder.build()
 
-    # If the name is explicitly given (or not given at all) so that the name is the same
-    # as the default name, the table name is not shown.
-    props = "'kudu.table_name'='impala::{db}.{table}'"
+    kudu_table_name = self.random_table_name()
+    try:
+      kudu_client.create_table(kudu_table_name, schema, partitioning)
+      kudu_table = kudu_client.table(kudu_table_name)
+
+      table_name_prop = "'kudu.table_name'='%s'" % kudu_table.name
+      self.assert_show_create_equals(cursor,
+          """
+          CREATE EXTERNAL TABLE {{table}} STORED AS KUDU
+          TBLPROPERTIES({props})""".format(
+              props=table_name_prop),
+          """
+          CREATE EXTERNAL TABLE {db}.{{table}}
+          STORED AS KUDU
+          TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}', {kudu_table})""".format(
+              db=cursor.conn.db_name, kudu_addr=KUDU_MASTER_HOSTS,
+              kudu_table=table_name_prop))
+    finally:
+      if kudu_client.table_exists(kudu_table_name):
+        kudu_client.delete_table(kudu_table_name)
+
+  def test_managed_kudu_table_name_with_show_create(self, cursor):
+    """Check that the generated kudu.table_name tblproperty is not present with
+       show create table with managed Kudu tables.
+    """
     self.assert_show_create_equals(cursor,
         """
-        CREATE TABLE {{table}} (c INT PRIMARY KEY)
+        CREATE TABLE {table} (c INT PRIMARY KEY)
         PARTITION BY HASH (c) PARTITIONS 3
-        STORED AS KUDU
-        TBLPROPERTIES ({props})""".format(props=props),
+        STORED AS KUDU""",
         """
         CREATE TABLE {db}.{{table}} (
           c INT NOT NULL ENCODING AUTO_ENCODING COMPRESSION DEFAULT_COMPRESSION,
@@ -894,7 +894,6 @@ class TestShowCreateTable(KuduTestSuite):
         TBLPROPERTIES ('kudu.master_addresses'='{kudu_addr}')""".format(
             db=cursor.conn.db_name, kudu_addr=KUDU_MASTER_HOSTS))
 
-
 class TestDropDb(KuduTestSuite):
 
   def test_drop_non_empty_db(self, unique_cursor, kudu_client):
@@ -932,9 +931,9 @@ class TestDropDb(KuduTestSuite):
       managed_table_name = self.random_table_name()
       unique_cursor.execute("""
           CREATE TABLE %s (a INT PRIMARY KEY) PARTITION BY HASH (a) PARTITIONS 3
-          STORED AS KUDU TBLPROPERTIES ('kudu.table_name' = '%s')"""
-          % (managed_table_name, managed_table_name))
-      assert kudu_client.table_exists(managed_table_name)
+          STORED AS KUDU""" % managed_table_name)
+      kudu_table_name = "impala::" + db_name + "." + managed_table_name
+      assert kudu_client.table_exists(kudu_table_name)
 
       # Create a table in HDFS
       hdfs_table_name = self.random_table_name()