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

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

                Author: ASF GitHub Bot
            Created on: 17/May/21 18:41
            Start Date: 17/May/21 18:41
    Worklog Time Spent: 10m 
      Work Description: ayushtkn commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r633751758



##########
File path: 
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosUsingSnapshots.java
##########
@@ -1258,6 +1259,134 @@ private void validateDiffSnapshotsCreated(String 
location) throws Exception {
         dfs.getFileStatus(new Path(locationPath, ".snapshot/" + 
secondSnapshot(primaryDbName.toLowerCase()))));
   }
 
+  @Test
+  public void testSnapshotsWithFiltersCustomDbLevelPaths() throws Throwable {
+    // Directory Structure:
+    //    /prefix/project/   <- Specified as custom Location.(Snapshot Root)
+    //                        /randomStuff <- Not to be copied as part of 
external data copy
+    //                        /warehouse1 <- To be copied, Contains table1 & 
table2
+    //                       /warehouse2 <- To be copied, Contains table3 & 
table4
+
+    // Create /prefix/project
+    Path project = new Path("/" + testName.getMethodName() + "/project");
+    DistributedFileSystem fs = primary.miniDFSCluster.getFileSystem();
+    fs.mkdirs(project);
+
+    // Create /prefix/project/warehouse1
+    Path warehouse1 = new Path(project, "warehouse1");
+    fs.mkdirs(warehouse1);
+
+    // Create /prefix/project/warehouse2
+    Path warehouse2 = new Path(project, "warehouse2");
+    fs.mkdirs(warehouse2);
+
+    // Table1 Path: /prefix/project/warehouse1/table1
+    Path table1 = new Path(warehouse1, "table1");
+    fs.mkdirs(table1);
+
+    // Table2 Path: /prefix/project/warehouse1/table2
+    Path table2 = new Path(warehouse1, "table2");
+    fs.mkdirs(table2);
+
+    // Table3 Path: /prefix/project/warehouse2/table3
+    Path table3 = new Path(warehouse2, "table3");
+    fs.mkdirs(table3);
+
+    // Table4 Path: /prefix/project/warehouse2/table4
+    Path table4 = new Path(warehouse2, "table4");
+    fs.mkdirs(table4);
+
+    // Random Dir inside the /prefix/project
+    Path random = new Path(project, "randomStuff");
+    fs.mkdirs(random);
+
+    fs.create(new Path(random, "file1")).close();
+    fs.create(new Path(random, "file2")).close();
+    fs.create(new Path(random, "file3")).close();
+
+    // Create a filter file for DistCp
+    Path filterFile = new Path("/tmp/filter");
+    try(FSDataOutputStream stream = fs.create(filterFile)) {
+      stream.writeBytes(".*randomStuff.*");
+    }
+    assertTrue(fs.exists(filterFile.makeQualified(fs.getUri(), 
fs.getWorkingDirectory())));
+    FileWriter myWriter = new FileWriter("/tmp/filter");
+    myWriter.write(".*randomStuff.*");
+    myWriter.close();
+
+    // Specify the project directory as the snapshot root using the single 
copy task path config.
+    List<String> withClause = 
ReplicationTestUtils.includeExternalTableClause(true);
+    withClause.add("'"
+        + REPL_EXTERNAL_WAREHOUSE_SINGLE_COPY_TASK_PATHS.varname + "'='" + 
project
+        .makeQualified(fs.getUri(), fs.getWorkingDirectory()).toString() + 
"'");
+
+    // Add Filter file
+    withClause.add("'distcp.options.filters'='" + "/tmp/filter" + "'");

Review comment:
       Done

##########
File path: 
shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
##########
@@ -1197,6 +1241,112 @@ public boolean runDistCp(List<Path> srcPaths, Path dst, 
Configuration conf) thro
     }
   }
 
