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/28 09:38:54 UTC

svn commit: r1426431 - in /hive/trunk/metastore/src: java/org/apache/hadoop/hive/metastore/ test/org/apache/hadoop/hive/metastore/

Author: namit
Date: Fri Dec 28 08:38:54 2012
New Revision: 1426431

URL: http://svn.apache.org/viewvc?rev=1426431&view=rev
Log:
HIVE-3838 Add input table name to MetaStoreEndFunctionContext for logging purposes
(Pamela Vagata via namit)


Modified:
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEndFunctionContext.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1426431&r1=1426430&r2=1426431&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Fri Dec 28 08:38:54 2012
@@ -482,8 +482,12 @@ public class HiveMetaStore extends Thrif
       return startFunction(function, " : db=" + db + " tbl=" + tbl + "partition=" + partName);
     }
 
-    public void endFunction(String function, boolean successful, Exception e) {
-      endFunction(function, new MetaStoreEndFunctionContext(successful, e));
+    private void endFunction(String function, boolean successful, Exception e) {
+      endFunction(function, successful, e, null);
+    }
+    public void endFunction(String function, boolean successful, Exception e,
+                            String inputTableName) {
+      endFunction(function, new MetaStoreEndFunctionContext(successful, e, inputTableName));
     }
 
     public void endFunction(String function, MetaStoreEndFunctionContext context) {
@@ -1112,7 +1116,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("create_table", success, ex);
+        endFunction("create_table", success, ex, tbl.getTableName());
       }
     }
 
@@ -1320,7 +1324,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("drop_table", success, ex);
+        endFunction("drop_table", success, ex, name);
       }
 
     }
@@ -1363,7 +1367,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_table", t != null, ex);
+        endFunction("get_table", t != null, ex, name);
       }
       return t;
     }
@@ -1413,7 +1417,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_multi_table", tables != null, ex);
+        endFunction("get_multi_table", tables != null, ex, join(names, ","));
       }
       return tables;
     }
@@ -1447,14 +1451,14 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_table_names_by_filter", tables != null, ex);
+        endFunction("get_table_names_by_filter", tables != null, ex, join(tables, ","));
       }
       return tables;
     }
 
     public boolean set_table_parameters(String dbname, String name,
         Map<String, String> params) throws NoSuchObjectException, MetaException {
-      endFunction(startTableFunction("set_table_parameters", dbname, name), false, null);
+      endFunction(startTableFunction("set_table_parameters", dbname, name), false, null, name);
       // TODO Auto-generated method stub
       return false;
     }
@@ -1577,7 +1581,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("append_partition", ret != null, ex);
+        endFunction("append_partition", ret != null, ex, tableName);
       }
       return ret;
     }
@@ -1638,7 +1642,8 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("add_partition", ret != null, ex);
+        String tableName = parts.get(0).getTableName();
+        endFunction("add_partition", ret != null, ex, tableName);
       }
       return ret;
     }
@@ -1822,7 +1827,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("add_partition", ret != null, ex);
+        endFunction("add_partition", ret != null, ex, part != null ?  part.getTableName(): null);
       }
       return ret;
     }
@@ -1922,7 +1927,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("drop_partition", ret, ex);
+        endFunction("drop_partition", ret, ex, tbl_name);
       }
       return ret;
 
@@ -1948,7 +1953,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_partition", ret != null, ex);
+        endFunction("get_partition", ret != null, ex, tbl_name);
       }
       return ret;
     }
@@ -1983,7 +1988,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_partition_with_auth", ret != null, ex);
+        endFunction("get_partition_with_auth", ret != null, ex, tbl_name);
       }
       return ret;
     }
@@ -2008,7 +2013,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_partitions", ret != null, ex);
+        endFunction("get_partitions", ret != null, ex, tbl_name);
       }
       return ret;
 
@@ -2043,7 +2048,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_partitions_with_auth", ret != null, ex);
+        endFunction("get_partitions_with_auth", ret != null, ex, tblName);
       }
       return ret;
 
@@ -2067,7 +2072,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_partition_names", ret != null, ex);
+        endFunction("get_partition_names", ret != null, ex, tbl_name);
       }
       return ret;
     }
@@ -2144,7 +2149,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("alter_partition", oldPart != null, ex);
+        endFunction("alter_partition", oldPart != null, ex, tbl_name);
       }
       return;
     }
@@ -2214,7 +2219,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("alter_partition", oldParts != null, ex);
+        endFunction("alter_partition", oldParts != null, ex, tbl_name);
       }
       return;
     }
