You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2018/03/19 09:56:08 UTC

[1/2] hive git commit: HIVE-18716: Delete unnecessary parameters from TaskFactory (Gergely Hajós via Zoltan Haindrich, Zoltan Haindrich)

Repository: hive
Updated Branches:
  refs/heads/master d2d50e694 -> 94152c997


http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
index b7fbea4..9a91e3f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
@@ -417,14 +417,14 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
     } else {
       CopyWork cw = new CopyWork(dataPath, destPath, false);
       cw.setSkipSourceMmDirs(isSourceMm);
-      copyTask = TaskFactory.get(cw, x.getConf());
+      copyTask = TaskFactory.get(cw);
     }
 
     LoadTableDesc loadTableWork = new LoadTableDesc(
         loadPath, Utilities.getTableDesc(table), new TreeMap<>(), lft, writeId);
     loadTableWork.setStmtId(stmtId);
     MoveWork mv = new MoveWork(x.getInputs(), x.getOutputs(), loadTableWork, null, false);
-    Task<?> loadTableTask = TaskFactory.get(mv, x.getConf());
+    Task<?> loadTableTask = TaskFactory.get(mv);
     copyTask.addDependentTask(loadTableTask);
     x.getTasks().add(copyTask);
     return loadTableTask;
@@ -470,7 +470,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
         x.getInputs(),
         x.getOutputs(),
         addPartitionDesc
-    ), x.getConf());
+    ));
   }
 
  private static Task<?> addSinglePartition(URI fromURI, FileSystem fs, ImportTableDesc tblDesc,
@@ -485,7 +485,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       // addPartitionDesc already has the right partition location
       @SuppressWarnings("unchecked")
       Task<?> addPartTask = TaskFactory.get(new DDLWork(x.getInputs(),
-          x.getOutputs(), addPartitionDesc), x.getConf());
+          x.getOutputs(), addPartitionDesc));
       return addPartTask;
     } else {
       String srcLocation = partSpec.getLocation();
@@ -515,11 +515,11 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       } else {
         CopyWork cw = new CopyWork(new Path(srcLocation), destPath, false);
         cw.setSkipSourceMmDirs(isSourceMm);
-        copyTask = TaskFactory.get(cw, x.getConf());
+        copyTask = TaskFactory.get(cw);
       }
 
       Task<?> addPartTask = TaskFactory.get(new DDLWork(x.getInputs(),
-          x.getOutputs(), addPartitionDesc), x.getConf());
+          x.getOutputs(), addPartitionDesc));
       // Note: this sets LoadFileType incorrectly for ACID; is that relevant for import?
       //       See setLoadFileType and setIsAcidIow calls elsewhere for an example.
       LoadTableDesc loadTableWork = new LoadTableDesc(moveTaskSrc, Utilities.getTableDesc(table),
@@ -529,7 +529,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       loadTableWork.setStmtId(stmtId);
       loadTableWork.setInheritTableSpecs(false);
       Task<?> loadPartTask = TaskFactory.get(new MoveWork(
-          x.getInputs(), x.getOutputs(), loadTableWork, null, false), x.getConf(), true);
+          x.getInputs(), x.getOutputs(), loadTableWork, null, false));
       copyTask.addDependentTask(loadPartTask);
       addPartTask.addDependentTask(loadPartTask);
       x.getTasks().add(copyTask);
@@ -831,7 +831,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       if (table.isPartitioned()) {
         x.getLOG().debug("table partitioned");
         Task<?> ict = createImportCommitTask(
-            table.getDbName(), table.getTableName(), writeId, stmtId, x.getConf(),
+            table.getDbName(), table.getTableName(), writeId, stmtId,
             AcidUtils.isInsertOnlyTable(table.getParameters()));
 
         for (AddPartitionDesc addPartitionDesc : partitionDescs) {
@@ -868,7 +868,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
 
       if (isPartitioned(tblDesc)) {
         Task<?> ict = createImportCommitTask(
-            tblDesc.getDatabaseName(), tblDesc.getTableName(), writeId, stmtId, x.getConf(),
+            tblDesc.getDatabaseName(), tblDesc.getTableName(), writeId, stmtId,
             AcidUtils.isInsertOnlyTable(tblDesc.getTblProps()));
         for (AddPartitionDesc addPartitionDesc : partitionDescs) {
           t.addDependentTask(addSinglePartition(fromURI, fs, tblDesc, table, wh, addPartitionDesc,
@@ -903,11 +903,10 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   private static Task<?> createImportCommitTask(
-      String dbName, String tblName, Long writeId, int stmtId, HiveConf conf, boolean isMmTable) {
+      String dbName, String tblName, Long writeId, int stmtId, boolean isMmTable) {
     // TODO: noop, remove?
-    @SuppressWarnings("unchecked")
     Task<ImportCommitWork> ict = (!isMmTable) ? null : TaskFactory.get(
-        new ImportCommitWork(dbName, tblName, writeId, stmtId), conf);
+        new ImportCommitWork(dbName, tblName, writeId, stmtId));
     return ict;
   }
 
@@ -996,7 +995,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       if (!replicationSpec.isMetadataOnly()) {
         if (isPartitioned(tblDesc)) {
           Task<?> ict = createImportCommitTask(
-              tblDesc.getDatabaseName(), tblDesc.getTableName(), writeId, stmtId, x.getConf(),
+              tblDesc.getDatabaseName(), tblDesc.getTableName(), writeId, stmtId,
               AcidUtils.isInsertOnlyTable(tblDesc.getTblProps()));
           for (AddPartitionDesc addPartitionDesc : partitionDescs) {
             addPartitionDesc.setReplicationSpec(replicationSpec);
@@ -1022,7 +1021,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
           Map<String, String> partSpec = addPartitionDesc.getPartition(0).getPartSpec();
           org.apache.hadoop.hive.ql.metadata.Partition ptn = null;
           Task<?> ict = replicationSpec.isMetadataOnly() ? null : createImportCommitTask(
-              tblDesc.getDatabaseName(), tblDesc.getTableName(), writeId, stmtId, x.getConf(),
+              tblDesc.getDatabaseName(), tblDesc.getTableName(), writeId, stmtId,
               AcidUtils.isInsertOnlyTable(tblDesc.getTblProps()));
           if ((ptn = x.getHive().getPartition(table, partSpec, false)) == null) {
             if (!replicationSpec.isMetadataOnly()){

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
index fb3bfda..d5aace0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
@@ -344,7 +344,7 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     Task<? extends Serializable> childTask = TaskFactory.get(
         new MoveWork(getInputs(), getOutputs(), loadTableWork, null, true,
-            isLocal), conf
+            isLocal)
     );
     if (rTask != null) {
       rTask.addDependentTask(childTask);
@@ -364,7 +364,7 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
       basicStatsWork.setNoStatsAggregator(true);
       basicStatsWork.setClearAggregatorStats(true);
       StatsWork columnStatsWork = new StatsWork(ts.tableHandle, basicStatsWork, conf);
-      statTask = TaskFactory.get(columnStatsWork, conf);
+      statTask = TaskFactory.get(columnStatsWork);
     }
 
     if (statTask != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
index a4c32f4..762e438 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
@@ -140,7 +140,7 @@ public class MacroSemanticAnalyzer extends BaseSemanticAnalyzer {
         body = sa.genExprNodeDesc((ASTNode)ast.getChild(2), rowResolver);
     }
     CreateMacroDesc desc = new CreateMacroDesc(functionName, macroColNames, macroColTypes, body);
-    rootTasks.add(TaskFactory.get(new FunctionWork(desc), conf));
+    rootTasks.add(TaskFactory.get(new FunctionWork(desc)));
 
     addEntities();
   }
@@ -164,7 +164,7 @@ public class MacroSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     DropMacroDesc desc = new DropMacroDesc(functionName);
-    rootTasks.add(TaskFactory.get(new FunctionWork(desc), conf));
+    rootTasks.add(TaskFactory.get(new FunctionWork(desc)));
 
     addEntities();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
index 6117034..5961059 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
@@ -103,7 +103,7 @@ public class ProcessAnalyzeTable implements NodeProcessor {
           PrunedPartitionList partList = new PrunedPartitionList(table, confirmedParts, partCols, false);
           statWork.addInputPartitions(partList.getPartitions());
         }
-        Task<StatsWork> snjTask = TaskFactory.get(statWork, parseContext.getConf());
+        Task<StatsWork> snjTask = TaskFactory.get(statWork);
         snjTask.setParentTasks(null);
         context.rootTasks.remove(context.currentTask);
         context.rootTasks.add(snjTask);
@@ -120,7 +120,7 @@ public class ProcessAnalyzeTable implements NodeProcessor {
         columnStatsWork.collectStatsFromAggregator(tableScan.getConf());
 
         columnStatsWork.setSourceTask(context.currentTask);
-        Task<StatsWork> statsTask = TaskFactory.get(columnStatsWork, parseContext.getConf());
+        Task<StatsWork> statsTask = TaskFactory.get(columnStatsWork);
         context.currentTask.addDependentTask(statsTask);
 
         // ANALYZE TABLE T [PARTITION (...)] COMPUTE STATISTICS noscan;

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
index 796ab0d..753f039 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
@@ -185,7 +185,7 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
               ErrorMsg.INVALID_PATH.getMsg(ast),
               maxEventLimit,
               ctx.getResFile().toUri().toString()
-          ), conf, true);
+      ), conf);
       rootTasks.add(replDumpWorkTask);
       if (dbNameOrPattern != null) {
         for (String dbName : Utils.matchesDb(db, dbNameOrPattern)) {
@@ -323,7 +323,7 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
         ReplLoadWork replLoadWork =
             new ReplLoadWork(conf, loadPath.toString(), dbNameOrPattern, tblNameOrPattern,
                 queryState.getLineageState(), SessionState.get().getTxnMgr().getCurrentTxnId());
-        rootTasks.add(TaskFactory.get(replLoadWork, conf, true));
+        rootTasks.add(TaskFactory.get(replLoadWork, conf));
         return;
       }
 
@@ -354,7 +354,7 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
 
         ReplLoadWork replLoadWork = new ReplLoadWork(conf, loadPath.toString(), dbNameOrPattern,
             queryState.getLineageState(), SessionState.get().getTxnMgr().getCurrentTxnId());
-        rootTasks.add(TaskFactory.get(replLoadWork, conf, true));
+        rootTasks.add(TaskFactory.get(replLoadWork, conf));
         //
         //        for (FileStatus dir : dirsInLoadPath) {
         //          analyzeDatabaseLoad(dbNameOrPattern, fs, dir);
@@ -364,7 +364,7 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
         // We need to guarantee that the directory listing we got is in order of evid.
         Arrays.sort(dirsInLoadPath, new EventDumpDirComparator());
 
-        Task<? extends Serializable> evTaskRoot = TaskFactory.get(new DependencyCollectionWork(), conf);
+        Task<? extends Serializable> evTaskRoot = TaskFactory.get(new DependencyCollectionWork());
         Task<? extends Serializable> taskChainTail = evTaskRoot;
 
         ReplLogger replLogger = new IncrementalLoadLogger(dbNameOrPattern,
@@ -403,7 +403,7 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
             ReplStateLogWork replStateLogWork = new ReplStateLogWork(replLogger,
                                                           dir.getPath().getName(),
                                                           eventDmd.getDumpType().toString());
-            Task<? extends Serializable> barrierTask = TaskFactory.get(replStateLogWork, conf);
+            Task<? extends Serializable> barrierTask = TaskFactory.get(replStateLogWork);
             for (Task<? extends Serializable> t : evTasks){
               t.addDependentTask(barrierTask);
               LOG.debug("Added {}:{} as a precursor of barrier task {}:{}",
@@ -420,7 +420,7 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
           Map<String, String> dbProps = new HashMap<>();
           dbProps.put(ReplicationSpec.KEY.CURR_STATE_ID.toString(), String.valueOf(dmd.getEventTo()));
           ReplStateLogWork replStateLogWork = new ReplStateLogWork(replLogger, dbProps);
-          Task<? extends Serializable> barrierTask = TaskFactory.get(replStateLogWork, conf);
+          Task<? extends Serializable> barrierTask = TaskFactory.get(replStateLogWork);
           taskChainTail.addDependentTask(barrierTask);
           LOG.debug("Added {}:{} as a precursor of barrier task {}:{}",
                   taskChainTail.getClass(), taskChainTail.getId(),
@@ -473,7 +473,7 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
     alterTblDesc.setPartSpec((HashMap<String, String>)partSpec);
 
     Task<? extends Serializable> updateReplIdTask = TaskFactory.get(
-                      new DDLWork(inputs, outputs, alterTblDesc), conf, true);
+        new DDLWork(inputs, outputs, alterTblDesc), conf);
 
     // Link the update repl state task with dependency collection task
     if (preCursor != null) {
@@ -495,7 +495,7 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
     AlterDatabaseDesc alterDbDesc = new AlterDatabaseDesc(
                             dbName, mapProp, new ReplicationSpec(replState, replState));
     Task<? extends Serializable> updateReplIdTask = TaskFactory.get(
-                            new DDLWork(inputs, outputs, alterDbDesc), conf, true);
+        new DDLWork(inputs, outputs, alterDbDesc), conf);
 
     // Link the update repl state task with dependency collection task
     if (preCursor != null) {
@@ -523,7 +523,7 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     // Create a barrier task for dependency collection of import tasks
-    Task<? extends Serializable> barrierTask = TaskFactory.get(new DependencyCollectionWork(), conf);
+    Task<? extends Serializable> barrierTask = TaskFactory.get(new DependencyCollectionWork());
 
     // Link import tasks to the barrier task which will in-turn linked with repl state update tasks
     for (Task<? extends Serializable> t : importTasks){

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 2342fff..aa2d060 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -6701,7 +6701,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     alterTblDesc.setOldName(tableName);
     alterTblDesc.setProps(mapProp);
     alterTblDesc.setDropIfExists(true);
-    this.rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTblDesc), conf));
+    this.rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTblDesc)));
   }
 
   private ImmutableBitSet getEnabledNotNullConstraints(Table tbl) throws HiveException{
@@ -7567,7 +7567,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     PreInsertTableDesc preInsertTableDesc = new PreInsertTableDesc(table, overwrite);
     InsertTableDesc insertTableDesc = new InsertTableDesc(table, overwrite);
     this.rootTasks
-        .add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), preInsertTableDesc), conf));
+        .add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), preInsertTableDesc)));
     TaskFactory
         .getAndMakeChild(new DDLWork(getInputs(), getOutputs(), insertTableDesc), conf, tasks);
   }
