[ https://issues.apache.org/jira/browse/HIVE-24852?focusedWorklogId=571230&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-571230 ]
ASF GitHub Bot logged work on HIVE-24852: ----------------------------------------- Author: ASF GitHub Bot Created on: 24/Mar/21 15:17 Start Date: 24/Mar/21 15:17 Worklog Time Spent: 10m Work Description: ayushtkn commented on a change in pull request #2043: URL: https://github.com/apache/hive/pull/2043#discussion_r600581645 ########## File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java ########## @@ -945,6 +962,13 @@ Long bootStrapDump(Path dumpRoot, DumpMetaData dmd, Path cmRoot, Hive hiveDb) conf.getBoolVar(REPL_EXTERNAL_WAREHOUSE_SINGLE_COPY_TASK) && work.replScope.includeAllTables(); boolean isExternalTablePresent = false; + boolean isSnapshotEnabed = Review comment: Extended a config holder class, To prevent dupe resolution of configs ########## File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/DirCopyTask.java ########## @@ -203,4 +213,87 @@ public String getName() { public boolean canExecuteInParallel() { return true; } + + void copyUsingDistCpSnapshots(Path sourcePath, Path targetPath, + UserGroupInformation proxyUser) throws IOException { + + DistributedFileSystem sourceFs = SnapshotUtils.getDFS(sourcePath, conf); + DistributedFileSystem targetFs = SnapshotUtils.getDFS(targetPath, conf); + if (sourceFs == null || targetFs == null) { + LOG.error("Source and Destination filesystem are not " + + "DistributedFileSystem, using normal copy instead of snapshot " + + "copy, Source Path {}, Target Path {}, Source fs is {}, and " + + "Target fs {}", sourcePath, targetPath, + sourcePath.getFileSystem(conf).getClass(), + targetPath.getFileSystem(conf).getClass()); + FileUtils.distCp(sourcePath.getFileSystem(conf), // source file system + Collections.singletonList(sourcePath), // list of source paths + targetPath, false, proxyUser, conf, ShimLoader.getHadoopShims()); + // Since source/dest aren't DFS, no point trying to create snapshot at + // target, return from here. + return; + } + String prefix = conf.getVar( + HiveConf.ConfVars.REPL_SNAPSHOT_PREFIX_FOR_EXTERNAL_TABLE_COPY); + if (getWork().getCopyMode() + .equals(SnapshotUtils.SnapshotCopyMode.DIFF_COPY)) { + LOG.info("Using snapshot diff copy for source: {} and target: {}", + sourcePath, targetPath); + boolean result = FileUtils + .distCpWithSnapshot(sourceFs, prefix + "old", prefix + "initial", + Collections.singletonList(sourcePath), targetPath, proxyUser, + conf, ShimLoader.getHadoopShims()); + if (!result) { + LOG.error("Can not copy using snapshot diff for source: {} and " + + "target: {}. Falling back to normal copy.", sourcePath, + targetPath); + FileUtils.distCp(sourcePath.getFileSystem(conf), // source file system Review comment: Done -- 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: 571230) Time Spent: 2h 10m (was: 2h) > 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: 2h 10m > 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)