@@ -2254,7 +2259,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("alter_index", success, ex);
+        endFunction("alter_index", success, ex, base_table_name);
       }
       return;
     }
@@ -2322,7 +2327,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("alter_table", success, ex);
+        endFunction("alter_table", success, ex, name);
       }
     }
 
@@ -2413,7 +2418,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_fields", ret != null, ex);
+        endFunction("get_fields", ret != null, ex, tableName);
       }
 
       return ret;
@@ -2474,7 +2479,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_schema", success, ex);
+        endFunction("get_schema", success, ex, tableName);
       }
     }
 
@@ -2599,7 +2604,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_partition_by_name", ret != null, ex);
+        endFunction("get_partition_by_name", ret != null, ex, tbl_name);
       }
       return ret;
     }
@@ -2632,7 +2637,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("append_partition_by_name", ret != null, ex);
+        endFunction("append_partition_by_name", ret != null, ex, tbl_name);
       }
       return ret;
     }
@@ -2681,7 +2686,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("drop_partition_by_name", ret, ex);
+        endFunction("drop_partition_by_name", ret, ex, tbl_name);
       }
 
       return ret;
@@ -2712,7 +2717,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_partitions_ps", ret != null, ex);
+        endFunction("get_partitions_ps", ret != null, ex, tbl_name);
       }
 
       return ret;
@@ -2747,7 +2752,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_partitions_ps_with_auth", ret != null, ex);
+        endFunction("get_partitions_ps_with_auth", ret != null, ex, tbl_name);
       }
       return ret;
     }
@@ -2775,7 +2780,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_partitions_names_ps", ret != null, ex);
+        endFunction("get_partitions_names_ps", ret != null, ex, tbl_name);
       }
       return ret;
     }
@@ -2825,7 +2830,8 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("add_index", ret != null, ex);
+        String tableName = indexTable != null ? indexTable.getTableName() : null;
+        endFunction("add_index", ret != null, ex, tableName);
       }
       return ret;
     }
@@ -2916,7 +2922,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("drop_index_by_name", ret, ex);
+        endFunction("drop_index_by_name", ret, ex, tblName);
       }
 
       return ret;
@@ -3006,7 +3012,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("drop_index_by_name", ret != null, ex);
+        endFunction("drop_index_by_name", ret != null, ex, tblName);
       }
       return ret;
     }
@@ -3044,7 +3050,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_index_names", ret != null, ex);
+        endFunction("get_index_names", ret != null, ex, tblName);
       }
       return ret;
     }
@@ -3073,7 +3079,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_indexes", ret != null, ex);
+        endFunction("get_indexes", ret != null, ex, tblName);
       }
       return ret;
     }
@@ -3110,7 +3116,7 @@ public class HiveMetaStore extends Thrif
       try {
         statsObj = getMS().getTableColumnStatistics(dbName, tableName, colName);
       } finally {
-        endFunction("get_column_statistics_by_table: ", statsObj != null, null);
+        endFunction("get_column_statistics_by_table: ", statsObj != null, null, tableName);
       }
       return statsObj;
     }
@@ -3132,7 +3138,7 @@ public class HiveMetaStore extends Thrif
         statsObj = getMS().getPartitionColumnStatistics(dbName, tableName, convertedPartName,
                                                             partVals, colName);
       } finally {
-        endFunction("get_column_statistics_by_partition: ", statsObj != null, null);
+        endFunction("get_column_statistics_by_partition: ", statsObj != null, null, tableName);
       }
       return statsObj;
    }
@@ -3171,7 +3177,7 @@ public class HiveMetaStore extends Thrif
         ret = getMS().updateTableColumnStatistics(colStats);
         return ret;
       } finally {
-        endFunction("write_column_statistics: ", ret != false, null);
+        endFunction("write_column_statistics: ", ret != false, null, tableName);
       }
     }
 
@@ -3217,7 +3223,7 @@ public class HiveMetaStore extends Thrif
         ret = getMS().updatePartitionColumnStatistics(colStats, partVals);
         return ret;
       } finally {
-        endFunction("write_partition_column_statistics: ", ret != false, null);
+        endFunction("write_partition_column_statistics: ", ret != false, null, tableName);
       }
     }
 
@@ -3240,7 +3246,7 @@ public class HiveMetaStore extends Thrif
         ret = getMS().deletePartitionColumnStatistics(dbName, tableName,
                                                       convertedPartName, partVals, colName);
       } finally {
-        endFunction("delete_column_statistics_by_partition: ", ret != false, null);
+        endFunction("delete_column_statistics_by_partition: ", ret != false, null, tableName);
       }
       return ret;
     }