@@ -12746,7 +12746,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       // outputs is empty, which means this create table happens in the current
       // database.
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-          crtTblDesc), conf));
+            crtTblDesc)));
       break;
 
     case CTLT: // create table like <tbl_name>
@@ -12766,7 +12766,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           storageFormat.getSerde(), storageFormat.getSerdeProps(), tblProps, ifNotExists,
           likeTableName, isUserStorageFormat);
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-          crtTblLikeDesc), conf));
+            crtTblLikeDesc)));
       break;
 
     case CTAS: // create table as select
@@ -12970,7 +12970,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           ifNotExists, orReplace, isAlterViewAs, storageFormat.getInputFormat(),
           storageFormat.getOutputFormat(), storageFormat.getSerde());
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-          createVwDesc), conf));
+          createVwDesc)));
       addDbAndTabToOutputs(qualTabName, TableType.VIRTUAL_VIEW, tblProps);
       queryState.setCommandType(HiveOperation.CREATEVIEW);
     }
@@ -14207,7 +14207,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
    */
   private void useCachedResult(QueryResultsCache.CacheEntry cacheEntry) {
     // Change query FetchTask to use new location specified in results cache.
-    FetchTask fetchTask = (FetchTask) TaskFactory.get(cacheEntry.getFetchWork(), conf);
+    FetchTask fetchTask = (FetchTask) TaskFactory.get(cacheEntry.getFetchWork());
     setFetchTask(fetchTask);
 
     queryState.setCommandType(cacheEntry.getQueryInfo().getHiveOperation());

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
index 5e94bb7..c268ddc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
@@ -198,7 +198,7 @@ public abstract class TaskCompiler {
           fetch.setIsUsingThriftJDBCBinarySerDe(false);
       }
 
-      pCtx.setFetchTask((FetchTask) TaskFactory.get(fetch, conf));
+      pCtx.setFetchTask((FetchTask) TaskFactory.get(fetch));
 
       // For the FetchTask, the limit optimization requires we fetch all the rows
       // in memory and count how many rows we get. It's not practical if the
@@ -219,8 +219,7 @@ public abstract class TaskCompiler {
     } else if (!isCStats) {
       for (LoadTableDesc ltd : loadTableWork) {
         Task<MoveWork> tsk = TaskFactory
-            .get(new MoveWork(null, null, ltd, null, false),
-                conf);
+            .get(new MoveWork(null, null, ltd, null, false));
         mvTask.add(tsk);
       }
 
@@ -235,8 +234,7 @@ public abstract class TaskCompiler {
           oneLoadFileForCtas = false;
         }
         mvTask.add(TaskFactory
-            .get(new MoveWork(null, null, null, lfd, false),
-                conf));
+            .get(new MoveWork(null, null, null, lfd, false)));
       }
     }
 
@@ -326,13 +324,13 @@ public abstract class TaskCompiler {
       CreateTableDesc crtTblDesc = pCtx.getCreateTable();
       crtTblDesc.validate(conf);
       Task<? extends Serializable> crtTblTask = TaskFactory.get(new DDLWork(
-          inputs, outputs, crtTblDesc), conf);
+          inputs, outputs, crtTblDesc));
       patchUpAfterCTASorMaterializedView(rootTasks, outputs, crtTblTask);
     } else if (pCtx.getQueryProperties().isMaterializedView()) {
       // generate a DDL task and make it a dependent task of the leaf
       CreateViewDesc viewDesc = pCtx.getCreateViewDesc();
       Task<? extends Serializable> crtViewTask = TaskFactory.get(new DDLWork(
-          inputs, outputs, viewDesc), conf);
+          inputs, outputs, viewDesc));
       patchUpAfterCTASorMaterializedView(rootTasks, outputs, crtViewTask);
     } else if (pCtx.getMaterializedViewUpdateDesc() != null) {
       // If there is a materialized view update desc, we create introduce it at the end
@@ -391,13 +389,13 @@ public abstract class TaskCompiler {
       if (partitions.size() > 0) {
         columnStatsWork.addInputPartitions(parseContext.getPrunedPartitions(tableScan).getPartitions());
       }
-      return TaskFactory.get(columnStatsWork, parseContext.getConf());
+      return TaskFactory.get(columnStatsWork);
     } else {
       BasicStatsWork statsWork = new BasicStatsWork(tableScan.getConf().getTableMetadata().getTableSpec());
       StatsWork columnStatsWork = new StatsWork(table, statsWork, parseContext.getConf());
       columnStatsWork.collectStatsFromAggregator(tableScan.getConf());
       columnStatsWork.setSourceTask(currentTask);
-      return TaskFactory.get(columnStatsWork, parseContext.getConf());
+      return TaskFactory.get(columnStatsWork);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
index 4cc57e5..18ed6fb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
@@ -57,15 +57,12 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 /**
  * Default implementation of HiveAuthorizationTaskFactory
  */
-@SuppressWarnings("unchecked")
 public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFactory {
 
-  private final HiveConf conf;
   // Assumes one instance of this + single-threaded compilation for each query.
   private final Hive db;
 
-  public HiveAuthorizationTaskFactoryImpl(HiveConf conf, Hive db) {
-    this.conf = conf;
+    public HiveAuthorizationTaskFactoryImpl(HiveConf conf, Hive db) {
     this.db = db;
   }
 
@@ -74,14 +71,14 @@ public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFa
       HashSet<WriteEntity> outputs) {
     String roleName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText());
     RoleDDLDesc roleDesc = new RoleDDLDesc(roleName, PrincipalType.ROLE, RoleDDLDesc.RoleOperation.CREATE_ROLE, null);
-    return TaskFactory.get(new DDLWork(inputs, outputs, roleDesc), conf);
+    return TaskFactory.get(new DDLWork(inputs, outputs, roleDesc));
   }
   @Override
   public Task<? extends Serializable> createDropRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
       HashSet<WriteEntity> outputs) {
     String roleName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText());
     RoleDDLDesc roleDesc = new RoleDDLDesc(roleName, PrincipalType.ROLE, RoleDDLDesc.RoleOperation.DROP_ROLE, null);
-    return TaskFactory.get(new DDLWork(inputs, outputs, roleDesc), conf);
+    return TaskFactory.get(new DDLWork(inputs, outputs, roleDesc));
   }
   @Override
   public Task<? extends Serializable> createShowRoleGrantTask(ASTNode ast, Path resultFile,
@@ -103,7 +100,7 @@ public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFa
     RoleDDLDesc roleDesc = new RoleDDLDesc(principalName, principalType,
         RoleDDLDesc.RoleOperation.SHOW_ROLE_GRANT, null);
     roleDesc.setResFile(resultFile.toString());
-    return TaskFactory.get(new DDLWork(inputs, outputs,  roleDesc), conf);
+    return TaskFactory.get(new DDLWork(inputs, outputs, roleDesc));
   }
   @Override
   public Task<? extends Serializable> createGrantTask(ASTNode ast, HashSet<ReadEntity> inputs,
@@ -130,7 +127,7 @@ public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFa
 
     GrantDesc grantDesc = new GrantDesc(privilegeObj, privilegeDesc,
         principalDesc, userName, PrincipalType.USER, grantOption);
-    return TaskFactory.get(new DDLWork(inputs, outputs, grantDesc), conf);
+    return TaskFactory.get(new DDLWork(inputs, outputs, grantDesc));
   }
 
   @Override
@@ -149,7 +146,7 @@ public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFa
       }
     }
     RevokeDesc revokeDesc = new RevokeDesc(privilegeDesc, principalDesc, hiveObj, grantOption);
-    return TaskFactory.get(new DDLWork(inputs, outputs, revokeDesc), conf);
+    return TaskFactory.get(new DDLWork(inputs, outputs, revokeDesc));
   }
   @Override
   public Task<? extends Serializable> createGrantRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
@@ -182,7 +179,7 @@ public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFa
 
     ShowGrantDesc showGrant = new ShowGrantDesc(resultFile.toString(),
         principalDesc, privHiveObj);
-    return TaskFactory.get(new DDLWork(inputs, outputs, showGrant), conf);
+    return TaskFactory.get(new DDLWork(inputs, outputs, showGrant));
   }
 
   @Override
@@ -217,7 +214,7 @@ public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFa
 
     GrantRevokeRoleDDL grantRevokeRoleDDL = new GrantRevokeRoleDDL(isGrant,
         roles, principalDesc, roleOwnerName, PrincipalType.USER, isAdmin);
-    return TaskFactory.get(new DDLWork(inputs, outputs, grantRevokeRoleDDL), conf);
+    return TaskFactory.get(new DDLWork(inputs, outputs, grantRevokeRoleDDL));
   }
 
   private PrivilegeObjectDesc analyzePrivilegeObject(ASTNode ast,
@@ -333,7 +330,7 @@ public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFa
       HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs)
       throws SemanticException {
     return TaskFactory.get(new DDLWork(inputs, outputs, new RoleDDLDesc(roleName, PrincipalType.ROLE,
-      RoleDDLDesc.RoleOperation.SET_ROLE, null)), conf);
+        RoleDDLDesc.RoleOperation.SET_ROLE, null)));
   }
 
   @Override
@@ -342,7 +339,7 @@ public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFa
       throws SemanticException {
     RoleDDLDesc ddlDesc = new RoleDDLDesc(null, RoleDDLDesc.RoleOperation.SHOW_CURRENT_ROLE);
     ddlDesc.setResFile(resFile.toString());
-    return TaskFactory.get(new DDLWork(inputs, outputs, ddlDesc), conf);
+    return TaskFactory.get(new DDLWork(inputs, outputs, ddlDesc));
   }
 
   @Override
@@ -360,7 +357,7 @@ public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFa
     RoleDDLDesc roleDDLDesc = new RoleDDLDesc(roleName, PrincipalType.ROLE,
      RoleOperation.SHOW_ROLE_PRINCIPALS, null);
     roleDDLDesc.setResFile(resFile.toString());
-    return TaskFactory.get(new DDLWork(inputs, outputs, roleDDLDesc), conf);
+    return TaskFactory.get(new DDLWork(inputs, outputs, roleDDLDesc));
   }
 
   @Override
@@ -369,7 +366,7 @@ public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFa
     RoleDDLDesc showRolesDesc = new RoleDDLDesc(null, null, RoleDDLDesc.RoleOperation.SHOW_ROLES,
         null);
     showRolesDesc.setResFile(resFile.toString());