+  @Override
+  public boolean runDistCpWithSnapshots(String oldSnapshot, String 
newSnapshot, List<Path> srcPaths, Path dst, Configuration conf)
+      throws IOException {
+    DistCpOptions options =
+        new DistCpOptions.Builder(srcPaths, 
dst).withSyncFolder(true).withUseDiff(oldSnapshot, newSnapshot)
+        
.preserve(FileAttribute.BLOCKSIZE).preserve(FileAttribute.XATTR).build();
+
+    List<String> params = constructDistCpWithSnapshotParams(srcPaths, dst, 
oldSnapshot, newSnapshot, conf, "-diff");
+    try {
+      conf.setBoolean("mapred.mapper.new-api", true);
+      DistCp distcp = new DistCp(conf, options);
+      int returnCode = distcp.run(params.toArray(new String[0]));
+      if (returnCode == 0) {
+        return true;
+      } else if (returnCode == DistCpConstants.INVALID_ARGUMENT) {
+        // Handling FileNotFoundException, if source got deleted, in that case 
we don't want to copy either, So it is
+        // like a success case, we didn't had anything to copy and we copied 
nothing, so, we need not to fail.
+        LOG.warn("Copy failed with INVALID_ARGUMENT for source: {} to target: 
{} snapshot1: {} snapshot2: {} "
+            + "params: {}", srcPaths, dst, oldSnapshot, newSnapshot, params);
+        return true;
+      } else if (returnCode == DistCpConstants.UNKNOWN_ERROR && conf
+          .getBoolean("hive.repl.externaltable.snapshot.overwrite.target", 
true)) {
+        // Check if this error is due to target modified.
+        if (shouldRdiff(dst, conf, oldSnapshot)) {
+          LOG.warn("Copy failed due to target modified. Attempting to restore 
back the target. source: {} target: {} "
+              + "snapshot: {}", srcPaths, dst, oldSnapshot);
+          List<String> rParams = constructDistCpWithSnapshotParams(srcPaths, 
dst, ".", oldSnapshot, conf, "-rdiff");
+          DistCp rDistcp = new DistCp(conf, options);
+          returnCode = rDistcp.run(rParams.toArray(new String[0]));
+          if (returnCode == 0) {
+            LOG.info("Target restored to previous state.  source: {} target: 
{} snapshot: {}. Reattempting to copy.",
+                srcPaths, dst, oldSnapshot);
+            dst.getFileSystem(conf).deleteSnapshot(dst, oldSnapshot);
+            dst.getFileSystem(conf).createSnapshot(dst, oldSnapshot);
+            returnCode = distcp.run(params.toArray(new String[0]));
+            if (returnCode == 0) {
+              return true;
+            } else {
+              LOG.error("Copy failed with after target restore for source: {} 
to target: {} snapshot1: {} snapshot2: "
+                  + "{} params: {}. Return code: {}", srcPaths, dst, 
oldSnapshot, newSnapshot, params, returnCode);
+              return false;
+            }
+          }
+        }
+      }
+    } catch (Exception e) {
+      throw new IOException("Cannot execute DistCp process: ", e);
+    } finally {
+      conf.setBoolean("mapred.mapper.new-api", false);
+    }
+    return false;
+  }
+
+  /**
+   * Checks wether reverse diff on the snapshot should be performed or not.
+   * @param p path where snapshot exists.
+   * @param conf the hive configuration.
+   * @param snapshot the name of snapshot.
+   * @return true, if we need to do rdiff.
+   */
+  private static boolean shouldRdiff(Path p, Configuration conf, String 
snapshot) throws Exception {
+    // Using the configuration in string form since hive-shims doesn't have a 
dependency on hive-common.
+    boolean isOverwrite = 
conf.getBoolean("hive.repl.externaltable.snapshot.overwrite.target", true);

Review comment:
       Extracted a constant in `HadoopShims.java` and used in all places, 
including the `HiveConf`. 




-- 
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: 598172)
    Time Spent: 6.5h  (was: 6h 20m)

> Add support for Snapshots during external table replication
> -----------------------------------------------------------
>
>                 Key: HIVE-24852
>                 URL: https://issues.apache.org/jira/browse/HIVE-24852
>             Project: Hive
>          Issue Type: Improvement
>            Reporter: Ayush Saxena
>            Assignee: Ayush Saxena
>            Priority: Critical
>              Labels: pull-request-available
>         Attachments: Design Doc HDFS Snapshots for External Table 
> Replication-01.pdf
>
>          Time Spent: 6.5h
>  Remaining Estimate: 0h
>
> Add support for use of snapshot diff for external table replication.



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

Reply via email to