You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2012/12/04 11:54:37 UTC

svn commit: r1416875 - in /hive/trunk: metastore/src/java/org/apache/hadoop/hive/metastore/ ql/src/test/org/apache/hadoop/hive/ql/hooks/ ql/src/test/queries/clientpositive/ ql/src/test/results/clientpositive/

Author: namit
Date: Tue Dec  4 10:54:36 2012
New Revision: 1416875

URL: http://svn.apache.org/viewvc?rev=1416875&view=rev
Log:
HIVE-3702 Renaming table changes table location scheme/authority
(Kevin Wilfong via namit)


Added:
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyOutputTableLocationSchemeIsFileHook.java
    hive/trunk/ql/src/test/queries/clientpositive/rename_table_location.q
    hive/trunk/ql/src/test/results/clientpositive/rename_table_location.q.out
Modified:
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java?rev=1416875&r1=1416874&r2=1416875&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java Tue Dec  4 10:54:36 2012
@@ -30,7 +30,6 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -38,6 +37,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 
 /**
  * Hive specific implementation of alter
@@ -127,7 +127,11 @@ public class HiveAlterHandler implements
         // that means user is asking metastore to move data to new location
         // corresponding to the new name
         // get new location
-        newTblLoc = wh.getTablePath(msdb.getDatabase(newt.getDbName()), newt.getTableName()).toString();
+        newTblLoc = wh.getTablePath(msdb.getDatabase(newt.getDbName()),
+            newt.getTableName()).toString();
+        Path newTblPath = constructRenamedPath(new Path(newTblLoc),
+            new Path(newt.getSd().getLocation()));
+        newTblLoc = newTblPath.toString();
         newt.getSd().setLocation(newTblLoc);
         oldTblLoc = oldt.getSd().getLocation();
         moveData = true;
@@ -296,7 +300,7 @@ public class HiveAlterHandler implements
         try {
           destPath = new Path(wh.getTablePath(msdb.getDatabase(dbname), name),
             Warehouse.makePartName(tbl.getPartitionKeys(), new_part.getValues()));
-          destPath = constructRenamedPartitionPath(destPath, new_part);
+          destPath = constructRenamedPath(destPath, new Path(new_part.getSd().getLocation()));
         } catch (NoSuchObjectException e) {
           LOG.debug(e);
           throw new InvalidOperationException(
@@ -431,13 +435,13 @@ public class HiveAlterHandler implements
   }
 
   /**
-   * Uses the scheme and authority of the partition's current location, and the path constructed
-   * using the partition's new name to construct a path for the partition's new location.
+   * Uses the scheme and authority of the object's current location and the path constructed
+   * using the object's new name to construct a path for the object's new location.
    */
-  private Path constructRenamedPartitionPath(Path defaultPath, Partition part) {
-    Path oldPath = new Path(part.getSd().getLocation());
-    URI oldUri = oldPath.toUri();
+  private Path constructRenamedPath(Path defaultNewPath, Path currentPath) {
+    URI currentUri = currentPath.toUri();
 
-    return new Path(oldUri.getScheme(), oldUri.getAuthority(), defaultPath.toUri().getPath());
+    return new Path(currentUri.getScheme(), currentUri.getAuthority(),
+        defaultNewPath.toUri().getPath());
   }
 }

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyOutputTableLocationSchemeIsFileHook.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyOutputTableLocationSchemeIsFileHook.java?rev=1416875&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyOutputTableLocationSchemeIsFileHook.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyOutputTableLocationSchemeIsFileHook.java Tue Dec  4 10:54:36 2012
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.hooks;
+
+import org.junit.Assert;
+
+
+public class VerifyOutputTableLocationSchemeIsFileHook implements ExecuteWithHookContext {
+
+  public void run(HookContext hookContext) {
+    for (WriteEntity output : hookContext.getOutputs()) {
+      if (output.getType() == WriteEntity.Type.TABLE) {
+        String scheme = output.getTable().getDataLocation().getScheme();
+        Assert.assertTrue(output.getTable().getTableName() + " has a location which has a " +
+              "scheme other than file: " + scheme, scheme.equals("file"));
+      }
+    }
+  }
+}

Added: hive/trunk/ql/src/test/queries/clientpositive/rename_table_location.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/rename_table_location.q?rev=1416875&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/rename_table_location.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/rename_table_location.q Tue Dec  4 10:54:36 2012
@@ -0,0 +1,17 @@
+-- This test verifies that if the tables location changes, renaming a table will not change
+-- the table location scheme
+
+CREATE TABLE rename_partition_table (key STRING, value STRING) PARTITIONED BY (part STRING)
+STORED AS RCFILE
+LOCATION 'pfile:${system:test.tmp.dir}/rename_partition_table';
+
+INSERT OVERWRITE TABLE rename_partition_table PARTITION (part = '1') SELECT * FROM src;
+
+ALTER TABLE rename_partition_table SET LOCATION 'file:${system:test.tmp.dir}/rename_partition_table';
+
+set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyOutputTableLocationSchemeIsFileHook;
+
+-- If the metastore attempts to change the scheme of the table back to the default pfile, it will get
+-- an exception related to the source and destination file systems not matching
+
+ALTER TABLE rename_partition_table RENAME TO rename_partition_table_renamed;

Added: hive/trunk/ql/src/test/results/clientpositive/rename_table_location.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/rename_table_location.q.out?rev=1416875&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/rename_table_location.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/rename_table_location.q.out Tue Dec  4 10:54:36 2012
@@ -0,0 +1,42 @@
+PREHOOK: query: -- This test verifies that if the tables location changes, renaming a table will not change
+-- the table location scheme
+
+CREATE TABLE rename_partition_table (key STRING, value STRING) PARTITIONED BY (part STRING)
+STORED AS RCFILE
+#### A masked pattern was here ####
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- This test verifies that if the tables location changes, renaming a table will not change
+-- the table location scheme
+
+CREATE TABLE rename_partition_table (key STRING, value STRING) PARTITIONED BY (part STRING)
+STORED AS RCFILE
+#### A masked pattern was here ####
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@rename_partition_table
+PREHOOK: query: INSERT OVERWRITE TABLE rename_partition_table PARTITION (part = '1') SELECT * FROM src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@rename_partition_table@part=1
+POSTHOOK: query: INSERT OVERWRITE TABLE rename_partition_table PARTITION (part = '1') SELECT * FROM src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@rename_partition_table@part=1
+POSTHOOK: Lineage: rename_partition_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: rename_partition_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+#### A masked pattern was here ####
+PREHOOK: type: ALTERTABLE_LOCATION
+PREHOOK: Input: default@rename_partition_table
+PREHOOK: Output: default@rename_partition_table
+#### A masked pattern was here ####
+POSTHOOK: type: ALTERTABLE_LOCATION
+POSTHOOK: Input: default@rename_partition_table
+POSTHOOK: Output: default@rename_partition_table
+POSTHOOK: Lineage: rename_partition_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: rename_partition_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: -- If the metastore attempts to change the scheme of the table back to the default pfile, it will get
+-- an exception related to the source and destination file systems not matching
+
+ALTER TABLE rename_partition_table RENAME TO rename_partition_table_renamed
+PREHOOK: type: ALTERTABLE_RENAME
+PREHOOK: Input: default@rename_partition_table
+PREHOOK: Output: default@rename_partition_table