-    return TaskFactory.get(new DDLWork(inputs, outputs, showRolesDesc), conf);
+    return TaskFactory.get(new DDLWork(inputs, outputs, showRolesDesc));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddForeignKeyHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddForeignKeyHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddForeignKeyHandler.java
index 0fd970a..8f76230 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddForeignKeyHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddForeignKeyHandler.java
@@ -69,7 +69,7 @@ public class AddForeignKeyHandler extends AbstractMessageHandler {
 
     AlterTableDesc addConstraintsDesc = new AlterTableDesc(actualDbName + "." + actualTblName, new ArrayList<SQLPrimaryKey>(), fks,
         new ArrayList<SQLUniqueConstraint>(), context.eventOnlyReplicationSpec());
-    Task<DDLWork> addConstraintsTask = TaskFactory.get(new DDLWork(readEntitySet, writeEntitySet, addConstraintsDesc), context.hiveConf);
+    Task<DDLWork> addConstraintsTask = TaskFactory.get(new DDLWork(readEntitySet, writeEntitySet, addConstraintsDesc));
     tasks.add(addConstraintsTask);
     context.log.debug("Added add constrains task : {}:{}", addConstraintsTask.getId(), actualTblName);
     updatedMetadata.set(context.dmd.getEventTo().toString(), actualDbName, actualTblName, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddNotNullConstraintHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddNotNullConstraintHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddNotNullConstraintHandler.java
index 3425858..7889e03 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddNotNullConstraintHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddNotNullConstraintHandler.java
@@ -66,7 +66,7 @@ public class AddNotNullConstraintHandler extends AbstractMessageHandler {
 
     AlterTableDesc addConstraintsDesc = new AlterTableDesc(actualDbName + "." + actualTblName, new ArrayList<SQLPrimaryKey>(), new ArrayList<SQLForeignKey>(),
         new ArrayList<SQLUniqueConstraint>(), nns, new ArrayList<SQLDefaultConstraint>(), context.eventOnlyReplicationSpec());
-    Task<DDLWork> addConstraintsTask = TaskFactory.get(new DDLWork(readEntitySet, writeEntitySet, addConstraintsDesc), context.hiveConf);
+    Task<DDLWork> addConstraintsTask = TaskFactory.get(new DDLWork(readEntitySet, writeEntitySet, addConstraintsDesc));
     tasks.add(addConstraintsTask);
     context.log.debug("Added add constrains task : {}:{}", addConstraintsTask.getId(), actualTblName);
     updatedMetadata.set(context.dmd.getEventTo().toString(), actualDbName, actualTblName, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddPrimaryKeyHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddPrimaryKeyHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddPrimaryKeyHandler.java
index d7ee223..f9a615a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddPrimaryKeyHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddPrimaryKeyHandler.java
@@ -64,7 +64,7 @@ public class AddPrimaryKeyHandler extends AbstractMessageHandler {
 
     AlterTableDesc addConstraintsDesc = new AlterTableDesc(actualDbName + "." + actualTblName, pks, new ArrayList<SQLForeignKey>(),
         new ArrayList<SQLUniqueConstraint>(), context.eventOnlyReplicationSpec());
-    Task<DDLWork> addConstraintsTask = TaskFactory.get(new DDLWork(readEntitySet, writeEntitySet, addConstraintsDesc), context.hiveConf);
+    Task<DDLWork> addConstraintsTask = TaskFactory.get(new DDLWork(readEntitySet, writeEntitySet, addConstraintsDesc));
     tasks.add(addConstraintsTask);
     context.log.debug("Added add constrains task : {}:{}", addConstraintsTask.getId(), actualTblName);
     updatedMetadata.set(context.dmd.getEventTo().toString(), actualDbName, actualTblName, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddUniqueConstraintHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddUniqueConstraintHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddUniqueConstraintHandler.java
index 0d9c700..757381a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddUniqueConstraintHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AddUniqueConstraintHandler.java
@@ -64,7 +64,7 @@ public class AddUniqueConstraintHandler extends AbstractMessageHandler {
 
     AlterTableDesc addConstraintsDesc = new AlterTableDesc(actualDbName + "." + actualTblName, new ArrayList<SQLPrimaryKey>(), new ArrayList<SQLForeignKey>(),
         uks, context.eventOnlyReplicationSpec());
-    Task<DDLWork> addConstraintsTask = TaskFactory.get(new DDLWork(readEntitySet, writeEntitySet, addConstraintsDesc), context.hiveConf);
+    Task<DDLWork> addConstraintsTask = TaskFactory.get(new DDLWork(readEntitySet, writeEntitySet, addConstraintsDesc));
     tasks.add(addConstraintsTask);
     context.log.debug("Added add constrains task : {}:{}", addConstraintsTask.getId(), actualTblName);
     updatedMetadata.set(context.dmd.getEventTo().toString(), actualDbName, actualTblName, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java
index 00ce977..6c6ee02 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java
@@ -80,7 +80,7 @@ public class AlterDatabaseHandler extends AbstractMessageHandler {
       }
 
       Task<DDLWork> alterDbTask = TaskFactory.get(
-          new DDLWork(readEntitySet, writeEntitySet, alterDbDesc), context.hiveConf);
+          new DDLWork(readEntitySet, writeEntitySet, alterDbDesc));
       context.log.debug("Added alter database task : {}:{}",
               alterDbTask.getId(), actualDbName);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateDatabaseHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateDatabaseHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateDatabaseHandler.java
index f8d8d1a..0dc72e0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateDatabaseHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateDatabaseHandler.java
@@ -61,12 +61,12 @@ public class CreateDatabaseHandler extends AbstractMessageHandler {
         new CreateDatabaseDesc(destinationDBName, db.getDescription(), null, true);
     createDatabaseDesc.setDatabaseProperties(db.getParameters());
     Task<DDLWork> createDBTask = TaskFactory.get(
-        new DDLWork(new HashSet<>(), new HashSet<>(), createDatabaseDesc), context.hiveConf);
+        new DDLWork(new HashSet<>(), new HashSet<>(), createDatabaseDesc));
     if (!db.getParameters().isEmpty()) {
       AlterDatabaseDesc alterDbDesc = new AlterDatabaseDesc(destinationDBName, db.getParameters(),
           context.eventOnlyReplicationSpec());
       Task<DDLWork> alterDbProperties = TaskFactory
-          .get(new DDLWork(new HashSet<>(), new HashSet<>(), alterDbDesc), context.hiveConf);
+          .get(new DDLWork(new HashSet<>(), new HashSet<>(), alterDbDesc));
       createDBTask.addDependentTask(alterDbProperties);
     }
     if (StringUtils.isNotEmpty(db.getOwnerName())) {
@@ -74,7 +74,7 @@ public class CreateDatabaseHandler extends AbstractMessageHandler {
           new PrincipalDesc(db.getOwnerName(), db.getOwnerType()),
           context.eventOnlyReplicationSpec());
       Task<DDLWork> alterDbTask = TaskFactory
-          .get(new DDLWork(new HashSet<>(), new HashSet<>(), alterDbOwner), context.hiveConf);
+          .get(new DDLWork(new HashSet<>(), new HashSet<>(), alterDbOwner));
       createDBTask.addDependentTask(alterDbTask);
     }
     updatedMetadata

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
index caf6f3f..77c2dd2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CreateFunctionHandler.java
@@ -65,8 +65,7 @@ public class CreateFunctionHandler extends AbstractMessageHandler {
 
       context.log.debug("Loading function desc : {}", descToLoad.toString());
       Task<FunctionWork> createTask = TaskFactory.get(
-          new FunctionWork(descToLoad), context.hiveConf
-      );
+          new FunctionWork(descToLoad));
       context.log.debug("Added create function task : {}:{},{}", createTask.getId(),
           descToLoad.getFunctionName(), descToLoad.getClassName());
       // This null check is specifically done as the same class is used to handle both incremental and
@@ -93,7 +92,7 @@ public class CreateFunctionHandler extends AbstractMessageHandler {
          *  which should only happen when the last task is finished, at which point the child of the barrier task is picked up.
          */
         Task<? extends Serializable> barrierTask =
-            TaskFactory.get(new DependencyCollectionWork(), context.hiveConf);
+            TaskFactory.get(new DependencyCollectionWork());
         builder.replCopyTasks.forEach(t -> t.addDependentTask(barrierTask));
         barrierTask.addDependentTask(createTask);
         return builder.replCopyTasks;

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropConstraintHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropConstraintHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropConstraintHandler.java
index d9d185b..7c7ce1d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropConstraintHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropConstraintHandler.java
@@ -39,7 +39,7 @@ public class DropConstraintHandler extends AbstractMessageHandler {
 
     AlterTableDesc dropConstraintsDesc = new AlterTableDesc(actualDbName + "." + actualTblName, constraintName,
         context.eventOnlyReplicationSpec());
-    Task<DDLWork> dropConstraintsTask = TaskFactory.get(new DDLWork(readEntitySet, writeEntitySet, dropConstraintsDesc), context.hiveConf);
+    Task<DDLWork> dropConstraintsTask = TaskFactory.get(new DDLWork(readEntitySet, writeEntitySet, dropConstraintsDesc));
     context.log.debug("Added drop constrain task : {}:{}", dropConstraintsTask.getId(), actualTblName);
     updatedMetadata.set(context.dmd.getEventTo().toString(), actualDbName, actualTblName, null);
     return Collections.singletonList(dropConstraintsTask);    

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropDatabaseHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropDatabaseHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropDatabaseHandler.java
index 8b11a9e..363f08c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropDatabaseHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropDatabaseHandler.java
@@ -40,7 +40,7 @@ public class DropDatabaseHandler extends AbstractMessageHandler {
         new DropDatabaseDesc(actualDbName, true, context.eventOnlyReplicationSpec());
     Task<? extends Serializable> dropDBTask =
         TaskFactory
-            .get(new DDLWork(new HashSet<>(), new HashSet<>(), desc), context.hiveConf);
+            .get(new DDLWork(new HashSet<>(), new HashSet<>(), desc));
     context.log.info(
         "Added drop database task : {}:{}", dropDBTask.getId(), desc.getDatabaseName());
     updatedMetadata.set(context.dmd.getEventTo().toString(), actualDbName, null, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropFunctionHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropFunctionHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropFunctionHandler.java
index fee2bb5..1fc7e13 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropFunctionHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropFunctionHandler.java
@@ -39,7 +39,7 @@ public class DropFunctionHandler extends AbstractMessageHandler {
         FunctionUtils.qualifyFunctionName(msg.getFunctionName(), actualDbName);
     DropFunctionDesc desc = new DropFunctionDesc(
             qualifiedFunctionName, false, context.eventOnlyReplicationSpec());
-    Task<FunctionWork> dropFunctionTask = TaskFactory.get(new FunctionWork(desc), context.hiveConf);
+    Task<FunctionWork> dropFunctionTask = TaskFactory.get(new FunctionWork(desc));
     context.log.debug(
         "Added drop function task : {}:{}", dropFunctionTask.getId(), desc.getFunctionName()
     );

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropPartitionHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropPartitionHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropPartitionHandler.java
index a88916e..1a28eec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropPartitionHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropPartitionHandler.java
@@ -53,8 +53,7 @@ public class DropPartitionHandler extends AbstractMessageHandler {
         DropTableDesc dropPtnDesc = new DropTableDesc(actualDbName + "." + actualTblName,
             partSpecs, null, true, context.eventOnlyReplicationSpec());
         Task<DDLWork> dropPtnTask = TaskFactory.get(
-            new DDLWork(readEntitySet, writeEntitySet, dropPtnDesc),
-            context.hiveConf
+            new DDLWork(readEntitySet, writeEntitySet, dropPtnDesc)
         );
         context.log.debug("Added drop ptn task : {}:{},{}", dropPtnTask.getId(),
             dropPtnDesc.getTableName(), msg.getPartitions());

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropTableHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropTableHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropTableHandler.java
index 4d400f4..3e567e9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropTableHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/DropTableHandler.java
@@ -40,8 +40,7 @@ public class DropTableHandler extends AbstractMessageHandler {
         null, true, true, context.eventOnlyReplicationSpec(), false
     );
     Task<DDLWork> dropTableTask = TaskFactory.get(
-        new DDLWork(readEntitySet, writeEntitySet, dropTableDesc),
-        context.hiveConf
+        new DDLWork(readEntitySet, writeEntitySet, dropTableDesc)
     );
     context.log.debug(
         "Added drop tbl task : {}:{}", dropTableTask.getId(), dropTableDesc.getTableName()

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/RenamePartitionHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/RenamePartitionHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/RenamePartitionHandler.java
index 43f2cbc..396b7ee 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/RenamePartitionHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/RenamePartitionHandler.java
@@ -62,7 +62,7 @@ public class RenamePartitionHandler extends AbstractMessageHandler {
     RenamePartitionDesc renamePtnDesc = new RenamePartitionDesc(
             tableName, oldPartSpec, newPartSpec, context.eventOnlyReplicationSpec());
     Task<DDLWork> renamePtnTask = TaskFactory.get(
-        new DDLWork(readEntitySet, writeEntitySet, renamePtnDesc), context.hiveConf);
+        new DDLWork(readEntitySet, writeEntitySet, renamePtnDesc));
     context.log.debug("Added rename ptn task : {}:{}->{}",
                       renamePtnTask.getId(), oldPartSpec, newPartSpec);
     updatedMetadata.set(context.dmd.getEventTo().toString(), actualDbName, actualTblName, newPartSpec);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/RenameTableHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/RenameTableHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/RenameTableHandler.java
index 83433d7..98bf625 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/RenameTableHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/RenameTableHandler.java
@@ -61,7 +61,7 @@ public class RenameTableHandler extends AbstractMessageHandler {
       AlterTableDesc renameTableDesc = new AlterTableDesc(
               oldName, newName, false, context.eventOnlyReplicationSpec());
       Task<DDLWork> renameTableTask = TaskFactory.get(
-          new DDLWork(readEntitySet, writeEntitySet, renameTableDesc), context.hiveConf);
+          new DDLWork(readEntitySet, writeEntitySet, renameTableDesc));
       context.log.debug("Added rename table task : {}:{}->{}",
                         renameTableTask.getId(), oldName, newName);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/TruncatePartitionHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/TruncatePartitionHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/TruncatePartitionHandler.java
index b983f95..1e7fa10 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/TruncatePartitionHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/TruncatePartitionHandler.java
@@ -58,8 +58,7 @@ public class TruncatePartitionHandler extends AbstractMessageHandler {
             actualDbName + "." + actualTblName, partSpec,
             context.eventOnlyReplicationSpec());
     Task<DDLWork> truncatePtnTask = TaskFactory.get(
-            new DDLWork(readEntitySet, writeEntitySet, truncateTableDesc),
-            context.hiveConf);
+        new DDLWork(readEntitySet, writeEntitySet, truncateTableDesc));
     context.log.debug("Added truncate ptn task : {}:{}", truncatePtnTask.getId(),
         truncateTableDesc.getTableName());
     updatedMetadata.set(context.dmd.getEventTo().toString(), actualDbName, actualTblName, partSpec);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/TruncateTableHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/TruncateTableHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/TruncateTableHandler.java
index c6d7739..bdef67f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/TruncateTableHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/TruncateTableHandler.java
@@ -39,8 +39,7 @@ public class TruncateTableHandler extends AbstractMessageHandler {
             actualDbName + "." + actualTblName,
             null, context.eventOnlyReplicationSpec());
     Task<DDLWork> truncateTableTask = TaskFactory.get(
-        new DDLWork(readEntitySet, writeEntitySet, truncateTableDesc),
-        context.hiveConf);
+        new DDLWork(readEntitySet, writeEntitySet, truncateTableDesc));
 
     context.log.debug("Added truncate tbl task : {}:{}", truncateTableTask.getId(),
         truncateTableDesc.getTableName());

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
index c9bb6a5..ede9abc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
@@ -179,7 +179,7 @@ public class GenSparkProcContext implements NodeProcessorCtx {
     this.linkChildOpWithDummyOp = new LinkedHashMap<Operator<?>, List<Operator<?>>>();
     this.dependencyTask = conf.getBoolVar(
         HiveConf.ConfVars.HIVE_MULTI_INSERT_MOVE_TASKS_SHARE_DEPENDENCIES)
-        ? (DependencyCollectionTask) TaskFactory.get(new DependencyCollectionWork(), conf)
+            ? (DependencyCollectionTask) TaskFactory.get(new DependencyCollectionWork())
         : null;
     this.unionWorkMap = new LinkedHashMap<Operator<?>, BaseWork>();
     this.currentUnionOperators = new LinkedList<UnionOperator>();

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java
index fd7bd3d..e81d6f3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java
@@ -111,7 +111,7 @@ public class SparkProcessAnalyzeTable implements NodeProcessor {
           PrunedPartitionList partList = new PrunedPartitionList(table, confirmedParts, partCols, false);
           statWork.addInputPartitions(partList.getPartitions());
         }
-        Task<StatsWork> snjTask = TaskFactory.get(statWork, parseContext.getConf());
+        Task<StatsWork> snjTask = TaskFactory.get(statWork);
         snjTask.setParentTasks(null);
         context.rootTasks.remove(context.currentTask);
         context.rootTasks.add(snjTask);
@@ -127,7 +127,7 @@ public class SparkProcessAnalyzeTable implements NodeProcessor {
         StatsWork columnStatsWork = new StatsWork(table, basicStatsWork, parseContext.getConf());
         columnStatsWork.collectStatsFromAggregator(tableScan.getConf());
         columnStatsWork.setSourceTask(context.currentTask);
-        Task<StatsWork> statsTask = TaskFactory.get(columnStatsWork, parseContext.getConf());
+        Task<StatsWork> statsTask = TaskFactory.get(columnStatsWork);
         context.currentTask.addDependentTask(statsTask);
 
         // ANALYZE TABLE T [PARTITION (...)] COMPUTE STATISTICS noscan;

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
index a4e21c1..7a955bc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
@@ -321,9 +321,9 @@ public class ImportTableDesc {
       HiveConf conf) {
     switch (getDescType()) {
     case TABLE:
-      return TaskFactory.get(new DDLWork(inputs, outputs, createTblDesc), conf, true);
+        return TaskFactory.get(new DDLWork(inputs, outputs, createTblDesc), conf);
     case VIEW:
-      return TaskFactory.get(new DDLWork(inputs, outputs, createViewDesc), conf, true);
+        return TaskFactory.get(new DDLWork(inputs, outputs, createViewDesc), conf);
     }
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
index 788ac52..fdc268c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
@@ -792,7 +792,7 @@ public class TestUtilities {
   }
 
   private Task<? extends Serializable> getDependencyCollectionTask(){
-    return TaskFactory.get(new DependencyCollectionWork(), new HiveConf());
+    return TaskFactory.get(new DependencyCollectionWork());
   }
 
   /**
@@ -911,7 +911,7 @@ public class TestUtilities {
   }
 
   private static Task<MapredWork> getMapredWork() {
-    return TaskFactory.get(MapredWork.class, new HiveConf());
+    return TaskFactory.get(MapredWork.class);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/test/org/apache/hadoop/hive/ql/exec/repl/bootstrap/AddDependencyToLeavesTest.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/repl/bootstrap/AddDependencyToLeavesTest.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/repl/bootstrap/AddDependencyToLeavesTest.java
index aa24d29..309debe 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/repl/bootstrap/AddDependencyToLeavesTest.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/repl/bootstrap/AddDependencyToLeavesTest.java
@@ -46,11 +46,11 @@ public class AddDependencyToLeavesTest {
   @Test
   public void shouldNotSkipIntermediateDependencyCollectionTasks() {
     Task<DependencyCollectionWork> collectionWorkTaskOne =
-        TaskFactory.get(new DependencyCollectionWork(), hiveConf);
+        TaskFactory.get(new DependencyCollectionWork());
     Task<DependencyCollectionWork> collectionWorkTaskTwo =
-        TaskFactory.get(new DependencyCollectionWork(), hiveConf);
+        TaskFactory.get(new DependencyCollectionWork());
     Task<DependencyCollectionWork> collectionWorkTaskThree =
-        TaskFactory.get(new DependencyCollectionWork(), hiveConf);
+        TaskFactory.get(new DependencyCollectionWork());
 
     @SuppressWarnings("unchecked") Task<? extends Serializable> rootTask = mock(Task.class);
     when(rootTask.getDependentTasks())


[2/2] hive git commit: HIVE-18716: Delete unnecessary parameters from TaskFactory (Gergely Hajós via Zoltan Haindrich, Zoltan Haindrich)

Posted by kg...@apache.org.
HIVE-18716: Delete unnecessary parameters from TaskFactory (Gergely Hajós via Zoltan Haindrich, Zoltan Haindrich)

Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>


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

Branch: refs/heads/master
Commit: 94152c9977fa98c294d5062325ee721c09f549e0
Parents: d2d50e6
Author: Gergely Hajós <ro...@gmail.com>
Authored: Mon Mar 19 10:45:58 2018 +0100
Committer: Zoltan Haindrich <ki...@rxd.hu>
Committed: Mon Mar 19 10:45:58 2018 +0100

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   2 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   2 +-
 .../hadoop/hive/ql/exec/ReplCopyTask.java       |   4 +-
 .../apache/hadoop/hive/ql/exec/TaskFactory.java |  37 ++---
 .../ql/exec/repl/bootstrap/ReplLoadTask.java    |   4 +-
 .../exec/repl/bootstrap/load/LoadDatabase.java  |   6 +-
 .../exec/repl/bootstrap/load/LoadFunction.java  |   2 +-
 .../bootstrap/load/table/LoadPartitions.java    |  10 +-
 .../repl/bootstrap/load/table/LoadTable.java    |   4 +-
 .../hive/ql/exec/spark/SparkUtilities.java      |   4 +-
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |   2 +-
 .../hive/ql/optimizer/GenMRProcContext.java     |   2 +-
 .../hive/ql/optimizer/GenMRTableScan1.java      |   6 +-
 .../hadoop/hive/ql/optimizer/GenMRUnion1.java   |   2 +-
 .../hive/ql/optimizer/GenMapRedUtils.java       |  15 +-
 .../hive/ql/optimizer/SimpleFetchOptimizer.java |   2 +-
 .../hive/ql/optimizer/StatsOptimizer.java       |   2 +-
 .../physical/CommonJoinTaskDispatcher.java      |   8 +-
 .../physical/GenMRSkewJoinProcessor.java        |   4 +-
 .../physical/GenSparkSkewJoinProcessor.java     |   4 +-
 .../ql/optimizer/physical/MapJoinResolver.java  |   3 +-
 .../physical/SortMergeJoinTaskDispatcher.java   |   8 +-
 .../physical/SparkMapJoinResolver.java          |   2 +-
 .../spark/SparkSkewJoinProcFactory.java         |   2 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |   2 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      | 160 +++++++++----------
 .../parse/ExplainSQRewriteSemanticAnalyzer.java |   2 +-
 .../hive/ql/parse/ExplainSemanticAnalyzer.java  |   2 +-
 .../hive/ql/parse/ExportSemanticAnalyzer.java   |   2 +-
 .../hive/ql/parse/FunctionSemanticAnalyzer.java |   6 +-
 .../hadoop/hive/ql/parse/GenTezProcContext.java |   4 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |  27 ++--
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |   4 +-
 .../hive/ql/parse/MacroSemanticAnalyzer.java    |   4 +-
 .../hive/ql/parse/ProcessAnalyzeTable.java      |   4 +-
 .../ql/parse/ReplicationSemanticAnalyzer.java   |  18 +--
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  12 +-
 .../hadoop/hive/ql/parse/TaskCompiler.java      |  16 +-
 .../HiveAuthorizationTaskFactoryImpl.java       |  27 ++--
 .../repl/load/message/AddForeignKeyHandler.java |   2 +-
 .../message/AddNotNullConstraintHandler.java    |   2 +-
 .../repl/load/message/AddPrimaryKeyHandler.java |   2 +-
 .../message/AddUniqueConstraintHandler.java     |   2 +-
 .../repl/load/message/AlterDatabaseHandler.java |   2 +-
 .../load/message/CreateDatabaseHandler.java     |   6 +-
 .../load/message/CreateFunctionHandler.java     |   5 +-
 .../load/message/DropConstraintHandler.java     |   2 +-
 .../repl/load/message/DropDatabaseHandler.java  |   2 +-
 .../repl/load/message/DropFunctionHandler.java  |   2 +-
 .../repl/load/message/DropPartitionHandler.java |   3 +-
 .../repl/load/message/DropTableHandler.java     |   3 +-
 .../load/message/RenamePartitionHandler.java    |   2 +-
 .../repl/load/message/RenameTableHandler.java   |   2 +-
 .../load/message/TruncatePartitionHandler.java  |   3 +-
 .../repl/load/message/TruncateTableHandler.java |   3 +-
 .../ql/parse/spark/GenSparkProcContext.java     |   2 +-
 .../parse/spark/SparkProcessAnalyzeTable.java   |   4 +-
 .../hadoop/hive/ql/plan/ImportTableDesc.java    |   4 +-
 .../hadoop/hive/ql/exec/TestUtilities.java      |   4 +-
 .../bootstrap/AddDependencyToLeavesTest.java    |   6 +-
 60 files changed, 229 insertions(+), 261 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index eefcaea..75f928b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -1854,7 +1854,7 @@ public class Driver implements IDriver {
 
   private void useFetchFromCache(CacheEntry cacheEntry) {
     // Change query FetchTask to use new location specified in results cache.
-    FetchTask fetchTaskFromCache = (FetchTask) TaskFactory.get(cacheEntry.getFetchWork(), conf);
+    FetchTask fetchTaskFromCache = (FetchTask) TaskFactory.get(cacheEntry.getFetchWork());
     fetchTaskFromCache.initialize(queryState, plan, null, ctx.getOpContext());
     plan.setFetchTask(fetchTaskFromCache);
     cacheUsage = new CacheUsage(CacheUsage.CacheStatus.QUERY_USING_CACHE, cacheEntry);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 4fc0a93..4eafcde 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -4358,7 +4358,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     MoveWork mw = new MoveWork(null, null, null, null, false);
     mw.setMultiFilesDesc(new LoadMultiFilesDesc(srcs, tgts, true, null, null));
     ImportCommitWork icw = new ImportCommitWork(tbl.getDbName(), tbl.getTableName(), mmWriteId, stmtId);
-    Task<?> mv = TaskFactory.get(mw, conf), ic = TaskFactory.get(icw, conf);
+    Task<?> mv = TaskFactory.get(mw), ic = TaskFactory.get(icw);
     mv.addDependentTask(ic);
     return Lists.<Task<?>>newArrayList(mv);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
index 7ec490b..1cad579 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
@@ -236,10 +236,10 @@ public class ReplCopyTask extends Task<ReplCopyWork> implements Serializable {
         String distCpDoAsUser = conf.getVar(HiveConf.ConfVars.HIVE_DISTCP_DOAS_USER);
         rcwork.setDistCpDoAsUser(distCpDoAsUser);
       }
-      copyTask = TaskFactory.get(rcwork, conf, true);
+      copyTask = TaskFactory.get(rcwork, conf);
     } else {
       LOG.debug("ReplCopyTask:\tcwork");
-      copyTask = TaskFactory.get(new CopyWork(srcPath, dstPath, false), conf, true);
+      copyTask = TaskFactory.get(new CopyWork(srcPath, dstPath, false), conf);
     }
     return copyTask;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java
index d049c37..ccfd4cb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java
@@ -53,6 +53,8 @@ import org.apache.hadoop.hive.ql.plan.ReplCopyWork;
 import org.apache.hadoop.hive.ql.plan.SparkWork;
 import org.apache.hadoop.hive.ql.plan.TezWork;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * TaskFactory implementation.
  **/
@@ -131,8 +133,8 @@ public final class TaskFactory {
   }
 
   @SuppressWarnings("unchecked")
-  public static <T extends Serializable> Task<T> get(Class<T> workClass,
-      HiveConf conf) {
+  @VisibleForTesting
+  static <T extends Serializable> Task<T> get(Class<T> workClass) {
 
     for (TaskTuple<? extends Serializable> t : taskvec) {
       if (t.workClass == workClass) {
@@ -149,36 +151,24 @@ public final class TaskFactory {
     throw new RuntimeException("No task for work class " + workClass.getName());
   }
 
-  @SafeVarargs
-  public static <T extends Serializable> Task<T> get(T work, HiveConf conf, boolean setConf,
-                                                     Task<? extends Serializable>... tasklist) {
-    Task<T> ret = get((Class<T>) work.getClass(), conf);
+  public static <T extends Serializable> Task<T> get(T work, HiveConf conf) {
+    @SuppressWarnings("unchecked")
+    Task<T> ret = get((Class<T>) work.getClass());
     ret.setWork(work);
-    if (setConf && (null != conf)) {
+    if (null != conf) {
       ret.setConf(conf);
     }
-    if (tasklist.length == 0) {
-      return (ret);
-    }
-
-    ArrayList<Task<? extends Serializable>> clist = new ArrayList<Task<? extends Serializable>>();
-    for (Task<? extends Serializable> tsk : tasklist) {
-      clist.add(tsk);
-    }
-    ret.setChildTasks(clist);
-    return (ret);
+    return ret;
   }
 
-  @SafeVarargs
-  public static <T extends Serializable> Task<T> get(T work, HiveConf conf,
-      Task<? extends Serializable>... tasklist) {
-    return get(work, conf, false, tasklist);
+  public static <T extends Serializable> Task<T> get(T work) {
+    return get(work, null);
   }
 
+  @SafeVarargs
   public static <T extends Serializable> Task<T> getAndMakeChild(T work,
       HiveConf conf, Task<? extends Serializable>... tasklist) {
-    Task<T> ret = get((Class<T>) work.getClass(), conf);
-    ret.setWork(work);
+    Task<T> ret = get(work);
     if (tasklist.length == 0) {
       return (ret);
     }
@@ -189,6 +179,7 @@ public final class TaskFactory {
   }
 
 
+  @SafeVarargs
   public static  void makeChild(Task<?> ret,
       Task<? extends Serializable>... tasklist) {
     // Add the new task as child of each of the passed in tasks

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/ReplLoadTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/ReplLoadTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/ReplLoadTask.java
index b8d5c18..6b333d7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/ReplLoadTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/ReplLoadTask.java
@@ -231,7 +231,7 @@ public class ReplLoadTask extends Task<ReplLoadWork> implements Serializable {
                                                   ReplLogger replLogger) throws SemanticException {
     Database dbInMetadata = work.databaseEvent(context.hiveConf).dbInMetadata(work.dbNameToLoadIn);
     ReplStateLogWork replLogWork = new ReplStateLogWork(replLogger, dbInMetadata.getParameters());
-    Task<ReplStateLogWork> replLogTask = TaskFactory.get(replLogWork, conf);
+    Task<ReplStateLogWork> replLogTask = TaskFactory.get(replLogWork);
     if (scope.rootTasks.isEmpty()) {
       scope.rootTasks.add(replLogTask);
     } else {
@@ -298,7 +298,7 @@ public class ReplLoadTask extends Task<ReplLoadWork> implements Serializable {
     use loadTask as dependencyCollection
    */
     if (shouldCreateAnotherLoadTask) {
-      Task<ReplLoadWork> loadTask = TaskFactory.get(work, conf, true);
+      Task<ReplLoadWork> loadTask = TaskFactory.get(work, conf);
       DAGTraversal.traverse(rootTasks, new AddDependencyToLeaves(loadTask));
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadDatabase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadDatabase.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadDatabase.java
index 9191d22..537c5e9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadDatabase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadDatabase.java
@@ -91,14 +91,14 @@ public class LoadDatabase {
     // db.
     // TODO: we might revisit this in create-drop-recreate cases, needs some thinking on.
     DDLWork work = new DDLWork(new HashSet<>(), new HashSet<>(), createDbDesc);
-    return TaskFactory.get(work, context.hiveConf, true);
+    return TaskFactory.get(work, context.hiveConf);
   }
 
   private static Task<? extends Serializable> alterDbTask(Database dbObj, HiveConf hiveConf) {
     AlterDatabaseDesc alterDbDesc =
         new AlterDatabaseDesc(dbObj.getName(), dbObj.getParameters(), null);
     DDLWork work = new DDLWork(new HashSet<>(), new HashSet<>(), alterDbDesc);
-    return TaskFactory.get(work, hiveConf, true);
+    return TaskFactory.get(work, hiveConf);
   }
 
   private Task<? extends Serializable> setOwnerInfoTask(Database dbObj) {
@@ -106,7 +106,7 @@ public class LoadDatabase {
             new PrincipalDesc(dbObj.getOwnerName(), dbObj.getOwnerType()),
             null);
     DDLWork work = new DDLWork(new HashSet<>(), new HashSet<>(), alterDbDesc);
-    return TaskFactory.get(work, context.hiveConf, true);
+    return TaskFactory.get(work, context.hiveConf);
   }
 
   private boolean existEmptyDb(String dbName) throws InvalidOperationException, HiveException {

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadFunction.java
index 2e9c37f..bc7d0ad 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadFunction.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/LoadFunction.java
@@ -61,7 +61,7 @@ public class LoadFunction {
   private void createFunctionReplLogTask(List<Task<? extends Serializable>> functionTasks,
                                          String functionName) {
     ReplStateLogWork replLogWork = new ReplStateLogWork(replLogger, functionName);
-    Task<ReplStateLogWork> replLogTask = TaskFactory.get(replLogWork, context.hiveConf);
+    Task<ReplStateLogWork> replLogTask = TaskFactory.get(replLogWork);
     DAGTraversal.traverse(functionTasks, new AddDependencyToLeaves(replLogTask));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
index eb9b1e5..06adc64 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
@@ -113,7 +113,7 @@ public class LoadPartitions {
   private void createTableReplLogTask() throws SemanticException {
     ReplStateLogWork replLogWork = new ReplStateLogWork(replLogger,
                                             tableDesc.getTableName(), tableDesc.tableType());
-    Task<ReplStateLogWork> replLogTask = TaskFactory.get(replLogWork, context.hiveConf, true);
+    Task<ReplStateLogWork> replLogTask = TaskFactory.get(replLogWork, context.hiveConf);
 
     if (tracker.tasks().isEmpty()) {
       tracker.addTask(replLogTask);
@@ -224,8 +224,7 @@ public class LoadPartitions {
 
     Task<?> addPartTask = TaskFactory.get(
         new DDLWork(new HashSet<>(), new HashSet<>(), addPartitionDesc),
-        context.hiveConf,
-        true
+        context.hiveConf
     );
 
     Task<?> movePartitionTask = movePartitionTask(table, partSpec, tmpPath);
@@ -249,7 +248,7 @@ public class LoadPartitions {
     );
     loadTableWork.setInheritTableSpecs(false);
     MoveWork work = new MoveWork(new HashSet<>(), new HashSet<>(), loadTableWork, null, false);
-    return TaskFactory.get(work, context.hiveConf, true);
+    return TaskFactory.get(work, context.hiveConf);
   }
 
   private Path locationOnReplicaWarehouse(Table table, AddPartitionDesc.OnePartitionDesc partSpec)
@@ -277,8 +276,7 @@ public class LoadPartitions {
     desc.getPartition(0).setLocation(ptn.getLocation()); // use existing location
     return TaskFactory.get(
         new DDLWork(new HashSet<>(), new HashSet<>(), desc),
-        context.hiveConf,
-        true
+        context.hiveConf
     );
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadTable.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadTable.java
index e0721f1..1395027 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadTable.java
@@ -78,7 +78,7 @@ public class LoadTable {
 
   private void createTableReplLogTask(String tableName, TableType tableType) throws SemanticException {
     ReplStateLogWork replLogWork = new ReplStateLogWork(replLogger,tableName, tableType);
-    Task<ReplStateLogWork> replLogTask = TaskFactory.get(replLogWork, context.hiveConf);
+    Task<ReplStateLogWork> replLogTask = TaskFactory.get(replLogWork);
     DAGTraversal.traverse(tracker.tasks(), new AddDependencyToLeaves(replLogTask));
 
     if (tracker.tasks().isEmpty()) {
@@ -234,7 +234,7 @@ public class LoadTable {
     );
     MoveWork moveWork =
         new MoveWork(new HashSet<>(), new HashSet<>(), loadTableWork, null, false);
-    Task<?> loadTableTask = TaskFactory.get(moveWork, context.hiveConf, true);
+    Task<?> loadTableTask = TaskFactory.get(moveWork, context.hiveConf);
     copyTask.addDependentTask(loadTableTask);
     return copyTask;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java
index 6d0dfba..fdc5361 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java
@@ -161,11 +161,11 @@ public class SparkUtilities {
 
   public static SparkTask createSparkTask(HiveConf conf) {
     return (SparkTask) TaskFactory.get(
-        new SparkWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID)), conf);
+        new SparkWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID)));
   }
 
   public static SparkTask createSparkTask(SparkWork work, HiveConf conf) {
-    return (SparkTask) TaskFactory.get(work, conf);
+    return (SparkTask) TaskFactory.get(work);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
index ea00f7a..0ae60b5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
@@ -269,7 +269,7 @@ public class ATSHook implements ExecuteWithHookContext {
                   null// cboInfo
               );
                 @SuppressWarnings("unchecked")
-                ExplainTask explain = (ExplainTask) TaskFactory.get(work, conf);
+                ExplainTask explain = (ExplainTask) TaskFactory.get(work);
                 explain.initialize(queryState, plan, null, null);
                 String query = plan.getQueryStr();
                 JSONObject explainPlan = explain.getJSONPlan(null, work);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java
index 2546cb3..f80395d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java
@@ -439,7 +439,7 @@ public class GenMRProcContext implements NodeProcessorCtx {
     if (dependencyTaskForMultiInsert == null) {
       if (conf.getBoolVar(ConfVars.HIVE_MULTI_INSERT_MOVE_TASKS_SHARE_DEPENDENCIES)) {
         dependencyTaskForMultiInsert =
-            (DependencyCollectionTask) TaskFactory.get(new DependencyCollectionWork(), conf);
+            (DependencyCollectionTask) TaskFactory.get(new DependencyCollectionWork());
       }
     }
     return dependencyTaskForMultiInsert;

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
index c139d23..06b7c93 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
@@ -71,7 +71,7 @@ public class GenMRTableScan1 implements NodeProcessor {
 
     // create a dummy MapReduce task
     MapredWork currWork = GenMapRedUtils.getMapRedWork(parseCtx);
-    MapRedTask currTask = (MapRedTask) TaskFactory.get(currWork, parseCtx.getConf());
+    MapRedTask currTask = (MapRedTask) TaskFactory.get(currWork);
     ctx.setCurrTask(currTask);
     ctx.setCurrTopOp(op);
 
@@ -101,7 +101,7 @@ public class GenMRTableScan1 implements NodeProcessor {
               PrunedPartitionList partList = new PrunedPartitionList(table, confirmedParts, partCols, false);
               statWork.addInputPartitions(partList.getPartitions());
             }
-            Task<StatsWork> snjTask = TaskFactory.get(statWork, parseCtx.getConf());
+            Task<StatsWork> snjTask = TaskFactory.get(statWork);
             ctx.setCurrTask(snjTask);
             ctx.setCurrTopOp(null);
             ctx.getRootTasks().clear();
@@ -118,7 +118,7 @@ public class GenMRTableScan1 implements NodeProcessor {
             columnStatsWork.collectStatsFromAggregator(op.getConf());
 
             columnStatsWork.setSourceTask(currTask);
-            Task<StatsWork> columnStatsTask = TaskFactory.get(columnStatsWork, parseCtx.getConf());
+            Task<StatsWork> columnStatsTask = TaskFactory.get(columnStatsWork);
             currTask.addDependentTask(columnStatsTask);
             if (!ctx.getRootTasks().contains(currTask)) {
               ctx.getRootTasks().add(currTask);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java
index 840a20e..8f01507 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java
@@ -230,7 +230,7 @@ public class GenMRUnion1 implements NodeProcessor {
     GenMRUnionCtx uCtxTask = ctx.getUnionTask(union);
     if (uCtxTask == null) {
       uPlan = GenMapRedUtils.getMapRedWork(parseCtx);
-      uTask = TaskFactory.get(uPlan, parseCtx.getConf());
+      uTask = TaskFactory.get(uPlan);
       uCtxTask = new GenMRUnionCtx(uTask);
       ctx.setUnionTask(union, uCtxTask);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
index 4f396a0..605bb09 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
@@ -409,8 +409,7 @@ public final class GenMapRedUtils {
     Task<? extends Serializable> parentTask = opProcCtx.getCurrTask();
 
     MapredWork childPlan = getMapRedWork(parseCtx);
-    Task<? extends Serializable> childTask = TaskFactory.get(childPlan, parseCtx
-        .getConf());
+    Task<? extends Serializable> childTask = TaskFactory.get(childPlan);
     Operator<? extends OperatorDesc> reducer = cRS.getChildOperators().get(0);
 
     // Add the reducer
@@ -1546,7 +1545,7 @@ public final class GenMapRedUtils {
     columnStatsWork.truncateExisting(truncate);
 
     columnStatsWork.setSourceTask(currTask);
-    Task<? extends Serializable> statsTask = TaskFactory.get(columnStatsWork, hconf);
+    Task<? extends Serializable> statsTask = TaskFactory.get(columnStatsWork);
 
     // subscribe feeds from the MoveTask so that MoveTask can forward the list
     // of dynamic partition list to the StatsTask
@@ -1801,10 +1800,10 @@ public final class GenMapRedUtils {
     // conflicts.
     // TODO: if we are not dealing with concatenate DDL, we should not create a merge+move path
     //       because it should be impossible to get incompatible outputs.
-    Task<? extends Serializable> mergeOnlyMergeTask = TaskFactory.get(mergeWork, conf);
-    Task<? extends Serializable> moveOnlyMoveTask = TaskFactory.get(workForMoveOnlyTask, conf);
-    Task<? extends Serializable> mergeAndMoveMergeTask = TaskFactory.get(mergeWork, conf);
-    Task<? extends Serializable> mergeAndMoveMoveTask = TaskFactory.get(moveWork, conf);
+    Task<? extends Serializable> mergeOnlyMergeTask = TaskFactory.get(mergeWork);
+    Task<? extends Serializable> moveOnlyMoveTask = TaskFactory.get(workForMoveOnlyTask);
+    Task<? extends Serializable> mergeAndMoveMergeTask = TaskFactory.get(mergeWork);
+    Task<? extends Serializable> mergeAndMoveMoveTask = TaskFactory.get(moveWork);
 
     // NOTE! It is necessary merge task is the parent of the move task, and not
     // the other way around, for the proper execution of the execute method of
@@ -1822,7 +1821,7 @@ public final class GenMapRedUtils {
     listTasks.add(mergeOnlyMergeTask);
     listTasks.add(mergeAndMoveMergeTask);
 
-    ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork, conf);
+    ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork);
     cndTsk.setListTasks(listTasks);
 
     // create resolver

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
index 54e6c58..6b46188 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
@@ -143,7 +143,7 @@ public class SimpleFetchOptimizer extends Transform {
     FetchData fetch = checkTree(aggressive, pctx, alias, source);
     if (fetch != null && checkThreshold(fetch, limit, pctx)) {
       FetchWork fetchWork = fetch.convertToWork();
-      FetchTask fetchTask = (FetchTask) TaskFactory.get(fetchWork, pctx.getConf());
+      FetchTask fetchTask = (FetchTask) TaskFactory.get(fetchWork);
       fetchWork.setSink(fetch.completed(pctx, fetchWork));
       fetchWork.setSource(source);
       fetchWork.setLimit(limit);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
index 0f3b4e0..e490627 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
@@ -758,7 +758,7 @@ public class StatsOptimizer extends Transform {
           StandardStructObjectInspector sOI = ObjectInspectorFactory.
               getStandardStructObjectInspector(colNames, ois);
           fWork = new FetchWork(allRows, sOI);
-          fTask = (FetchTask)TaskFactory.get(fWork, pctx.getConf());
+          fTask = (FetchTask) TaskFactory.get(fWork);
           pctx.setFetchTask(fTask);
         }
         fWork.setLimit(fWork.getRowsComputedUsingStats().size());

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java
index 533a488..a52e5e6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java
@@ -175,8 +175,7 @@ public class CommonJoinTaskDispatcher extends AbstractJoinTaskDispatcher impleme
   private MapRedTask convertTaskToMapJoinTask(MapredWork newWork, int bigTablePosition)
       throws UnsupportedEncodingException, SemanticException {
     // create a mapred task for this work
-    MapRedTask newTask = (MapRedTask) TaskFactory.get(newWork, physicalContext
-        .getParseContext().getConf());
+    MapRedTask newTask = (MapRedTask) TaskFactory.get(newWork);
     JoinOperator newJoinOp = getJoinOp(newTask);
     // optimize this newWork given the big table position
     MapJoinProcessor.genMapJoinOpAndLocalWork(physicalContext.getParseContext().getConf(),
@@ -397,9 +396,6 @@ public class CommonJoinTaskDispatcher extends AbstractJoinTaskDispatcher impleme
     HashMap<Path, ArrayList<String>> pathToAliases = currWork.getPathToAliases();
     Map<String, Operator<? extends OperatorDesc>> aliasToWork = currWork.getAliasToWork();
 
-    // get parseCtx for this Join Operator
-    ParseContext parseCtx = physicalContext.getParseContext();
-
     // start to generate multiple map join tasks
     JoinDesc joinDesc = joinOp.getConf();
 
@@ -528,7 +524,7 @@ public class CommonJoinTaskDispatcher extends AbstractJoinTaskDispatcher impleme
 
     // create conditional task and insert conditional task into task tree
     ConditionalWork cndWork = new ConditionalWork(listWorks);
-    ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork, parseCtx.getConf());
+    ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork);
     cndTsk.setListTasks(listTasks);
 
     // set resolver and resolver context

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java
index 82af9af..f7cedfe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java
@@ -331,7 +331,7 @@ public final class GenMRSkewJoinProcessor {
       MapredWork w = new MapredWork();
       w.setMapWork(newPlan);
 
-      Task<? extends Serializable> skewJoinMapJoinTask = TaskFactory.get(w, jc);
+      Task<? extends Serializable> skewJoinMapJoinTask = TaskFactory.get(w);
       skewJoinMapJoinTask.setFetchSource(currTask.isFetchSource());
 
       bigKeysDirToTaskMap.put(bigKeyDirPath, skewJoinMapJoinTask);
@@ -354,7 +354,7 @@ public final class GenMRSkewJoinProcessor {
         new ConditionalResolverSkewJoinCtx(bigKeysDirToTaskMap, children);
 
     ConditionalWork cndWork = new ConditionalWork(listWorks);
-    ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork, parseCtx.getConf());
+    ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork);
     cndTsk.setListTasks(listTasks);
     cndTsk.setResolver(new ConditionalResolverSkewJoin());
     cndTsk.setResolverCtx(context);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java
index 76fc5d1..7f7f49b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java
@@ -223,7 +223,7 @@ public class GenSparkSkewJoinProcessor {
       HiveConf hiveConf = new HiveConf(parseCtx.getConf(),
           GenSparkSkewJoinProcessor.class);
       SparkWork sparkWork = new SparkWork(parseCtx.getConf().getVar(HiveConf.ConfVars.HIVEQUERYID));
-      Task<? extends Serializable> skewJoinMapJoinTask = TaskFactory.get(sparkWork, hiveConf);
+      Task<? extends Serializable> skewJoinMapJoinTask = TaskFactory.get(sparkWork);
       skewJoinMapJoinTask.setFetchSource(currTask.isFetchSource());
 
       // create N TableScans
@@ -346,7 +346,7 @@ public class GenSparkSkewJoinProcessor {
         new ConditionalResolverSkewJoin.ConditionalResolverSkewJoinCtx(bigKeysDirToTaskMap, children);
 
     ConditionalWork cndWork = new ConditionalWork(listWorks);
-    ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork, parseCtx.getConf());
+    ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork);
     cndTsk.setListTasks(listTasks);
     cndTsk.setResolver(new ConditionalResolverSkewJoin());
     cndTsk.setResolverCtx(context);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java
index 576a5bc..eac4768 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java
@@ -110,8 +110,7 @@ public class MapJoinResolver implements PhysicalPlanResolver {
           ctx.getMRTmpPath(), currTask.getId()));
         // create a task for this local work; right now, this local work is shared
         // by the original MapredTask and this new generated MapredLocalTask.
-        MapredLocalTask localTask = (MapredLocalTask) TaskFactory.get(localwork, physicalContext
-            .getParseContext().getConf());
+        MapredLocalTask localTask = (MapredLocalTask) TaskFactory.get(localwork);
 
         // set the backup task from curr task
         localTask.setBackupTask(currTask.getBackupTask());

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java
index f56ffd9..d077b19 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java
@@ -168,8 +168,7 @@ public class SortMergeJoinTaskDispatcher extends AbstractJoinTaskDispatcher impl
     // deep copy a new mapred work
     MapredWork newWork = SerializationUtilities.clonePlan(origWork);
     // create a mapred task for this work
-    MapRedTask newTask = (MapRedTask) TaskFactory.get(newWork, physicalContext
-        .getParseContext().getConf());
+    MapRedTask newTask = (MapRedTask) TaskFactory.get(newWork);
     // generate the map join operator; already checked the map join
     MapJoinOperator newMapJoinOp =
         getMapJoinOperator(newTask, newWork, smbJoinOp, bigTablePosition);
@@ -238,9 +237,6 @@ public class SortMergeJoinTaskDispatcher extends AbstractJoinTaskDispatcher impl
 
     currTask.setTaskTag(Task.CONVERTED_SORTMERGEJOIN);
 
-    // get parseCtx for this Join Operator
-    ParseContext parseCtx = physicalContext.getParseContext();
-
     // Convert the work containing to sort-merge join into a work, as if it had a regular join.
     // Note that the operator tree is not changed - is still contains the SMB join, but the
     // plan is changed (aliasToWork etc.) to contain all the paths as if it was a regular join.
@@ -334,7 +330,7 @@ public class SortMergeJoinTaskDispatcher extends AbstractJoinTaskDispatcher impl
 
     // create conditional task and insert conditional task into task tree
     ConditionalWork cndWork = new ConditionalWork(listWorks);
-    ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork, parseCtx.getConf());
+    ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork);
     cndTsk.setListTasks(listTasks);
 
     // set resolver and resolver context

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
index a8ebecd..55b46e5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
@@ -236,7 +236,7 @@ public class SparkMapJoinResolver implements PhysicalPlanResolver {
         return createdTaskMap.get(sparkWork);
       }
       SparkTask resultTask = originalTask.getWork() == sparkWork ?
-          originalTask : (SparkTask) TaskFactory.get(sparkWork, physicalContext.conf);
+          originalTask : (SparkTask) TaskFactory.get(sparkWork);
       if (!dependencyGraph.get(sparkWork).isEmpty()) {
         for (SparkWork parentWork : dependencyGraph.get(sparkWork)) {
           SparkTask parentTask =

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
index a3b5837..a7c18b0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
@@ -170,7 +170,7 @@ public class SparkSkewJoinProcFactory {
           tableScanOp, mapWork, false, tableDesc);
       // insert the new task between current task and its child
       @SuppressWarnings("unchecked")
-      Task<? extends Serializable> newTask = TaskFactory.get(newWork, parseContext.getConf());
+      Task<? extends Serializable> newTask = TaskFactory.get(newWork);
       List<Task<? extends Serializable>> childTasks = currentTask.getChildTasks();
       // must have at most one child
       if (childTasks != null && childTasks.size() > 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index 7a74a60..cc783cc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -2082,7 +2082,7 @@ public abstract class BaseSemanticAnalyzer {
         new FetchWork(ctx.getResFile(), new TableDesc(TextInputFormat.class,
             IgnoreKeyTextOutputFormat.class, prop), -1);
     fetch.setSerializationNullFormat(" ");
-    return (FetchTask) TaskFactory.get(fetch, conf);
+    return (FetchTask) TaskFactory.get(fetch);
   }
 
   protected HiveTxnManager getTxnMgr() {

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index 5761795..1605d7d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -614,7 +614,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       desc = new CacheMetadataDesc(tbl.getDbName(), tbl.getTableName(), tbl.isPartitioned());
       inputs.add(new ReadEntity(tbl));
     }
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private void analyzeAlterTableUpdateStats(ASTNode ast, String tblName, Map<String, String> partSpec)
@@ -649,7 +649,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     ColumnStatsUpdateWork columnStatsUpdateWork =
         new ColumnStatsUpdateWork(partName, mapProp, tbl.getDbName(), tbl.getTableName(), colName, colType);
     ColumnStatsUpdateTask cStatsUpdateTask = (ColumnStatsUpdateTask) TaskFactory
-        .get(columnStatsUpdateWork, conf);
+        .get(columnStatsUpdateWork);
     rootTasks.add(cStatsUpdateTask);
   }
 
@@ -777,7 +777,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
   private void addAlterDbDesc(AlterDatabaseDesc alterDesc) throws SemanticException {
     Database database = getDatabase(alterDesc.getDatabaseName());
     outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL_NO_LOCK));
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterDesc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterDesc)));
   }
 
   private void analyzeAlterDatabaseOwner(ASTNode ast) throws SemanticException {
@@ -857,7 +857,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     AlterTableExchangePartition alterTableExchangePartition =
       new AlterTableExchangePartition(sourceTable, destTable, partSpecs);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-      alterTableExchangePartition), conf));
+        alterTableExchangePartition)));
 
     inputs.add(new ReadEntity(sourceTable));
     outputs.add(new WriteEntity(destTable, WriteType.DDL_SHARED));
@@ -913,7 +913,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     CreateResourcePlanDesc desc = new CreateResourcePlanDesc(
         resourcePlanName, queryParallelism, likeName);
     addServiceOutput();
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private void analyzeShowResourcePlan(ASTNode ast) throws SemanticException {
@@ -927,7 +927,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     ShowResourcePlanDesc showResourcePlanDesc = new ShowResourcePlanDesc(rpName, ctx.getResFile());
     addServiceOutput();
     rootTasks.add(TaskFactory.get(
-        new DDLWork(getInputs(), getOutputs(), showResourcePlanDesc), conf));
+        new DDLWork(getInputs(), getOutputs(), showResourcePlanDesc)));
     setFetchTask(createFetchTask(showResourcePlanDesc.getSchema(rpName)));
   }
 
@@ -946,7 +946,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       AlterResourcePlanDesc desc = new AlterResourcePlanDesc(
           anyRp, null, false, false, true, false);
       addServiceOutput();
-      rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+        rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
       return;
     default: // Continue to handle changes to a specific plan.
     }
@@ -1043,7 +1043,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       desc.setResFile(ctx.getResFile().toString());
     }
     addServiceOutput();
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
     if (validate) {
       setFetchTask(createFetchTask(AlterResourcePlanDesc.getSchema()));
     }
@@ -1057,7 +1057,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     DropResourcePlanDesc desc = new DropResourcePlanDesc(rpName);
     addServiceOutput();
     rootTasks.add(TaskFactory.get(
-        new DDLWork(getInputs(), getOutputs(), desc), conf));
+        new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private void analyzeCreateTrigger(ASTNode ast) throws SemanticException {
@@ -1075,7 +1075,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     CreateWMTriggerDesc desc = new CreateWMTriggerDesc(trigger);
     addServiceOutput();
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private String buildTriggerExpression(ASTNode ast) throws SemanticException {
@@ -1131,7 +1131,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     AlterWMTriggerDesc desc = new AlterWMTriggerDesc(trigger);
     addServiceOutput();
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private void analyzeDropTrigger(ASTNode ast) throws SemanticException {
@@ -1144,7 +1144,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     DropWMTriggerDesc desc = new DropWMTriggerDesc(rpName, triggerName);
     addServiceOutput();
     rootTasks.add(TaskFactory.get(
-        new DDLWork(getInputs(), getOutputs(), desc), conf));
+        new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private void analyzeCreatePool(ASTNode ast) throws SemanticException {
@@ -1188,7 +1188,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     CreateOrAlterWMPoolDesc desc = new CreateOrAlterWMPoolDesc(pool, poolPath, false);
     addServiceOutput();
     rootTasks.add(TaskFactory.get(
-        new DDLWork(getInputs(), getOutputs(), desc), conf));
+        new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private void analyzeAlterPool(ASTNode ast) throws SemanticException {
@@ -1221,7 +1221,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         String triggerName = unescapeIdentifier(param.getText());
         rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
             new CreateOrDropTriggerToPoolMappingDesc(
-                rpName, triggerName, poolPath, drop, isUnmanagedPool)), conf));
+                rpName, triggerName, poolPath, drop, isUnmanagedPool))));
       } else {
         if (isUnmanagedPool) {
           throw new SemanticException("Cannot alter the unmanaged pool");
@@ -1255,7 +1255,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
     if (poolChanges != null) {
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-          new CreateOrAlterWMPoolDesc(poolChanges, poolPath, true)), conf));
+          new CreateOrAlterWMPoolDesc(poolChanges, poolPath, true))));
     }
   }
 
@@ -1269,7 +1269,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     DropWMPoolDesc desc = new DropWMPoolDesc(rpName, poolPath);
     addServiceOutput();
     rootTasks.add(TaskFactory.get(
-        new DDLWork(getInputs(), getOutputs(), desc), conf));
+        new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private void analyzeCreateOrAlterMapping(ASTNode ast, boolean update) throws SemanticException {
@@ -1290,7 +1290,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     CreateOrAlterWMMappingDesc desc = new CreateOrAlterWMMappingDesc(mapping, update);
     addServiceOutput();
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private void analyzeDropMapping(ASTNode ast) throws SemanticException {
@@ -1303,7 +1303,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     DropWMMappingDesc desc = new DropWMMappingDesc(new WMMapping(rpName, entityType, entityName));
     addServiceOutput();
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private void analyzeCreateDatabase(ASTNode ast) throws SemanticException {
@@ -1343,7 +1343,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL_NO_LOCK));
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        createDatabaseDesc), conf));
+        createDatabaseDesc)));
   }
 
   private void analyzeDropDatabase(ASTNode ast) throws SemanticException {
@@ -1389,7 +1389,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     DropDatabaseDesc dropDatabaseDesc = new DropDatabaseDesc(dbName, ifExists, ifCascade,
         new ReplicationSpec());
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropDatabaseDesc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropDatabaseDesc)));
   }
 
   private void analyzeSwitchDatabase(ASTNode ast) throws SemanticException {
@@ -1400,7 +1400,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     inputs.add(dbReadEntity);
     SwitchDatabaseDesc switchDatabaseDesc = new SwitchDatabaseDesc(dbName);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        switchDatabaseDesc), conf));
+        switchDatabaseDesc)));
   }
 
 
@@ -1425,7 +1425,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     boolean ifPurge = (ast.getFirstChildWithType(HiveParser.KW_PURGE) != null);
     DropTableDesc dropTblDesc = new DropTableDesc(tableName, expectedType, ifExists, ifPurge, replicationSpec);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        dropTblDesc), conf));
+        dropTblDesc)));
   }
 
   private void analyzeTruncateTable(ASTNode ast) throws SemanticException {
@@ -1467,7 +1467,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     TruncateTableDesc truncateTblDesc = new TruncateTableDesc(tableName, partSpec, null);
 
     DDLWork ddlWork = new DDLWork(getInputs(), getOutputs(), truncateTblDesc);
-    Task<? extends Serializable> truncateTask = TaskFactory.get(ddlWork, conf);
+    Task<? extends Serializable> truncateTask = TaskFactory.get(ddlWork);
 
     // Is this a truncate column command
     List<String> columnNames = null;
@@ -1581,7 +1581,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         ltd.setLbCtx(lbCtx);
         @SuppressWarnings("unchecked")
         Task<MoveWork> moveTsk =
-            TaskFactory.get(new MoveWork(null, null, ltd, null, false), conf);
+            TaskFactory.get(new MoveWork(null, null, ltd, null, false));
         truncateTask.addDependentTask(moveTsk);
 
         // Recalculate the HDFS stats if auto gather stats is set
@@ -1598,7 +1598,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           basicStatsWork.setClearAggregatorStats(true);
           StatsWork columnStatsWork = new StatsWork(table, basicStatsWork, conf);
 
-          Task<? extends Serializable> statTask = TaskFactory.get(columnStatsWork, conf);
+          Task<? extends Serializable> statTask = TaskFactory.get(columnStatsWork);
           moveTsk.addDependentTask(statTask);
         }
       } catch (HiveException e) {
@@ -1742,7 +1742,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         || mapProp.containsKey(hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES);
     addInputsOutputsAlterTable(tableName, partSpec, alterTblDesc, isPotentialMmSwitch);
 
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTblDesc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTblDesc)));
   }
 
   private void analyzeAlterTableSerdeProps(ASTNode ast, String tableName,
@@ -1758,7 +1758,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     addInputsOutputsAlterTable(tableName, partSpec, alterTblDesc);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
   }
 
   private void analyzeAlterTableSerde(ASTNode ast, String tableName,
@@ -1778,7 +1778,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     addInputsOutputsAlterTable(tableName, partSpec, alterTblDesc);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
   }
 
   private void analyzeAlterTableFileFormat(ASTNode ast, String tableName,
@@ -1797,7 +1797,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     addInputsOutputsAlterTable(tableName, partSpec, alterTblDesc);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
   }
 
   private WriteType determineAlterTableWriteType(Table tab, AlterTableDesc desc, AlterTableTypes op) {
@@ -1922,7 +1922,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     addInputsOutputsAlterTable(tableName, partSpec, alterTblDesc);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
 
   }
 
