You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2013/09/18 04:32:52 UTC

svn commit: r1524278 - in /hive/trunk/ql/src/java/org/apache/hadoop/hive/ql: Driver.java exec/Utilities.java io/CombineHiveInputFormat.java io/HiveInputFormat.java log/PerfLogger.java optimizer/ppr/PartitionPruner.java

Author: hashutosh
Date: Wed Sep 18 02:32:52 2013
New Revision: 1524278

URL: http://svn.apache.org/r1524278
Log:
HIVE-5288 : Perflogger should log under single class (Sergey Shelukhin via Ashutosh Chauhan)

Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java?rev=1524278&r1=1524277&r2=1524278&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java Wed Sep 18 02:32:52 2013
@@ -112,7 +112,8 @@ import org.apache.hadoop.util.Reflection
 
 public class Driver implements CommandProcessor {
 
-  static final private Log LOG = LogFactory.getLog(Driver.class.getName());
+  static final private String CLASS_NAME = Driver.class.getName();
+  static final private Log LOG = LogFactory.getLog(CLASS_NAME);
   static final private LogHelper console = new LogHelper(LOG);
 
   private static final Object compileMonitor = new Object();
@@ -397,7 +398,7 @@ public class Driver implements CommandPr
    */
   public int compile(String command, boolean resetTaskIds) {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
-    perfLogger.PerfLogBegin(LOG, PerfLogger.COMPILE);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.COMPILE);
 
     //holder for parent command type/string when executing reentrant queries
     QueryState queryState = new QueryState();
@@ -419,13 +420,13 @@ public class Driver implements CommandPr
       ctx.setCmd(command);
       ctx.setHDFSCleanup(true);
 
-      perfLogger.PerfLogBegin(LOG, PerfLogger.PARSE);
+      perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PARSE);
       ParseDriver pd = new ParseDriver();
       ASTNode tree = pd.parse(command, ctx);
       tree = ParseUtils.findRootNonNullToken(tree);
-      perfLogger.PerfLogEnd(LOG, PerfLogger.PARSE);
+      perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PARSE);
 
-      perfLogger.PerfLogBegin(LOG, PerfLogger.ANALYZE);
+      perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ANALYZE);
       BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, tree);
       List<HiveSemanticAnalyzerHook> saHooks =
           getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK,
@@ -451,7 +452,7 @@ public class Driver implements CommandPr
 
       // validate the plan
       sem.validate();
-      perfLogger.PerfLogEnd(LOG, PerfLogger.ANALYZE);
+      perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.ANALYZE);
 
       plan = new QueryPlan(command, sem, perfLogger.getStartTime(PerfLogger.DRIVER_RUN));
 
@@ -489,7 +490,7 @@ public class Driver implements CommandPr
       if (HiveConf.getBoolVar(conf,
           HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) {
         try {
-          perfLogger.PerfLogBegin(LOG, PerfLogger.DO_AUTHORIZATION);
+          perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.DO_AUTHORIZATION);
           doAuthorization(sem);
         } catch (AuthorizationException authExp) {
           errorMessage = "Authorization failed:" + authExp.getMessage()
@@ -497,7 +498,7 @@ public class Driver implements CommandPr
           console.printError(errorMessage);
           return 403;
         } finally {
-          perfLogger.PerfLogEnd(LOG, PerfLogger.DO_AUTHORIZATION);
+          perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.DO_AUTHORIZATION);
         }
       }
 
@@ -524,7 +525,7 @@ public class Driver implements CommandPr
           + org.apache.hadoop.util.StringUtils.stringifyException(e));
       return error.getErrorCode();
     } finally {
-      perfLogger.PerfLogEnd(LOG, PerfLogger.COMPILE);
+      perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.COMPILE);
       restoreSession(queryState);
     }
   }
