You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ct...@apache.org on 2016/10/27 13:52:37 UTC

hive git commit: Preserve the location of table created with the location clause in table rename (Chaoyu Tang, reviewed by Aihua Xu)

Repository: hive
Updated Branches:
  refs/heads/master e49170af3 -> 130617443


Preserve the location of table created with the location clause in table rename (Chaoyu Tang, reviewed by Aihua Xu)


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

Branch: refs/heads/master
Commit: 130617443bb05d79c18420c0c4e903a76da3651c
Parents: e49170a
Author: ctang <ct...@cloudera.com>
Authored: Thu Oct 27 09:52:23 2016 -0400
Committer: ctang <ct...@cloudera.com>
Committed: Thu Oct 27 09:52:23 2016 -0400

----------------------------------------------------------------------
 .../hive/hcatalog/cli/TestSemanticAnalysis.java |   9 +-
 .../hadoop/hive/metastore/HiveAlterHandler.java | 128 +++++++++--------
 .../clientpositive/encryption_move_tbl.q        |  43 +++++-
 .../encrypted/encryption_move_tbl.q.out         | 140 ++++++++++++++++++-
 4 files changed, 248 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/13061744/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java
index ce4ef75..e41b1f1 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java
@@ -248,11 +248,16 @@ public class TestSemanticAnalysis extends HCatBaseTest {
     hcatDriver.run("drop table newname");
     hcatDriver.run("create table oldname (a int)");
     Table tbl = client.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "oldname");
-    assertTrue(tbl.getSd().getLocation().contains("oldname"));
+    assertTrue("The old table location is: " + tbl.getSd().getLocation(), tbl.getSd().getLocation().contains("oldname"));
 
     hcatDriver.run("alter table oldname rename to newNAME");
     tbl = client.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "newname");
-    assertTrue(tbl.getSd().getLocation().contains("newname"));
+    // since the oldname table is not under its database (See HIVE-15059), the renamed oldname table will keep
+    // its location after HIVE-14909. I changed to check the existence of the newname table and its name instead
+    // of verifying its location
+    // assertTrue(tbl.getSd().getLocation().contains("newname"));
+    assertTrue(tbl != null);
+    assertTrue(tbl.getTableName().equalsIgnoreCase("newname"));
 
     hcatDriver.run("drop table newname");
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/13061744/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
index 7b84595..ede9b52 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
@@ -163,72 +163,84 @@ public class HiveAlterHandler implements AlterHandler {
         }
       }
 