@@ -3262,7 +3268,7 @@ public class HiveMetaStore extends Thrif
       try {
         ret = getMS().deleteTableColumnStatistics(dbName, tableName, colName);
       } finally {
-        endFunction("delete_column_statistics_by_table: ", ret != false, null);
+        endFunction("delete_column_statistics_by_table: ", ret != false, null, tableName);
       }
       return ret;
    }
@@ -3291,7 +3297,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_partitions_by_filter", ret != null, ex);
+        endFunction("get_partitions_by_filter", ret != null, ex, tblName);
       }
       return ret;
     }
@@ -3321,7 +3327,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-        endFunction("get_partitions_by_names", ret != null, ex);
+        endFunction("get_partitions_by_names", ret != null, ex, tblName);
       }
       return ret;
     }
@@ -3956,7 +3962,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-                endFunction("markPartitionForEvent", tbl != null, ex);
+                endFunction("markPartitionForEvent", tbl != null, ex, tbl_name);
       }
     }
 
@@ -3992,7 +3998,7 @@ public class HiveMetaStore extends Thrif
           throw me;
         }
       } finally {
-                endFunction("isPartitionMarkedForEvent", ret != null, ex);
+                endFunction("isPartitionMarkedForEvent", ret != null, ex, tbl_name);
       }
 
       return ret;

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEndFunctionContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEndFunctionContext.java?rev=1426431&r1=1426430&r2=1426431&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEndFunctionContext.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEndFunctionContext.java Fri Dec 28 08:38:54 2012
@@ -29,14 +29,16 @@ public class MetaStoreEndFunctionContext
    */
   private final boolean success;
   private final Exception e;
+  private final String inputTableName;
 
-  public MetaStoreEndFunctionContext(boolean success, Exception e) {
+  public MetaStoreEndFunctionContext(boolean success, Exception e, String inputTableName) {
     this.success = success;
     this.e = e;
+    this.inputTableName = inputTableName;
   }
 
   public MetaStoreEndFunctionContext(boolean success) {
-    this(success, null);
+    this(success, null, null);
   }
 
   /**
@@ -50,4 +52,8 @@ public class MetaStoreEndFunctionContext
     return e;
   }
 
+  public String getInputTableName() {
+    return inputTableName;
+  }
+
 }

Modified: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java?rev=1426431&r1=1426430&r2=1426431&view=diff
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java (original)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java Fri Dec 28 08:38:54 2012
@@ -20,7 +20,6 @@ package org.apache.hadoop.hive.metastore
 
 
 
-
 import junit.framework.TestCase;
 
 import org.apache.hadoop.hive.cli.CliSessionState;
@@ -89,12 +88,13 @@ public class TestMetaStoreEndFunctionLis
     Exception e = context.getException();
     assertTrue((e!=null));
     assertTrue((e instanceof NoSuchObjectException));
+    assertEquals(context.getInputTableName(), null);
 
     driver.run("use " + dbName);
     driver.run(String.format("create table %s (a string) partitioned by (b string)", tblName));
-
+    String tableName = "Unknown";
     try {
-      msc.getTable(dbName, "Unknown");
+      msc.getTable(dbName, tableName);
     }
     catch (Exception e1) {
     }
@@ -106,9 +106,10 @@ public class TestMetaStoreEndFunctionLis
     e = context.getException();
     assertTrue((e!=null));
     assertTrue((e instanceof NoSuchObjectException));
+    assertEquals(context.getInputTableName(), tableName);
 
     try {
-      msc.getPartition("tmpdb", "tmptbl", "b=2012");
+      msc.getPartition("tmpdb", tblName, "b=2012");
     }
     catch (Exception e2) {
     }
@@ -120,7 +121,7 @@ public class TestMetaStoreEndFunctionLis
     e = context.getException();
     assertTrue((e!=null));
     assertTrue((e instanceof NoSuchObjectException));
-
+    assertEquals(context.getInputTableName(), tblName);
     try {
       driver.run("drop table Unknown");
     }
@@ -134,6 +135,7 @@ public class TestMetaStoreEndFunctionLis
     e = context.getException();
     assertTrue((e!=null));
     assertTrue((e instanceof NoSuchObjectException));
+    assertEquals(context.getInputTableName(), "Unknown");
 
   }