[ https://issues.apache.org/jira/browse/HIVE-24852?focusedWorklogId=570985&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-570985 ]
ASF GitHub Bot logged work on HIVE-24852: ----------------------------------------- Author: ASF GitHub Bot Created on: 24/Mar/21 07:59 Start Date: 24/Mar/21 07:59 Worklog Time Spent: 10m Work Description: aasha commented on a change in pull request #2043: URL: https://github.com/apache/hive/pull/2043#discussion_r600246810 ########## File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplExternalTables.java ########## @@ -221,7 +251,66 @@ private void dirLocationToCopy(FileList fileList, Path sourcePath, HiveConf conf targetPath = new Path(Utils.replaceHost(targetPath.toString(), sourcePath.toUri().getHost())); sourcePath = new Path(Utils.replaceHost(sourcePath.toString(), remoteNS)); } - fileList.add(new DirCopyWork(sourcePath, targetPath).convertToString()); + fileList.add( + new DirCopyWork(sourcePath, targetPath, copyMode).convertToString()); + } + + private SnapshotUtils.SnapshotCopyMode createSnapshotsAtSource( + Path sourcePath, String snapshotPrefix, boolean isSnapshotEnabled, + HiveConf conf, HashSet<String> numFailedSnapshotPaths) { + if (!isSnapshotEnabled) { + LOG.info("Snapshot copy not enabled for path {} Will use normal " + + "distCp for copying data.", + sourcePath); + return FALLBACK_COPY; + } + DistributedFileSystem sourceDfs = SnapshotUtils.getDFS(sourcePath, conf); + + if (sourceDfs != null) { + try { + // check if second snapshot exists. + boolean isSecondSnapAvlb = SnapshotUtils + .isSnapshotAvailable(sourceDfs, sourcePath, snapshotPrefix, + "old"); + if (isSecondSnapAvlb) { + sourceDfs.deleteSnapshot(sourcePath, snapshotPrefix + "old"); + sourceDfs.renameSnapshot(sourcePath, snapshotPrefix + "initial", + snapshotPrefix + "old"); + sourceDfs.createSnapshot(sourcePath, snapshotPrefix + "initial"); + return DIFF_COPY; + } else { + // Check if first snapshot is available + boolean isFirstSnapshotAvailable = SnapshotUtils + .isSnapshotAvailable(sourceDfs, sourcePath, snapshotPrefix, + "initial"); + if (isFirstSnapshotAvailable) { + sourceDfs.renameSnapshot(sourcePath, snapshotPrefix + "initial", + snapshotPrefix + "old"); + sourceDfs.createSnapshot(sourcePath, snapshotPrefix + "initial"); + return DIFF_COPY; + } else { + if (SnapshotUtils.allowSnapshot(sourceDfs, sourcePath)) { + ReplUtils.createSnapshotSafe(sourceDfs, sourcePath, + snapshotPrefix + "initial"); + return INITIAL_COPY; + } else { + numFailedSnapshotPaths.add(sourcePath.toString()); + LOG.error("Can not allow Snapshot for path {}", sourcePath); + return FALLBACK_COPY; + } + } + } + } catch (Exception e) { + LOG.error("Error encountered during snapshot setup for path {}", + sourcePath, e); + numFailedSnapshotPaths.add(sourcePath.toString()); + return FALLBACK_COPY; + } + } else { Review comment: this else is not needed. all other paths return from the top. -- 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: 570985) Time Spent: 50m (was: 40m) > 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: 50m > 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)