@@ -780,7 +781,7 @@ public class Driver implements CommandPr
    **/
   public int acquireReadWriteLocks() {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
-    perfLogger.PerfLogBegin(LOG, PerfLogger.ACQUIRE_READ_WRITE_LOCKS);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ACQUIRE_READ_WRITE_LOCKS);
 
     try {
       boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
@@ -863,7 +864,7 @@ public class Driver implements CommandPr
           + org.apache.hadoop.util.StringUtils.stringifyException(e));
       return (10);
     } finally {
-      perfLogger.PerfLogEnd(LOG, PerfLogger.ACQUIRE_READ_WRITE_LOCKS);
+      perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.ACQUIRE_READ_WRITE_LOCKS);
     }
   }
 
@@ -874,14 +875,14 @@ public class Driver implements CommandPr
    **/
   private void releaseLocks(List<HiveLock> hiveLocks) {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
-    perfLogger.PerfLogBegin(LOG, PerfLogger.RELEASE_LOCKS);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.RELEASE_LOCKS);
 
     if (hiveLocks != null) {
       ctx.getHiveLockMgr().releaseLocks(hiveLocks);
     }
     ctx.setHiveLocks(null);
 
-    perfLogger.PerfLogEnd(LOG, PerfLogger.RELEASE_LOCKS);
+    perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.RELEASE_LOCKS);
   }
 
   public CommandProcessorResponse run(String command) throws CommandNeedRetryException {
@@ -969,8 +970,8 @@ public class Driver implements CommandPr
 
     // Reset the perf logger
     PerfLogger perfLogger = PerfLogger.getPerfLogger(true);
-    perfLogger.PerfLogBegin(LOG, PerfLogger.DRIVER_RUN);
-    perfLogger.PerfLogBegin(LOG, PerfLogger.TIME_TO_SUBMIT);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.DRIVER_RUN);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TIME_TO_SUBMIT);
 
     int ret;
     synchronized (compileMonitor) {
@@ -1027,7 +1028,7 @@ public class Driver implements CommandPr
     //if needRequireLock is false, the release here will do nothing because there is no lock
     releaseLocks(ctx.getHiveLocks());
 
-    perfLogger.PerfLogEnd(LOG, PerfLogger.DRIVER_RUN);
+    perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.DRIVER_RUN);
     perfLogger.close(LOG, plan);
 
     // Take all the driver run hooks and post-execute them.
@@ -1097,7 +1098,7 @@ public class Driver implements CommandPr
 
   public int execute() throws CommandNeedRetryException {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
-    perfLogger.PerfLogBegin(LOG, PerfLogger.DRIVER_EXECUTE);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.DRIVER_EXECUTE);
 
     boolean noName = StringUtils.isEmpty(conf.getVar(HiveConf.ConfVars.HADOOPJOBNAME));
     int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH);
@@ -1130,18 +1131,18 @@ public class Driver implements CommandPr
 
       for (Hook peh : getHooks(HiveConf.ConfVars.PREEXECHOOKS)) {
         if (peh instanceof ExecuteWithHookContext) {
-          perfLogger.PerfLogBegin(LOG, PerfLogger.PRE_HOOK + peh.getClass().getName());
+          perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
 
           ((ExecuteWithHookContext) peh).run(hookContext);
 
-          perfLogger.PerfLogEnd(LOG, PerfLogger.PRE_HOOK + peh.getClass().getName());
+          perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
         } else if (peh instanceof PreExecute) {
-          perfLogger.PerfLogBegin(LOG, PerfLogger.PRE_HOOK + peh.getClass().getName());
+          perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
 
           ((PreExecute) peh).run(SessionState.get(), plan.getInputs(), plan.getOutputs(),
               ShimLoader.getHadoopShims().getUGIForConf(conf));
 
-          perfLogger.PerfLogEnd(LOG, PerfLogger.PRE_HOOK + peh.getClass().getName());
+          perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
         }
       }
 
@@ -1181,14 +1182,14 @@ public class Driver implements CommandPr
         driverCxt.addToRunnable(tsk);
       }
 
