[ 
https://issues.apache.org/jira/browse/HIVE-22997?focusedWorklogId=405380&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-405380
 ]

ASF GitHub Bot logged work on HIVE-22997:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 18/Mar/20 12:13
            Start Date: 18/Mar/20 12:13
    Worklog Time Spent: 10m 
      Work Description: aasha commented on pull request #951: HIVE-22997 : Copy 
external table to target during Repl Dump operation
URL: https://github.com/apache/hive/pull/951#discussion_r394297682
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
 ##########
 @@ -662,17 +690,48 @@ void dumpTable(String dbName, String tblName, String 
validTxnList, Path dbRoot,
     replLogger.tableLog(tblName, tableSpec.tableHandle.getTableType());
     if (tableSpec.tableHandle.getTableType().equals(TableType.EXTERNAL_TABLE)
             || Utils.shouldDumpMetaDataOnly(conf)) {
-      return;
+      return Collections.EMPTY_LIST;
+    }
+    return replPathMappings;
+  }
+
+  private void intitiateDataCopyTasks() {
+    Iterator<ExternalTableCopyTaskBuilder.DirCopyWork> extCopyWorkItr = 
work.getDirCopyIterator();
+    ReplOperationCompleteAckWork replDumpCompleteAckWork = new 
ReplOperationCompleteAckWork(dumpAckFile);
+    Task<ReplOperationCompleteAckWork> dumpCompleteAckWorkTask = 
TaskFactory.get(replDumpCompleteAckWork, conf);
+    List<Task<?>> childTasks = new ArrayList<>();
+    int maxTasks = 
conf.getIntVar(HiveConf.ConfVars.REPL_APPROX_MAX_LOAD_TASKS);
+    TaskTracker taskTracker = new TaskTracker(maxTasks);
+    while (taskTracker.canAddMoreTasks() && hasMoreCopyWork()) {
+      if (work.replPathIteratorInitialized() && extCopyWorkItr.hasNext()) {
+        childTasks.addAll(new ExternalTableCopyTaskBuilder(work, 
conf).tasks(taskTracker));
+      } else {
+        childTasks.addAll(ReplPathMapping.tasks(work, taskTracker, conf));
+      }
     }
-    for (ReplPathMapping replPathMapping: replPathMappings) {
-      Task<?> copyTask = ReplCopyTask.getLoadCopyTask(
-              tuple.replicationSpec, replPathMapping.getSrcPath(), 
replPathMapping.getTargetPath(), conf, false);
-      this.addDependentTask(copyTask);
-      LOG.info("Scheduled a repl copy task from [{}] to [{}]",
-              replPathMapping.getSrcPath(), replPathMapping.getTargetPath());
+    if (!childTasks.isEmpty()) {
+      boolean ackTaskAdded = false;
+      if (taskTracker.canAddMoreTasks()) {
+        childTasks.add(dumpCompleteAckWorkTask);
+        ackTaskAdded = true;
+      }
+      if (hasMoreCopyWork() || !ackTaskAdded) {
+        DAGTraversal.traverse(childTasks, new 
AddDependencyToLeaves(TaskFactory.get(work, conf)));
 
 Review comment:
   Add the ack only after no more work is pending
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 405380)
    Time Spent: 5h  (was: 4h 50m)

> Copy external table to target during Repl Dump operation
> --------------------------------------------------------
>
>                 Key: HIVE-22997
>                 URL: https://issues.apache.org/jira/browse/HIVE-22997
>             Project: Hive
>          Issue Type: Task
>            Reporter: PRAVIN KUMAR SINHA
>            Assignee: PRAVIN KUMAR SINHA
>            Priority: Major
>              Labels: pull-request-available
>         Attachments: HIVE-22997.03.patch, HIVE-22997.04.patch, 
> HIVE-22997.1.patch, HIVE-22997.10.patch, HIVE-22997.11.patch, 
> HIVE-22997.2.patch, HIVE-22997.4.patch, HIVE-22997.5.patch, 
> HIVE-22997.6.patch, HIVE-22997.7.patch, HIVE-22997.8.patch, HIVE-22997.9.patch
>
>          Time Spent: 5h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to