-      // if this alter is a rename, the table is not a virtual view, the user
-      // didn't change the default location (or new location is empty), and
-      // table is not an external table, that means user is asking metastore to
-      // move data to the new location corresponding to the new name
+      // rename needs change the data location and move the data to the new location corresponding
+      // to the new name if:
+      // 1) the table is not a virtual view, and
+      // 2) the table is not an external table, and
+      // 3) the user didn't change the default location (or new location is empty), and
+      // 4) the table was not initially created with a specified location
       if (rename
           && !oldt.getTableType().equals(TableType.VIRTUAL_VIEW.toString())
           && (oldt.getSd().getLocation().compareTo(newt.getSd().getLocation()) == 0
             || StringUtils.isEmpty(newt.getSd().getLocation()))
           && !MetaStoreUtils.isExternalTable(oldt)) {
-
+        Database olddb = msdb.getDatabase(dbname);
+        // if a table was created in a user specified location using the DDL like
+        // create table tbl ... location ...., it should be treated like an external table
+        // in the table rename, its data location should not be changed. We can check
+        // if the table directory was created directly under its database directory to tell
+        // if it is such a table
         srcPath = new Path(oldt.getSd().getLocation());
-        srcFs = wh.getFs(srcPath);
-
-        // that means user is asking metastore to move data to new location
-        // corresponding to the new name
-        // get new location
-        Database db = msdb.getDatabase(newt.getDbName());
-        Path databasePath = constructRenamedPath(wh.getDatabasePath(db), srcPath);
-        destPath = new Path(databasePath, newt.getTableName().toLowerCase());
-        destFs = wh.getFs(destPath);
-
-        newt.getSd().setLocation(destPath.toString());
-        moveData = true;
-
-        // check that destination does not exist otherwise we will be
-        // overwriting data
-        // check that src and dest are on the same file system
-        if (!FileUtils.equalsFileSystem(srcFs, destFs)) {
-          throw new InvalidOperationException("table new location " + destPath
-              + " is on a different file system than the old location "
-              + srcPath + ". This operation is not supported");
-        }
-        try {
-          srcFs.exists(srcPath); // check that src exists and also checks
-                                 // permissions necessary
-          if (destFs.exists(destPath)) {
-            throw new InvalidOperationException("New location for this table "
-                + newt.getDbName() + "." + newt.getTableName()
-                + " already exists : " + destPath);
+        String oldtRelativePath = (new Path(olddb.getLocationUri()).toUri())
+            .relativize(srcPath.toUri()).toString();
+        boolean tableInSpecifiedLoc = !oldtRelativePath.equalsIgnoreCase(name)
+            && !oldtRelativePath.equalsIgnoreCase(name + Path.SEPARATOR);
+
+        if (!tableInSpecifiedLoc) {
+          srcFs = wh.getFs(srcPath);
+
+          // get new location
+          Database db = msdb.getDatabase(newt.getDbName());
+          Path databasePath = constructRenamedPath(wh.getDatabasePath(db), srcPath);
+          destPath = new Path(databasePath, newt.getTableName().toLowerCase());
+          destFs = wh.getFs(destPath);
+
+          newt.getSd().setLocation(destPath.toString());
+          moveData = true;
+
+          // check that destination does not exist otherwise we will be
+          // overwriting data
+          // check that src and dest are on the same file system
+          if (!FileUtils.equalsFileSystem(srcFs, destFs)) {
+            throw new InvalidOperationException("table new location " + destPath
+                + " is on a different file system than the old location "
+                + srcPath + ". This operation is not supported");
           }
-        } catch (IOException e) {
-          throw new InvalidOperationException("Unable to access new location "
-              + destPath + " for table " + newt.getDbName() + "."
-              + newt.getTableName());
-        }
-        String oldTblLocPath = srcPath.toUri().getPath();
-        String newTblLocPath = destPath.toUri().getPath();
-
-        // also the location field in partition
-        List<Partition> parts = msdb.getPartitions(dbname, name, -1);
-        for (Partition part : parts) {
-          String oldPartLoc = part.getSd().getLocation();
-          if (oldPartLoc.contains(oldTblLocPath)) {
-            URI oldUri = new Path(oldPartLoc).toUri();
-            String newPath = oldUri.getPath().replace(oldTblLocPath, newTblLocPath);
-            Path newPartLocPath = new Path(oldUri.getScheme(), oldUri.getAuthority(), newPath);
-            altps.add(ObjectPair.create(part, part.getSd().getLocation()));
-            part.getSd().setLocation(newPartLocPath.toString());
-            String oldPartName = Warehouse.makePartName(oldt.getPartitionKeys(), part.getValues());
-            try {
-              //existing partition column stats is no longer valid, remove them
-              msdb.deletePartitionColumnStatistics(dbname, name, oldPartName, part.getValues(), null);
-            } catch (InvalidInputException iie) {
-              throw new InvalidOperationException("Unable to update partition stats in table rename." + iie);
+          try {
+            srcFs.exists(srcPath); // check that src exists and also checks
+                                   // permissions necessary
+            if (destFs.exists(destPath)) {
+              throw new InvalidOperationException("New location for this table "
+                  + newt.getDbName() + "." + newt.getTableName()
+                  + " already exists : " + destPath);
+            }
+          } catch (IOException e) {
+            throw new InvalidOperationException("Unable to access new location "
+                + destPath + " for table " + newt.getDbName() + "."
+                + newt.getTableName());
+          }
+          String oldTblLocPath = srcPath.toUri().getPath();
+          String newTblLocPath = destPath.toUri().getPath();
+
+          // also the location field in partition
+          List<Partition> parts = msdb.getPartitions(dbname, name, -1);
+          for (Partition part : parts) {
+            String oldPartLoc = part.getSd().getLocation();
+            if (oldPartLoc.contains(oldTblLocPath)) {
+              URI oldUri = new Path(oldPartLoc).toUri();
+              String newPath = oldUri.getPath().replace(oldTblLocPath, newTblLocPath);
+              Path newPartLocPath = new Path(oldUri.getScheme(), oldUri.getAuthority(), newPath);
+              altps.add(ObjectPair.create(part, part.getSd().getLocation()));
+              part.getSd().setLocation(newPartLocPath.toString());
+              String oldPartName = Warehouse.makePartName(oldt.getPartitionKeys(), part.getValues());
+              try {
+                //existing partition column stats is no longer valid, remove them
+                msdb.deletePartitionColumnStatistics(dbname, name, oldPartName, part.getValues(), null);
+              } catch (InvalidInputException iie) {
+                throw new InvalidOperationException("Unable to update partition stats in table rename." + iie);
+              }
+              msdb.alterPartition(dbname, name, part.getValues(), part);
             }
-            msdb.alterPartition(dbname, name, part.getValues(), part);
           }
         }
       } else if (MetaStoreUtils.requireCalStats(hiveConf, null, null, newt, environmentContext) &&

http://git-wip-us.apache.org/repos/asf/hive/blob/13061744/ql/src/test/queries/clientpositive/encryption_move_tbl.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/encryption_move_tbl.q b/ql/src/test/queries/clientpositive/encryption_move_tbl.q
index a25d955..7a5de7b 100644
--- a/ql/src/test/queries/clientpositive/encryption_move_tbl.q
+++ b/ql/src/test/queries/clientpositive/encryption_move_tbl.q
@@ -6,26 +6,57 @@ set hive.cli.errors.ignore=true;
 
 DROP TABLE IF EXISTS encrypted_table PURGE;
 DROP DATABASE IF EXISTS encrypted_db;
-CREATE TABLE encrypted_table (key INT, value STRING) LOCATION '${hiveconf:hive.metastore.warehouse.dir}/default/encrypted_table';
+
+-- create table default.encrypted_table in its default warehouse location ${hiveconf:hive.metastore.warehouse.dir}/encrypted_table
+CREATE TABLE encrypted_table (key INT, value STRING) LOCATION '${hiveconf:hive.metastore.warehouse.dir}/encrypted_table';
 CRYPTO CREATE_KEY --keyName key_128 --bitLength 128;
-CRYPTO CREATE_ZONE --keyName key_128 --path ${hiveconf:hive.metastore.warehouse.dir}/default/encrypted_table;
+CRYPTO CREATE_ZONE --keyName key_128 --path ${hiveconf:hive.metastore.warehouse.dir}/encrypted_table;
 
-CREATE DATABASE encrypted_db LOCATION '${hiveconf:hive.metastore.warehouse.dir}/encrypted_db';
+-- create database encrypted_db in its default warehouse location {hiveconf:hive.metastore.warehouse.dir}/encrypted_db.db
+CREATE DATABASE encrypted_db LOCATION '${hiveconf:hive.metastore.warehouse.dir}/encrypted_db.db';
 CRYPTO CREATE_KEY --keyName key_128_2 --bitLength 128;
-CRYPTO CREATE_ZONE --keyName key_128_2 --path ${hiveconf:hive.metastore.warehouse.dir}/encrypted_db;
+CRYPTO CREATE_ZONE --keyName key_128_2 --path ${hiveconf:hive.metastore.warehouse.dir}/encrypted_db.db;
 
 INSERT OVERWRITE TABLE encrypted_table SELECT * FROM src;
 SHOW TABLES;
--- should fail
+-- should fail, since they are in different encryption zones
 ALTER TABLE default.encrypted_table RENAME TO encrypted_db.encrypted_table_2;
 SHOW TABLES;
+
 -- should succeed in Hadoop 2.7 but fail in 2.6  (HDFS-7530)
 ALTER TABLE default.encrypted_table RENAME TO default.plain_table;
 SHOW TABLES;
 
+-- create table encrypted_table_outloc under default database but in a specified location other than the default db location in the warehouse
+-- rename should succeed since it does not need to move data (HIVE-14909), otherwise, it would fail.
+CREATE TABLE encrypted_table_outloc (key INT, value STRING) LOCATION '${hiveconf:hive.metastore.warehouse.dir}/../specified_table_location';
+CRYPTO CREATE_KEY --keyName key_128_3 --bitLength 128;
+CRYPTO CREATE_ZONE --keyName key_128_3 --path ${hiveconf:hive.metastore.warehouse.dir}/../specified_table_location;
+ALTER TABLE encrypted_table_outloc RENAME TO renamed_encrypted_table_outloc;
+SHOW TABLES;
+
+-- create database encrypted_db_outloc in a specified location other than its default in warehouse
+CREATE DATABASE encrypted_db_outloc LOCATION '${hiveconf:hive.metastore.warehouse.dir}/../specified_db_location';
+CRYPTO CREATE_KEY --keyName key_128_4 --bitLength 128;
+CRYPTO CREATE_ZONE --keyName key_128_4 --path ${hiveconf:hive.metastore.warehouse.dir}/../specified_db_location;
+
+USE encrypted_db_outloc;
+CREATE TABLE encrypted_table (key INT, value STRING);
+INSERT OVERWRITE TABLE encrypted_table SELECT * FROM default.src;
+ALTER TABLE encrypted_table RENAME TO renamed_encrypted_table;
+-- should succeed since data moves within specified_db_location
+SHOW TABLES;
+-- should fail, since they are in different encryption zones
+ALTER TABLE encrypted_db_outloc.renamed_encrypted_table RENAME TO default.plain_table_2;
+SHOW TABLES;
 
-DROP TABLE encrypted_table PURGE;
+DROP TABLE default.encrypted_table PURGE;
 DROP TABLE default.plain_table PURGE;
+DROP TABLE default.renamed_encrypted_table_outloc PURGE;
 DROP DATABASE encrypted_db;
+DROP TABLE encrypted_db_outloc.renamed_encrypted_table PURGE;
+DROP DATABASE encrypted_db_outloc;
 CRYPTO DELETE_KEY --keyName key_128;
 CRYPTO DELETE_KEY --keyName key_128_2;
+CRYPTO DELETE_KEY --keyName key_128_3;
+CRYPTO DELETE_KEY --keyName key_128_4;

http://git-wip-us.apache.org/repos/asf/hive/blob/13061744/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
index 910ce25..26396b2 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_move_tbl.q.out
@@ -17,16 +17,19 @@ POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@encrypted_table
 Encryption key created: 'key_128'
-Encryption zone created: '/build/ql/test/data/warehouse/default/encrypted_table' using key: 'key_128'
+Encryption zone created: '/build/ql/test/data/warehouse/encrypted_table' using key: 'key_128'
+PREHOOK: query: -- create database encrypted_db in its default warehouse location {hiveconf:hive.metastore.warehouse.dir}/encrypted_db.db
 #### A masked pattern was here ####
 PREHOOK: type: CREATEDATABASE
 PREHOOK: Output: database:encrypted_db
 #### A masked pattern was here ####
+POSTHOOK: query: -- create database encrypted_db in its default warehouse location {hiveconf:hive.metastore.warehouse.dir}/encrypted_db.db
+#### A masked pattern was here ####
 POSTHOOK: type: CREATEDATABASE
 POSTHOOK: Output: database:encrypted_db
 #### A masked pattern was here ####
 Encryption key created: 'key_128_2'
-Encryption zone created: '/build/ql/test/data/warehouse/encrypted_db' using key: 'key_128_2'
+Encryption zone created: '/build/ql/test/data/warehouse/encrypted_db.db' using key: 'key_128_2'
 PREHOOK: query: INSERT OVERWRITE TABLE encrypted_table SELECT * FROM src
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -45,12 +48,12 @@ POSTHOOK: type: SHOWTABLES
 POSTHOOK: Input: database:default
 encrypted_table
 src
-PREHOOK: query: -- should fail
+PREHOOK: query: -- should fail, since they are in different encryption zones
 ALTER TABLE default.encrypted_table RENAME TO encrypted_db.encrypted_table_2
 PREHOOK: type: ALTERTABLE_RENAME
 PREHOOK: Input: default@encrypted_table
 PREHOOK: Output: default@encrypted_table
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. Alter Table operation for default.encrypted_table failed to move data due to: '/build/ql/test/data/warehouse/default/encrypted_table can't be moved from encryption zone /build/ql/test/data/warehouse/default/encrypted_table to encryption zone /build/ql/test/data/warehouse/encrypted_db.' See hive log file for details.
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. Alter Table operation for default.encrypted_table failed to move data due to: '/build/ql/test/data/warehouse/encrypted_table can't be moved from encryption zone /build/ql/test/data/warehouse/encrypted_table to encryption zone /build/ql/test/data/warehouse/encrypted_db.db.' See hive log file for details.
 PREHOOK: query: SHOW TABLES
 PREHOOK: type: SHOWTABLES
 PREHOOK: Input: database:default
@@ -78,9 +81,110 @@ POSTHOOK: type: SHOWTABLES
 POSTHOOK: Input: database:default
 plain_table
 src
-PREHOOK: query: DROP TABLE encrypted_table PURGE
+PREHOOK: query: -- create table encrypted_table_outloc under default database but in a specified location other than the default db location in the warehouse
+-- rename should succeed since it does not need to move data (HIVE-14909), otherwise, it would fail.
+#### A masked pattern was here ####
+PREHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+PREHOOK: Output: database:default
+PREHOOK: Output: default@encrypted_table_outloc
+POSTHOOK: query: -- create table encrypted_table_outloc under default database but in a specified location other than the default db location in the warehouse
+-- rename should succeed since it does not need to move data (HIVE-14909), otherwise, it would fail.
+#### A masked pattern was here ####
+POSTHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@encrypted_table_outloc
+Encryption key created: 'key_128_3'
+Encryption zone created: '/build/ql/test/data/specified_table_location' using key: 'key_128_3'
+PREHOOK: query: ALTER TABLE encrypted_table_outloc RENAME TO renamed_encrypted_table_outloc
+PREHOOK: type: ALTERTABLE_RENAME
+PREHOOK: Input: default@encrypted_table_outloc
+PREHOOK: Output: default@encrypted_table_outloc
+POSTHOOK: query: ALTER TABLE encrypted_table_outloc RENAME TO renamed_encrypted_table_outloc
+POSTHOOK: type: ALTERTABLE_RENAME
+POSTHOOK: Input: default@encrypted_table_outloc
+POSTHOOK: Output: default@encrypted_table_outloc
+POSTHOOK: Output: default@renamed_encrypted_table_outloc
+PREHOOK: query: SHOW TABLES
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:default
+POSTHOOK: query: SHOW TABLES
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:default
+plain_table
+renamed_encrypted_table_outloc
+src
+PREHOOK: query: -- create database encrypted_db_outloc in a specified location other than its default in warehouse
+#### A masked pattern was here ####
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:encrypted_db_outloc
+#### A masked pattern was here ####
+POSTHOOK: query: -- create database encrypted_db_outloc in a specified location other than its default in warehouse
+#### A masked pattern was here ####
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:encrypted_db_outloc
+#### A masked pattern was here ####
+Encryption key created: 'key_128_4'
+Encryption zone created: '/build/ql/test/data/specified_db_location' using key: 'key_128_4'
+PREHOOK: query: USE encrypted_db_outloc
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:encrypted_db_outloc
+POSTHOOK: query: USE encrypted_db_outloc
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:encrypted_db_outloc
+PREHOOK: query: CREATE TABLE encrypted_table (key INT, value STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:encrypted_db_outloc
+PREHOOK: Output: encrypted_db_outloc@encrypted_table
+POSTHOOK: query: CREATE TABLE encrypted_table (key INT, value STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:encrypted_db_outloc
+POSTHOOK: Output: encrypted_db_outloc@encrypted_table
+PREHOOK: query: INSERT OVERWRITE TABLE encrypted_table SELECT * FROM default.src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: encrypted_db_outloc@encrypted_table
+POSTHOOK: query: INSERT OVERWRITE TABLE encrypted_table SELECT * FROM default.src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: encrypted_db_outloc@encrypted_table
+POSTHOOK: Lineage: encrypted_table.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: encrypted_table.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: ALTER TABLE encrypted_table RENAME TO renamed_encrypted_table
+PREHOOK: type: ALTERTABLE_RENAME
+PREHOOK: Input: encrypted_db_outloc@encrypted_table
+PREHOOK: Output: encrypted_db_outloc@encrypted_table
+POSTHOOK: query: ALTER TABLE encrypted_table RENAME TO renamed_encrypted_table
+POSTHOOK: type: ALTERTABLE_RENAME
+POSTHOOK: Input: encrypted_db_outloc@encrypted_table
+POSTHOOK: Output: encrypted_db_outloc@encrypted_table
+POSTHOOK: Output: encrypted_db_outloc@renamed_encrypted_table
+PREHOOK: query: -- should succeed since data moves within specified_db_location
+SHOW TABLES
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:encrypted_db_outloc
+POSTHOOK: query: -- should succeed since data moves within specified_db_location
+SHOW TABLES
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:encrypted_db_outloc
+renamed_encrypted_table
+PREHOOK: query: -- should fail, since they are in different encryption zones
+ALTER TABLE encrypted_db_outloc.renamed_encrypted_table RENAME TO default.plain_table_2
+PREHOOK: type: ALTERTABLE_RENAME
+PREHOOK: Input: encrypted_db_outloc@renamed_encrypted_table
+PREHOOK: Output: encrypted_db_outloc@renamed_encrypted_table
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. Alter Table operation for encrypted_db_outloc.renamed_encrypted_table failed to move data due to: '/build/ql/test/data/specified_db_location/renamed_encrypted_table can't be moved from an encryption zone.' See hive log file for details.
+PREHOOK: query: SHOW TABLES
+PREHOOK: type: SHOWTABLES
+PREHOOK: Input: database:encrypted_db_outloc
+POSTHOOK: query: SHOW TABLES
+POSTHOOK: type: SHOWTABLES
+POSTHOOK: Input: database:encrypted_db_outloc
+renamed_encrypted_table
+PREHOOK: query: DROP TABLE default.encrypted_table PURGE
 PREHOOK: type: DROPTABLE
-POSTHOOK: query: DROP TABLE encrypted_table PURGE
+POSTHOOK: query: DROP TABLE default.encrypted_table PURGE
 POSTHOOK: type: DROPTABLE
 PREHOOK: query: DROP TABLE default.plain_table PURGE
 PREHOOK: type: DROPTABLE
@@ -90,6 +194,14 @@ POSTHOOK: query: DROP TABLE default.plain_table PURGE
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@plain_table
 POSTHOOK: Output: default@plain_table
+PREHOOK: query: DROP TABLE default.renamed_encrypted_table_outloc PURGE
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@renamed_encrypted_table_outloc
+PREHOOK: Output: default@renamed_encrypted_table_outloc
+POSTHOOK: query: DROP TABLE default.renamed_encrypted_table_outloc PURGE
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@renamed_encrypted_table_outloc
+POSTHOOK: Output: default@renamed_encrypted_table_outloc
 PREHOOK: query: DROP DATABASE encrypted_db
 PREHOOK: type: DROPDATABASE
 PREHOOK: Input: database:encrypted_db
@@ -98,3 +210,19 @@ POSTHOOK: query: DROP DATABASE encrypted_db
 POSTHOOK: type: DROPDATABASE
 POSTHOOK: Input: database:encrypted_db
 POSTHOOK: Output: database:encrypted_db
+PREHOOK: query: DROP TABLE encrypted_db_outloc.renamed_encrypted_table PURGE
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: encrypted_db_outloc@renamed_encrypted_table
+PREHOOK: Output: encrypted_db_outloc@renamed_encrypted_table
+POSTHOOK: query: DROP TABLE encrypted_db_outloc.renamed_encrypted_table PURGE
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: encrypted_db_outloc@renamed_encrypted_table
+POSTHOOK: Output: encrypted_db_outloc@renamed_encrypted_table
+PREHOOK: query: DROP DATABASE encrypted_db_outloc
+PREHOOK: type: DROPDATABASE
+PREHOOK: Input: database:encrypted_db_outloc
+PREHOOK: Output: database:encrypted_db_outloc
+POSTHOOK: query: DROP DATABASE encrypted_db_outloc
+POSTHOOK: type: DROPDATABASE
+POSTHOOK: Input: database:encrypted_db_outloc
+POSTHOOK: Output: database:encrypted_db_outloc