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

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

                Author: ASF GitHub Bot
            Created on: 30/Nov/21 08:16
            Start Date: 30/Nov/21 08:16
    Worklog Time Spent: 10m 
      Work Description: ArkoSharma commented on a change in pull request #2539:
URL: https://github.com/apache/hive/pull/2539#discussion_r759026088



##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplExternalTables.java
##########
@@ -189,57 +191,137 @@ private void dirLocationToCopy(String tableName, 
FileList fileList, Path sourceP
       targetPath = new Path(Utils.replaceHost(targetPath.toString(), 
sourcePath.toUri().getHost()));
       sourcePath = new Path(Utils.replaceHost(sourcePath.toString(), 
remoteNS));
     }
-    fileList.add(new DirCopyWork(tableName, sourcePath, targetPath, copyMode, 
snapshotPrefix).convertToString());
+    fileList.add(new DirCopyWork(tableName, sourcePath, targetPath, copyMode, 
snapshotPrefix, isBootstrap).convertToString());
   }
 
-  private SnapshotUtils.SnapshotCopyMode createSnapshotsAtSource(Path 
sourcePath, String snapshotPrefix,
-      boolean isSnapshotEnabled, HiveConf conf, 
SnapshotUtils.ReplSnapshotCount replSnapshotCount, FileList snapPathFileList,
-      ArrayList<String> prevSnaps, boolean isBootstrap) throws IOException {
+  SnapshotUtils.SnapshotCopyMode createSnapshotsAtSource(Path sourcePath, Path 
targetPath, String snapshotPrefix,
+                                                                              
boolean isSnapshotEnabled, HiveConf conf, SnapshotUtils.ReplSnapshotCount 
replSnapshotCount, FileList snapPathFileList,
+                                                                              
ArrayList<String> prevSnaps, boolean isBootstrap) throws IOException {
     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);
     try {
-      if(isBootstrap) {
-        // Delete any pre existing snapshots.
-        SnapshotUtils.deleteSnapshotIfExists(sourceDfs, sourcePath, 
firstSnapshot(snapshotPrefix), conf);
-        SnapshotUtils.deleteSnapshotIfExists(sourceDfs, sourcePath, 
secondSnapshot(snapshotPrefix), conf);
-        allowAndCreateInitialSnapshot(sourcePath, snapshotPrefix, conf, 
replSnapshotCount, snapPathFileList, sourceDfs);
-        return INITIAL_COPY;
+      if(isBootstrap && 
conf.getBoolVar(HiveConf.ConfVars.REPL_REUSE_SNAPSHOTS)) {
+        try {
+          FileStatus[] listing = sourceDfs.listStatus(new Path(sourcePath, 
".snapshot"));
+          for (FileStatus elem : listing) {
+            String snapShotName = elem.getPath().getName();
+            String prefix;
+            if (snapShotName.contains(OLD_SNAPSHOT)) {
+              prefix = snapShotName.substring(0, 
snapShotName.lastIndexOf(OLD_SNAPSHOT));
+              if(!prefix.equals(snapshotPrefix)) {
+                sourceDfs.renameSnapshot(sourcePath, firstSnapshot(prefix), 
firstSnapshot(snapshotPrefix));

Review comment:
       done.

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplExternalTables.java
##########
@@ -189,57 +191,137 @@ private void dirLocationToCopy(String tableName, 
FileList fileList, Path sourceP
       targetPath = new Path(Utils.replaceHost(targetPath.toString(), 
sourcePath.toUri().getHost()));
       sourcePath = new Path(Utils.replaceHost(sourcePath.toString(), 
remoteNS));
     }
-    fileList.add(new DirCopyWork(tableName, sourcePath, targetPath, copyMode, 
snapshotPrefix).convertToString());
+    fileList.add(new DirCopyWork(tableName, sourcePath, targetPath, copyMode, 
snapshotPrefix, isBootstrap).convertToString());
   }
 
-  private SnapshotUtils.SnapshotCopyMode createSnapshotsAtSource(Path 
sourcePath, String snapshotPrefix,
-      boolean isSnapshotEnabled, HiveConf conf, 
SnapshotUtils.ReplSnapshotCount replSnapshotCount, FileList snapPathFileList,
-      ArrayList<String> prevSnaps, boolean isBootstrap) throws IOException {
+  SnapshotUtils.SnapshotCopyMode createSnapshotsAtSource(Path sourcePath, Path 
targetPath, String snapshotPrefix,
+                                                                              
boolean isSnapshotEnabled, HiveConf conf, SnapshotUtils.ReplSnapshotCount 
replSnapshotCount, FileList snapPathFileList,
+                                                                              
ArrayList<String> prevSnaps, boolean isBootstrap) throws IOException {
     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);
     try {
-      if(isBootstrap) {
-        // Delete any pre existing snapshots.
-        SnapshotUtils.deleteSnapshotIfExists(sourceDfs, sourcePath, 
firstSnapshot(snapshotPrefix), conf);
-        SnapshotUtils.deleteSnapshotIfExists(sourceDfs, sourcePath, 
secondSnapshot(snapshotPrefix), conf);
-        allowAndCreateInitialSnapshot(sourcePath, snapshotPrefix, conf, 
replSnapshotCount, snapPathFileList, sourceDfs);
-        return INITIAL_COPY;
+      if(isBootstrap && 
conf.getBoolVar(HiveConf.ConfVars.REPL_REUSE_SNAPSHOTS)) {
+        try {
+          FileStatus[] listing = sourceDfs.listStatus(new Path(sourcePath, 
".snapshot"));
+          for (FileStatus elem : listing) {
+            String snapShotName = elem.getPath().getName();
+            String prefix;
+            if (snapShotName.contains(OLD_SNAPSHOT)) {
+              prefix = snapShotName.substring(0, 
snapShotName.lastIndexOf(OLD_SNAPSHOT));
+              if(!prefix.equals(snapshotPrefix)) {
+                sourceDfs.renameSnapshot(sourcePath, firstSnapshot(prefix), 
firstSnapshot(snapshotPrefix));
+              }
+            }
+            if (snapShotName.contains(NEW_SNAPSHOT)) {
+              prefix = snapShotName.substring(0, 
snapShotName.lastIndexOf(NEW_SNAPSHOT));
+              if(!prefix.equals(snapshotPrefix)) {
+                sourceDfs.renameSnapshot(sourcePath, secondSnapshot(prefix), 
secondSnapshot(snapshotPrefix));

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.

To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

    Worklog Id:     (was: 687831)
    Time Spent: 4h  (was: 3h 50m)

> Snapshot support for controlled failover
> ----------------------------------------
>
>                 Key: HIVE-25397
>                 URL: https://issues.apache.org/jira/browse/HIVE-25397
>             Project: Hive
>          Issue Type: Bug
>            Reporter: Arko Sharma
>            Assignee: Arko Sharma
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 4h
>  Remaining Estimate: 0h
>
> In case the same locations are used for external tables on the source and 
> target, then the snapshots created during replication can be re-used during 
> reverse replication. This patch enables re-using the snapshots  during 
> reverse replication using a configuration.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to