@@ -2029,7 +2029,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       addInputsOutputsAlterTable(tableName, partSpec, AlterTableTypes.MERGEFILES);
       DDLWork ddlWork = new DDLWork(getInputs(), getOutputs(), mergeDesc);
       ddlWork.setNeedLock(true);
-      Task<? extends Serializable> mergeTask = TaskFactory.get(ddlWork, conf);
+      Task<? extends Serializable> mergeTask = TaskFactory.get(ddlWork);
       TableDesc tblDesc = Utilities.getTableDesc(tblObj);
       Path queryTmpdir = ctx.getExternalTmpPath(newTblPartLoc);
       mergeDesc.setOutputDir(queryTmpdir);
@@ -2038,7 +2038,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           partSpec == null ? new HashMap<>() : partSpec);
       ltd.setLbCtx(lbCtx);
       Task<MoveWork> moveTsk =
-          TaskFactory.get(new MoveWork(null, null, ltd, null, false), conf);
+          TaskFactory.get(new MoveWork(null, null, ltd, null, false));
       mergeTask.addDependentTask(moveTsk);
 
       if (conf.getBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER)) {
@@ -2054,7 +2054,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         basicStatsWork.setClearAggregatorStats(true);
         StatsWork columnStatsWork = new StatsWork(tblObj, basicStatsWork, conf);
 
-        Task<? extends Serializable> statTask = TaskFactory.get(columnStatsWork, conf);
+        Task<? extends Serializable> statTask = TaskFactory.get(columnStatsWork);
         moveTsk.addDependentTask(statTask);
       }
 