-      perfLogger.PerfLogEnd(LOG, PerfLogger.TIME_TO_SUBMIT);
-      perfLogger.PerfLogBegin(LOG, PerfLogger.RUN_TASKS);
+      perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TIME_TO_SUBMIT);
+      perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.RUN_TASKS);
       // Loop while you either have tasks running, or tasks queued up
       while (running.size() != 0 || runnable.peek() != null) {
         // Launch upto maxthreads tasks
         while (runnable.peek() != null && running.size() < maxthreads) {
           Task<? extends Serializable> tsk = runnable.remove();
-          perfLogger.PerfLogBegin(LOG, PerfLogger.TASK + tsk.getName() + "." + tsk.getId());
+          perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TASK + tsk.getName() + "." + tsk.getId());
           launchTask(tsk, queryId, noName, running, jobname, jobs, driverCxt);
         }
 
@@ -1196,7 +1197,7 @@ public class Driver implements CommandPr
         TaskResult tskRes = pollTasks(running.keySet());
         TaskRunner tskRun = running.remove(tskRes);
         Task<? extends Serializable> tsk = tskRun.getTask();
-        perfLogger.PerfLogEnd(LOG, PerfLogger.TASK + tsk.getName() + "." + tsk.getId());
+        perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TASK + tsk.getName() + "." + tsk.getId());
         hookContext.addCompleteTask(tskRun);
 
         int exitVal = tskRes.getExitVal();
@@ -1227,11 +1228,11 @@ public class Driver implements CommandPr
             hookContext.setHookType(HookContext.HookType.ON_FAILURE_HOOK);
             // Get all the failure execution hooks and execute them.
             for (Hook ofh : getHooks(HiveConf.ConfVars.ONFAILUREHOOKS)) {
-              perfLogger.PerfLogBegin(LOG, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
+              perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
 
               ((ExecuteWithHookContext) ofh).run(hookContext);
 
-              perfLogger.PerfLogEnd(LOG, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
+              perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
             }
             setErrorMsgAndDetail(exitVal, tskRes.getTaskError(), tsk);
             SQLState = "08S01";
@@ -1260,7 +1261,7 @@ public class Driver implements CommandPr
           }
         }
       }
-      perfLogger.PerfLogEnd(LOG, PerfLogger.RUN_TASKS);
+      perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.RUN_TASKS);
 
       // in case we decided to run everything in local mode, restore the
       // the jobtracker setting to its initial value
@@ -1284,19 +1285,19 @@ public class Driver implements CommandPr
       // Get all the post execution hooks and execute them.
       for (Hook peh : getHooks(HiveConf.ConfVars.POSTEXECHOOKS)) {
         if (peh instanceof ExecuteWithHookContext) {
-          perfLogger.PerfLogBegin(LOG, PerfLogger.POST_HOOK + peh.getClass().getName());
+          perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.POST_HOOK + peh.getClass().getName());
 
           ((ExecuteWithHookContext) peh).run(hookContext);
 
-          perfLogger.PerfLogEnd(LOG, PerfLogger.POST_HOOK + peh.getClass().getName());
+          perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.POST_HOOK + peh.getClass().getName());
         } else if (peh instanceof PostExecute) {
-          perfLogger.PerfLogBegin(LOG, PerfLogger.POST_HOOK + peh.getClass().getName());
+          perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.POST_HOOK + peh.getClass().getName());
 
           ((PostExecute) peh).run(SessionState.get(), plan.getInputs(), plan.getOutputs(),
               (SessionState.get() != null ? SessionState.get().getLineageState().getLineageInfo()
                   : null), ShimLoader.getHadoopShims().getUGIForConf(conf));
 
-          perfLogger.PerfLogEnd(LOG, PerfLogger.POST_HOOK + peh.getClass().getName());
+          perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.POST_HOOK + peh.getClass().getName());
         }
       }
 
@@ -1328,7 +1329,7 @@ public class Driver implements CommandPr
       if (noName) {
         conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, "");
       }
