aasha commented on a change in pull request #1084:
URL: https://github.com/apache/hive/pull/1084#discussion_r438162512



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
##########
@@ -367,6 +369,17 @@ public static boolean shouldReplicate(NotificationEvent 
tableForEvent,
     }
   }
 
+  public static boolean onSameHDFSFileSystem(Path dataPath, Path tgtPath) {

Review comment:
       Unit tests for this method

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
##########
@@ -243,29 +246,34 @@ private void addPartition(boolean hasMorePartitions, 
AlterTableAddPartitionDesc
               : LoadFileType.OVERWRITE_EXISTING);
       stagingDir = 
PathUtils.getExternalTmpPath(replicaWarehousePartitionLocation, 
context.pathInfo);
     }
-
-    Task<?> copyTask = ReplCopyTask.getLoadCopyTask(
-        event.replicationSpec(),
-        new Path(event.dataPath() + Path.SEPARATOR + 
getPartitionName(sourceWarehousePartitionLocation)),
-        stagingDir,
-        context.hiveConf, false, false
-    );
-
+    Path partDataSrc = new Path(event.dataPath() + File.separator + 
getPartitionName(sourceWarehousePartitionLocation));

Review comment:
       Can create a path directly from the constructor of Path instead of using 
File Separator

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
##########
@@ -686,9 +687,25 @@ private static ImportTableDesc 
getBaseCreateTableDescFromTable(String dbName,
         loadTableWork.setInheritTableSpecs(false);
         moveWork.setLoadTableWork(loadTableWork);
       }
+      Task<?> loadPartTask = TaskFactory.get(moveWork, x.getConf());
+      if (performOnlyMove) {
+        if (addPartTask != null) {
+          addPartTask.addDependentTask(loadPartTask);
+        }
+        x.getTasks().add(loadPartTask);
+        return addPartTask == null ? loadPartTask : addPartTask;

Review comment:
       can be simplified. There is already a not null check above

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
##########
@@ -452,7 +453,7 @@ private static ImportTableDesc 
getBaseCreateTableDescFromTable(String dbName,
     boolean isAutoPurge = false;
     boolean needRecycle = false;
     boolean copyToMigratedTxnTable = replicationSpec.isMigratingToTxnTable();
-
+    boolean performOnlyMove = replicationSpec.isInReplicationScope() && 
Utils.onSameHDFSFileSystem(dataPath, tgtPath);

Review comment:
       why do we need this extra condition 
replicationSpec.isInReplicationScope()?

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
##########
@@ -243,29 +246,34 @@ private void addPartition(boolean hasMorePartitions, 
AlterTableAddPartitionDesc
               : LoadFileType.OVERWRITE_EXISTING);
       stagingDir = 
PathUtils.getExternalTmpPath(replicaWarehousePartitionLocation, 
context.pathInfo);
     }
-
-    Task<?> copyTask = ReplCopyTask.getLoadCopyTask(
-        event.replicationSpec(),
-        new Path(event.dataPath() + Path.SEPARATOR + 
getPartitionName(sourceWarehousePartitionLocation)),
-        stagingDir,
-        context.hiveConf, false, false
-    );
-
+    Path partDataSrc = new Path(event.dataPath() + File.separator + 
getPartitionName(sourceWarehousePartitionLocation));
+    Path moveSource = performOnlyMove ? partDataSrc : stagingDir;
     Task<?> movePartitionTask = null;
     if (loadFileType != LoadFileType.IGNORE) {
       // no need to create move task, if file is moved directly to target 
location.
-      movePartitionTask = movePartitionTask(table, partSpec, stagingDir, 
loadFileType);
+      movePartitionTask = movePartitionTask(table, partSpec, moveSource, 
loadFileType);
     }
-
-    if (ptnRootTask == null) {
-      ptnRootTask = copyTask;
+    if (performOnlyMove) {
+      if (ptnRootTask == null) {
+        ptnRootTask = addPartTask;

Review comment:
       will the addPartTask not be added already as part of DDL operation?

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
##########
@@ -225,6 +227,7 @@ private void addPartition(boolean hasMorePartitions, 
AlterTableAddPartitionDesc
     }
 
     Path stagingDir = replicaWarehousePartitionLocation;
+    boolean performOnlyMove = Utils.onSameHDFSFileSystem(event.dataPath(), 
replicaWarehousePartitionLocation);

Review comment:
       can use stagingDir instead of replicaWarehousePartitionLocation

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
##########
@@ -243,29 +246,34 @@ private void addPartition(boolean hasMorePartitions, 
AlterTableAddPartitionDesc
               : LoadFileType.OVERWRITE_EXISTING);
       stagingDir = 
PathUtils.getExternalTmpPath(replicaWarehousePartitionLocation, 
context.pathInfo);
     }
-
-    Task<?> copyTask = ReplCopyTask.getLoadCopyTask(
-        event.replicationSpec(),
-        new Path(event.dataPath() + Path.SEPARATOR + 
getPartitionName(sourceWarehousePartitionLocation)),
-        stagingDir,
-        context.hiveConf, false, false
-    );
-
+    Path partDataSrc = new Path(event.dataPath() + File.separator + 
getPartitionName(sourceWarehousePartitionLocation));
+    Path moveSource = performOnlyMove ? partDataSrc : stagingDir;
     Task<?> movePartitionTask = null;
     if (loadFileType != LoadFileType.IGNORE) {
       // no need to create move task, if file is moved directly to target 
location.
-      movePartitionTask = movePartitionTask(table, partSpec, stagingDir, 
loadFileType);
+      movePartitionTask = movePartitionTask(table, partSpec, moveSource, 
loadFileType);

Review comment:
       why are we moving from source here?

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadTable.java
##########
@@ -297,17 +298,15 @@ static TableLocationTuple tableLocation(ImportTableDesc 
tblDesc, Database parent
     LOG.debug("adding dependent CopyWork/AddPart/MoveWork for table "
             + table.getCompleteName() + " with source location: "
             + dataPath.toString() + " and target location " + 
tgtPath.toString());
-
-    Task<?> copyTask = ReplCopyTask.getLoadCopyTask(replicationSpec, dataPath, 
tmpPath, context.hiveConf,
-            false, false);
+    Path moveSrcPath = performOnlyMove ? dataPath : tmpPath;

Review comment:
       What is moveSrcPath ? What is tmpPath? dataPath is on source or target? 
It would be good if we could rename these variables for readability




----------------------------------------------------------------
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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to