@@ -2098,7 +2098,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException("Invalid operation " + ast.getChild(0).getType());
     }
     addInputsOutputsAlterTable(tableName, partSpec, alterTblDesc);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTblDesc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTblDesc)));
   }
 
   private void analyzeAlterTableCompact(ASTNode ast, String tableName,
@@ -2132,7 +2132,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       tableName, newPartSpec, type, isBlocking);
     desc.setProps(mapProp);
 
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private void analyzeAlterTableDropConstraint(ASTNode ast, String tableName)
@@ -2141,7 +2141,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     AlterTableDesc alterTblDesc = new AlterTableDesc(tableName, dropConstraintName, (ReplicationSpec)null);
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
   }
 
   private void analyzeAlterTableAddConstraint(ASTNode ast, String tableName)
@@ -2174,7 +2174,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
             uniqueConstraints, null);
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
   }
 
   static HashMap<String, String> getProps(ASTNode prop) {
@@ -2416,7 +2416,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     inputs.add(new ReadEntity(getTable(tableName)));
     Task ddlTask = TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        descTblDesc), conf);
+        descTblDesc));
     rootTasks.add(ddlTask);
     String schema = DescTableDesc.getSchema(showColStats);
     setFetchTask(createFetchTask(schema));
@@ -2447,7 +2447,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     DescDatabaseDesc descDbDesc = new DescDatabaseDesc(ctx.getResFile(),
         dbName, isExtended);
     inputs.add(new ReadEntity(getDatabase(dbName)));
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), descDbDesc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), descDbDesc)));
     setFetchTask(createFetchTask(descDbDesc.getSchema()));
   }
 