-      perfLogger.PerfLogEnd(LOG, PerfLogger.DRIVER_EXECUTE);
+      perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.DRIVER_EXECUTE);
 
       if (SessionState.get().getLastMapRedStatsList() != null
           && SessionState.get().getLastMapRedStatsList().size() > 0) {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java?rev=1524278&r1=1524277&r2=1524278&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java Wed Sep 18 02:32:52 2013
@@ -99,6 +99,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
@@ -218,7 +219,8 @@ public final class Utilities {
 
   private static Map<Path, BaseWork> gWorkMap = Collections
       .synchronizedMap(new HashMap<Path, BaseWork>());
-  private static final Log LOG = LogFactory.getLog(Utilities.class.getName());
+  private static final String CLASS_NAME = Utilities.class.getName();
+  private static final Log LOG = LogFactory.getLog(CLASS_NAME);
 
   public static void clearWork(Configuration conf) {
     Path mapPath = getPlanPath(conf, MAP_PLAN_NAME);
@@ -649,7 +651,7 @@ public final class Utilities {
   }
   private static void serializePlan(Object plan, OutputStream out, Configuration conf, boolean cloningPlan) {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
-    perfLogger.PerfLogBegin(LOG, PerfLogger.SERIALIZE_PLAN);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SERIALIZE_PLAN);
     String serializationType = conf.get(HiveConf.ConfVars.PLAN_SERIALIZATION.varname, "kryo");
     LOG.info("Serializing " + plan.getClass().getSimpleName() + " via " + serializationType);
     if("javaXML".equalsIgnoreCase(serializationType)) {
@@ -661,7 +663,7 @@ public final class Utilities {
         serializeObjectByKryo(runtimeSerializationKryo.get(), plan, out);
       }
     }
-    perfLogger.PerfLogEnd(LOG, PerfLogger.SERIALIZE_PLAN);
+    perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SERIALIZE_PLAN);
   }
   /**
    * Serializes the plan.
@@ -675,7 +677,7 @@ public final class Utilities {
 
   private static <T> T deserializePlan(InputStream in, Class<T> planClass, Configuration conf, boolean cloningPlan) {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
-    perfLogger.PerfLogBegin(LOG, PerfLogger.DESERIALIZE_PLAN);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.DESERIALIZE_PLAN);
     T plan;
     String serializationType = conf.get(HiveConf.ConfVars.PLAN_SERIALIZATION.varname, "kryo");
     LOG.info("Deserializing " + planClass.getSimpleName() + " via " + serializationType);
@@ -688,7 +690,7 @@ public final class Utilities {
         plan = deserializeObjectByKryo(runtimeSerializationKryo.get(), in, planClass);
       }
     }
-    perfLogger.PerfLogEnd(LOG, PerfLogger.DESERIALIZE_PLAN);
+    perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.DESERIALIZE_PLAN);
     return plan;
   }
   /**
@@ -709,13 +711,13 @@ public final class Utilities {
   public static MapredWork clonePlan(MapredWork plan) {
     // TODO: need proper clone. Meanwhile, let's at least keep this horror in one place
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
-    perfLogger.PerfLogBegin(LOG, PerfLogger.CLONE_PLAN);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.CLONE_PLAN);
     ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
     Configuration conf = new Configuration();
     serializePlan(plan, baos, conf, true);
     MapredWork newPlan = deserializePlan(new ByteArrayInputStream(baos.toByteArray()),
         MapredWork.class, conf, true);
-    perfLogger.PerfLogEnd(LOG, PerfLogger.CLONE_PLAN);
+    perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.CLONE_PLAN);
     return newPlan;
   }
 
@@ -1985,7 +1987,7 @@ public final class Utilities {
   public static ContentSummary getInputSummary(Context ctx, MapWork work, PathFilter filter)
       throws IOException {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
-    perfLogger.PerfLogBegin(LOG, PerfLogger.INPUT_SUMMARY);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.INPUT_SUMMARY);
 
     long[] summary = {0, 0, 0};
 
@@ -2121,7 +2123,7 @@ public final class Utilities {
               + cs.getFileCount() + " directory count: " + cs.getDirectoryCount());
         }
 
-        perfLogger.PerfLogEnd(LOG, PerfLogger.INPUT_SUMMARY);
+        perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.INPUT_SUMMARY);
         return new ContentSummary(summary[0], summary[1], summary[2]);
       } finally {
         HiveInterruptUtils.remove(interrup);

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java?rev=1524278&r1=1524277&r2=1524278&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java Wed Sep 18 02:32:52 2013
@@ -68,8 +68,8 @@ import org.apache.hadoop.mapred.TextInpu
 public class CombineHiveInputFormat<K extends WritableComparable, V extends Writable>
     extends HiveInputFormat<K, V> {
 
-  public static final Log LOG = LogFactory
-      .getLog("org.apache.hadoop.hive.ql.io.CombineHiveInputFormat");
+  private static final String CLASS_NAME = CombineHiveInputFormat.class.getName();
+  public static final Log LOG = LogFactory.getLog(CLASS_NAME);
 
   /**
    * CombineHiveInputSplit encapsulates an InputSplit with its corresponding
@@ -264,7 +264,7 @@ public class CombineHiveInputFormat<K ex
   @Override
   public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
-    perfLogger.PerfLogBegin(LOG, PerfLogger.GET_SPLITS);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.GET_SPLITS);
     init(job);
     Map<String, ArrayList<String>> pathToAliases = mrwork.getPathToAliases();
     Map<String, Operator<? extends OperatorDesc>> aliasToWork =
@@ -275,7 +275,7 @@ public class CombineHiveInputFormat<K ex
     InputSplit[] splits = null;
     if (combine == null) {
       splits = super.getSplits(job, numSplits);
-      perfLogger.PerfLogEnd(LOG, PerfLogger.GET_SPLITS);
+      perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.GET_SPLITS);
       return splits;
     }
 
@@ -328,7 +328,7 @@ public class CombineHiveInputFormat<K ex
             dirs.offer(path);
           } else if ((new CompressionCodecFactory(job)).getCodec(path) != null) {
             splits = super.getSplits(job, numSplits);
-            perfLogger.PerfLogEnd(LOG, PerfLogger.GET_SPLITS);
+            perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.GET_SPLITS);
             return splits;
           }
 
@@ -341,7 +341,7 @@ public class CombineHiveInputFormat<K ex
               } else if ((new CompressionCodecFactory(job)).getCodec(
                   fStatus[idx].getPath()) != null) {
                 splits = super.getSplits(job, numSplits);
-                perfLogger.PerfLogEnd(LOG, PerfLogger.GET_SPLITS);
+                perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.GET_SPLITS);
                 return splits;
               }
             }
@@ -351,7 +351,7 @@ public class CombineHiveInputFormat<K ex
 
       if (inputFormat instanceof SymlinkTextInputFormat) {
         splits = super.getSplits(job, numSplits);
-        perfLogger.PerfLogEnd(LOG, PerfLogger.GET_SPLITS);
+        perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.GET_SPLITS);
         return splits;
       }
 
@@ -422,7 +422,7 @@ public class CombineHiveInputFormat<K ex
     }
 
     LOG.info("number of splits " + result.size());
-    perfLogger.PerfLogEnd(LOG, PerfLogger.GET_SPLITS);
+    perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.GET_SPLITS);
     return result.toArray(new CombineHiveInputSplit[result.size()]);
   }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java?rev=1524278&r1=1524277&r2=1524278&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java Wed Sep 18 02:32:52 2013
@@ -65,8 +65,8 @@ import org.apache.hadoop.util.Reflection
 public class HiveInputFormat<K extends WritableComparable, V extends Writable>
     implements InputFormat<K, V>, JobConfigurable {
 
-  public static final Log LOG = LogFactory
-      .getLog("org.apache.hadoop.hive.ql.io.HiveInputFormat");
+  public static final String CLASS_NAME = HiveInputFormat.class.getName();
+  public static final Log LOG = LogFactory.getLog(CLASS_NAME);
 
   /**
    * HiveInputSplit encapsulates an InputSplit with its corresponding
@@ -259,7 +259,7 @@ public class HiveInputFormat<K extends W
 
   public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
-    perfLogger.PerfLogBegin(LOG, PerfLogger.GET_SPLITS);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.GET_SPLITS);
     init(job);
 
     Path[] dirs = FileInputFormat.getInputPaths(job);
@@ -298,7 +298,7 @@ public class HiveInputFormat<K extends W
     }
 
     LOG.info("number of splits " + result.size());
-    perfLogger.PerfLogEnd(LOG, PerfLogger.GET_SPLITS);
+    perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.GET_SPLITS);
     return result.toArray(new HiveInputSplit[result.size()]);
   }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java?rev=1524278&r1=1524277&r2=1524278&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java Wed Sep 18 02:32:52 2013
@@ -92,9 +92,9 @@ public class PerfLogger {
    * @param _log the logging object to be used.
    * @param method method or ID that identifies this perf log element.
    */
-  public void PerfLogBegin(Log _log, String method) {
+  public void PerfLogBegin(String callerName, String method) {
     long startTime = System.currentTimeMillis();
-    _log.info("<PERFLOG method=" + method + ">");
+    LOG.info("<PERFLOG method=" + method + " from=" + callerName + ">");
     startTimes.put(method, new Long(startTime));
   }
 
@@ -104,7 +104,7 @@ public class PerfLogger {
    * @param method
    * @return long duration  the difference between now and startTime, or -1 if startTime is null
    */
-  public long PerfLogEnd(Log _log, String method) {
+  public long PerfLogEnd(String callerName, String method) {
     Long startTime = startTimes.get(method);
     long endTime = System.currentTimeMillis();
     long duration = -1;
@@ -120,8 +120,8 @@ public class PerfLogger {
       duration = endTime - startTime.longValue();
       sb.append(" duration=").append(duration);
     }
-    sb.append(">");
-    _log.info(sb);
+    sb.append(" from=").append(callerName).append(">");
+    LOG.info(sb);
 
     return duration;
   }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java?rev=1524278&r1=1524277&r2=1524278&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java Wed Sep 18 02:32:52 2013
@@ -66,8 +66,8 @@ import org.apache.hadoop.hive.serde2.obj
 public class PartitionPruner implements Transform {
 
   // The log
-  private static final Log LOG = LogFactory
-    .getLog("hive.ql.optimizer.ppr.PartitionPruner");
+  public static final String CLASS_NAME = PartitionPruner.class.getName();
+  public static final Log LOG = LogFactory.getLog(CLASS_NAME);
 
   /*
    * (non-Javadoc)
@@ -307,7 +307,7 @@ public class PartitionPruner implements 
   static private boolean pruneBySequentialScan(Table tab, Set<Partition> partitions,
       ExprNodeDesc prunerExpr, HiveConf conf) throws Exception {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
-    perfLogger.PerfLogBegin(LOG, PerfLogger.PRUNE_LISTING);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PRUNE_LISTING);
 
     List<String> partNames = Hive.get().getPartitionNames(
         tab.getDbName(), tab.getTableName(), (short) -1);
@@ -317,13 +317,13 @@ public class PartitionPruner implements 
 
     boolean hasUnknownPartitions = prunePartitionNames(
         partCols, prunerExpr, defaultPartitionName, partNames);
-    perfLogger.PerfLogEnd(LOG, PerfLogger.PRUNE_LISTING);
+    perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PRUNE_LISTING);
 
-    perfLogger.PerfLogBegin(LOG, PerfLogger.PARTITION_RETRIEVING);
+    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PARTITION_RETRIEVING);
     if (!partNames.isEmpty()) {
       partitions.addAll(Hive.get().getPartitionsByNames(tab, partNames));
     }
-    perfLogger.PerfLogEnd(LOG, PerfLogger.PARTITION_RETRIEVING);
+    perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PARTITION_RETRIEVING);
     return hasUnknownPartitions;
   }