@@ -2494,7 +2494,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     showPartsDesc = new ShowPartitionsDesc(tableName, ctx.getResFile(), partSpec);
     inputs.add(new ReadEntity(getTable(tableName)));
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showPartsDesc), conf));
+        showPartsDesc)));
     setFetchTask(createFetchTask(showPartsDesc.getSchema()));
   }
 
@@ -2506,7 +2506,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     Database database = getDatabase(dbName);
     inputs.add(new ReadEntity(database));
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showCreateDbDesc), conf));
+        showCreateDbDesc)));
     setFetchTask(createFetchTask(showCreateDbDesc.getSchema()));
   }
 
@@ -2519,7 +2519,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     Table tab = getTable(tableName);
     inputs.add(new ReadEntity(tab));
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showCreateTblDesc), conf));
+        showCreateTblDesc)));
     setFetchTask(createFetchTask(showCreateTblDesc.getSchema()));
   }
 
@@ -2531,7 +2531,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     } else {
       showDatabasesDesc = new ShowDatabasesDesc(ctx.getResFile());
     }
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), showDatabasesDesc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), showDatabasesDesc)));
     setFetchTask(createFetchTask(showDatabasesDesc.getSchema()));
   }
 
@@ -2568,7 +2568,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
     inputs.add(new ReadEntity(getDatabase(dbName)));
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showTblsDesc), conf));
+        showTblsDesc)));
     setFetchTask(createFetchTask(showTblsDesc.getSchema()));
   }
 
@@ -2613,7 +2613,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     Table tab = getTable(tableName);
     inputs.add(new ReadEntity(tab));
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showColumnsDesc), conf));
+        showColumnsDesc)));
     setFetchTask(createFetchTask(showColumnsDesc.getSchema()));
   }
 
@@ -2647,7 +2647,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     showTblStatusDesc = new ShowTableStatusDesc(ctx.getResFile().toString(), dbName,
         tableNames, partSpec);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showTblStatusDesc), conf));
+        showTblStatusDesc)));
     setFetchTask(createFetchTask(showTblStatusDesc.getSchema()));
   }
 
@@ -2665,7 +2665,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     showTblPropertiesDesc = new ShowTblPropertiesDesc(ctx.getResFile().toString(), tableNames,
         propertyName);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showTblPropertiesDesc), conf));
+        showTblPropertiesDesc)));
     setFetchTask(createFetchTask(showTblPropertiesDesc.getSchema()));
   }
 
@@ -2691,7 +2691,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       showFuncsDesc = new ShowFunctionsDesc(ctx.getResFile());
     }
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showFuncsDesc), conf));
+        showFuncsDesc)));
     setFetchTask(createFetchTask(showFuncsDesc.getSchema()));
   }
 
@@ -2738,7 +2738,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     ShowLocksDesc showLocksDesc = new ShowLocksDesc(ctx.getResFile(), tableName,
         partSpec, isExtended, txnManager.useNewShowLocksFormat());
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showLocksDesc), conf));
+        showLocksDesc)));
     setFetchTask(createFetchTask(showLocksDesc.getSchema()));
 
     // Need to initialize the lock manager
@@ -2768,7 +2768,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     ShowLocksDesc showLocksDesc = new ShowLocksDesc(ctx.getResFile(), dbName,
                                                     isExtended, txnManager.useNewShowLocksFormat());
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showLocksDesc), conf));
+        showLocksDesc)));
     setFetchTask(createFetchTask(showLocksDesc.getSchema()));
 
     // Need to initialize the lock manager
@@ -2779,7 +2779,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     String confName = stripQuotes(ast.getChild(0).getText());
     ShowConfDesc showConfDesc = new ShowConfDesc(ctx.getResFile(), confName);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showConfDesc), conf));
+        showConfDesc)));
     setFetchTask(createFetchTask(showConfDesc.getSchema()));
   }
 
@@ -2818,7 +2818,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showViewsDesc), conf));
+        showViewsDesc)));
     setFetchTask(createFetchTask(showViewsDesc.getSchema()));
   }
 
@@ -2859,7 +2859,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showMaterializedViewsDesc), conf));
+        showMaterializedViewsDesc)));
     setFetchTask(createFetchTask(showMaterializedViewsDesc.getSchema()));
   }
 
@@ -2889,7 +2889,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         HiveConf.getVar(conf, ConfVars.HIVEQUERYID));
     lockTblDesc.setQueryStr(this.ctx.getCmd());
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        lockTblDesc), conf));
+        lockTblDesc)));
 
     // Need to initialize the lock manager
     ctx.setNeedLockMgr(true);
@@ -2902,7 +2902,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
    */
   private void analyzeShowCompactions(ASTNode ast) throws SemanticException {
     ShowCompactionsDesc desc = new ShowCompactionsDesc(ctx.getResFile());
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
     setFetchTask(createFetchTask(desc.getSchema()));
   }
 
@@ -2913,7 +2913,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
    */
   private void analyzeShowTxns(ASTNode ast) throws SemanticException {
     ShowTxnsDesc desc = new ShowTxnsDesc(ctx.getResFile());
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
     setFetchTask(createFetchTask(desc.getSchema()));
   }
 
@@ -2929,7 +2929,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       txnids.add(Long.parseLong(ast.getChild(i).getText()));
     }
     AbortTxnsDesc desc = new AbortTxnsDesc(txnids);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
    /**
@@ -2945,7 +2945,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
     addServiceOutput();
     KillQueryDesc desc = new KillQueryDesc(queryIds);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
   }
 
   private void addServiceOutput() throws SemanticException {
@@ -2989,7 +2989,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     UnlockTableDesc unlockTblDesc = new UnlockTableDesc(tableName, partSpec);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        unlockTblDesc), conf));
+        unlockTblDesc)));
 
     // Need to initialize the lock manager
     ctx.setNeedLockMgr(true);
@@ -3009,7 +3009,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
                         HiveConf.getVar(conf, ConfVars.HIVEQUERYID));
     lockDatabaseDesc.setQueryStr(ctx.getCmd());
     DDLWork work = new DDLWork(getInputs(), getOutputs(), lockDatabaseDesc);
-    rootTasks.add(TaskFactory.get(work, conf));
+    rootTasks.add(TaskFactory.get(work));
     ctx.setNeedLockMgr(true);
   }
 
@@ -3025,7 +3025,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     UnlockDatabaseDesc unlockDatabaseDesc = new UnlockDatabaseDesc(dbName);
     DDLWork work = new DDLWork(getInputs(), getOutputs(), unlockDatabaseDesc);
-    rootTasks.add(TaskFactory.get(work, conf));
+    rootTasks.add(TaskFactory.get(work));
     // Need to initialize the lock manager
     ctx.setNeedLockMgr(true);
   }
@@ -3056,7 +3056,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     DescFunctionDesc descFuncDesc = new DescFunctionDesc(ctx.getResFile(),
         funcName, isExtended);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        descFuncDesc), conf));
+        descFuncDesc)));
     setFetchTask(createFetchTask(descFuncDesc.getSchema()));
   }
 
@@ -3071,7 +3071,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     AlterTableDesc alterTblDesc = new AlterTableDesc(sourceName, targetName, expectView, null);
     addInputsOutputsAlterTable(sourceName, null, alterTblDesc);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
   }
 
   private void analyzeAlterTableRenameCol(String[] qualified, ASTNode ast,
@@ -3180,7 +3180,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     addInputsOutputsAlterTable(tblName, partSpec, alterTblDesc);
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
   }
 
   private void analyzeAlterTableRenamePart(ASTNode ast, String tblName,
@@ -3202,7 +3202,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     addTablePartsOutputs(tab, partSpecs, WriteEntity.WriteType.DDL_EXCLUSIVE);
     RenamePartitionDesc renamePartitionDesc = new RenamePartitionDesc(tblName, oldPartSpec, newPartSpec, null);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        renamePartitionDesc), conf));
+        renamePartitionDesc)));
   }
 
   private void analyzeAlterTableBucketNum(ASTNode ast, String tblName,
@@ -3218,7 +3218,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     AlterTableDesc alterBucketNum = new AlterTableDesc(tblName, partSpec, bucketNum);
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterBucketNum), conf));
+        alterBucketNum)));
   }
 
   private void analyzeAlterTableModifyCols(String[] qualified, ASTNode ast,
@@ -3236,7 +3236,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     addInputsOutputsAlterTable(tblName, partSpec, alterTblDesc);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
   }
 
   private void analyzeAlterTableDropParts(String[] qualified, ASTNode ast, boolean expectView)
@@ -3293,7 +3293,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     DropTableDesc dropTblDesc =
         new DropTableDesc(getDotName(qualified), partSpecs, expectView ? TableType.VIRTUAL_VIEW : null,
                 mustPurge, replicationSpec);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropTblDesc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropTblDesc)));
   }
 
   private void analyzeAlterTablePartColType(String[] qualified, ASTNode ast)
@@ -3342,7 +3342,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
             new AlterTableAlterPartDesc(getDotName(qualified), newCol);
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-            alterTblAlterPartDesc), conf));
+        alterTblAlterPartDesc)));
   }
 
     /**
@@ -3424,7 +3424,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       return;
     }
 
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), addPartitionDesc), conf));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), addPartitionDesc)));
 
     if (isView) {
       // Compile internal query to capture underlying table partition dependencies
@@ -3448,7 +3448,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           } else {
             cmd.append(" AND ");
           }
-          cmd.append(HiveUtils.unparseIdentifier(entry.getKey(), conf));
+          cmd.append(HiveUtils.unparseIdentifier(entry.getKey()));
           cmd.append(" = '");
           cmd.append(HiveUtils.escapeString(entry.getValue()));
           cmd.append("'");
@@ -3492,7 +3492,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           AlterTableDesc.AlterTableTypes.TOUCH);
       outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL_NO_LOCK));
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-          touchDesc), conf));
+          touchDesc)));
     } else {
       addTablePartsOutputs(tab, partSpecs, WriteEntity.WriteType.DDL_NO_LOCK);
       for (Map<String, String> partSpec : partSpecs) {
@@ -3500,7 +3500,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
             getDotName(qualified), partSpec,
             AlterTableDesc.AlterTableTypes.TOUCH);
         rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-            touchDesc), conf));
+            touchDesc)));
       }
     }
   }
@@ -3539,7 +3539,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         getDotName(qualified), partSpec,
         (isUnArchive ? AlterTableTypes.UNARCHIVE : AlterTableTypes.ARCHIVE));
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        archiveDesc), conf));
+        archiveDesc)));
 
   }
 
@@ -3568,7 +3568,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     MsckDesc checkDesc = new MsckDesc(tableName, specs, ctx.getResFile(),
         repair);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        checkDesc), conf));
+        checkDesc)));
   }
 
   /**
@@ -3874,7 +3874,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           new ArrayList<String>(), new ArrayList<List<String>>());
       alterTblDesc.setStoredAsSubDirectories(false);
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-          alterTblDesc), conf));
+          alterTblDesc)));
     } else {
       switch (((ASTNode) ast.getChild(0)).getToken().getType()) {
       case HiveParser.TOK_TABLESKEWED:
@@ -3908,7 +3908,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         skewedColNames, skewedColValues);
     alterTblDesc.setStoredAsSubDirectories(false);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
   }
 
   /**
@@ -3940,7 +3940,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     alterTblDesc.setTable(tab);
     alterTblDesc.validate();
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
   }
 
   /**
@@ -4058,7 +4058,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     AlterTableDesc alterTblDesc = new AlterTableDesc(tableName, locations, partSpec);
     addInputsOutputsAlterTable(tableName, partSpec, alterTblDesc);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
+        alterTblDesc)));
   }
 
   private void addLocationToOutputs(String newLocation) throws SemanticException {
@@ -4164,7 +4164,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     inputs.add(new ReadEntity(materializedViewTable));
     outputs.add(new WriteEntity(materializedViewTable, WriteEntity.WriteType.DDL_EXCLUSIVE));
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterMVDesc), conf));
+        alterMVDesc)));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSQRewriteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSQRewriteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSQRewriteSemanticAnalyzer.java
index c427355..20e4290 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSQRewriteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSQRewriteSemanticAnalyzer.java
@@ -53,7 +53,7 @@ public class ExplainSQRewriteSemanticAnalyzer extends BaseSemanticAnalyzer {
         ctx
         );
 
-    ExplainSQRewriteTask explTask = (ExplainSQRewriteTask) TaskFactory.get(work, conf);
+    ExplainSQRewriteTask explTask = (ExplainSQRewriteTask) TaskFactory.get(work);
 
     fieldList = explTask.getResultSchema();
     rootTasks.add(explTask);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
index 63b13c8..918cc5a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
@@ -208,7 +208,7 @@ public class ExplainSemanticAnalyzer extends BaseSemanticAnalyzer {
     work.setAppendTaskType(
         HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEEXPLAINDEPENDENCYAPPENDTASKTYPES));
 
-    ExplainTask explTask = (ExplainTask) TaskFactory.get(work, conf);
+    ExplainTask explTask = (ExplainTask) TaskFactory.get(work);
 
     fieldList = explTask.getResultSchema();
     rootTasks.add(explTask);

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/ExportSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExportSemanticAnalyzer.java
index ef3e80d2..33f426c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExportSemanticAnalyzer.java
@@ -96,7 +96,7 @@ public class ExportSemanticAnalyzer extends BaseSemanticAnalyzer {
     ExportWork exportWork =
         new ExportWork(exportRootDirName, ts, replicationSpec, ErrorMsg.INVALID_PATH.getMsg(ast));
     // Create an export task and add it as a root task
-    Task<ExportWork> exportTask = TaskFactory.get(exportWork, conf);
+    Task<ExportWork> exportTask = TaskFactory.get(exportWork);
     rootTasks.add(exportTask);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
index bfa8ac1..2cfcc6b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
@@ -61,7 +61,7 @@ public class FunctionSemanticAnalyzer extends BaseSemanticAnalyzer {
     } else if (ast.getType() == HiveParser.TOK_DROPFUNCTION) {
       analyzeDropFunction(ast);
     } else if (ast.getType() == HiveParser.TOK_RELOADFUNCTION) {
-      rootTasks.add(TaskFactory.get(new FunctionWork(new ReloadFunctionDesc()), conf));
+      rootTasks.add(TaskFactory.get(new FunctionWork(new ReloadFunctionDesc())));
     }
 
     LOG.info("analyze done");
@@ -86,7 +86,7 @@ public class FunctionSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     CreateFunctionDesc desc =
         new CreateFunctionDesc(functionName, isTemporaryFunction, className, resources, null);
-    rootTasks.add(TaskFactory.get(new FunctionWork(desc), conf));
+    rootTasks.add(TaskFactory.get(new FunctionWork(desc)));
 
     addEntities(functionName, className, isTemporaryFunction, resources);
   }
@@ -114,7 +114,7 @@ public class FunctionSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     boolean isTemporaryFunction = (ast.getFirstChildWithType(HiveParser.TOK_TEMPORARY) != null);
     DropFunctionDesc desc = new DropFunctionDesc(functionName, isTemporaryFunction, null);
-    rootTasks.add(TaskFactory.get(new FunctionWork(desc), conf));
+    rootTasks.add(TaskFactory.get(new FunctionWork(desc)));
 
     addEntities(functionName, info.getClassName(), isTemporaryFunction, null);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/94152c99/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
index 3c2a092..0df581a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
@@ -174,7 +174,7 @@ public class GenTezProcContext implements NodeProcessorCtx{
     this.inputs = inputs;
     this.outputs = outputs;
     this.currentTask = (TezTask) TaskFactory.get(
-         new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID), conf), conf);
+        new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID), conf));
     this.leafOperatorToFollowingWork = new LinkedHashMap<Operator<?>, BaseWork>();
     this.linkOpWithWorkMap = new LinkedHashMap<Operator<?>, Map<BaseWork, TezEdgeProperty>>();
     this.linkWorkWithReduceSinkMap = new LinkedHashMap<BaseWork, List<ReduceSinkOperator>>();
@@ -186,7 +186,7 @@ public class GenTezProcContext implements NodeProcessorCtx{
     this.currentMapJoinOperators = new LinkedHashSet<MapJoinOperator>();
     this.linkChildOpWithDummyOp = new LinkedHashMap<Operator<?>, List<Operator<?>>>();
     this.dependencyTask = (DependencyCollectionTask)
-        TaskFactory.get(new DependencyCollectionWork(), conf);
+    TaskFactory.get(new DependencyCollectionWork());
     this.unionWorkMap = new LinkedHashMap<Operator<?>, BaseWork>();
     this.rootUnionWorkMap = new LinkedHashMap<Operator<?>, UnionWork>();
     this.currentUnionOperators = new LinkedList<UnionOperator>();