[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-18 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 18/May/21 16:17
Start Date: 18/May/21 16:17
Worklog Time Spent: 10m 
  Work Description: aasha merged pull request #2043:
URL: https://github.com/apache/hive/pull/2043


   


-- 
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: 598668)
Time Spent: 7h  (was: 6h 50m)

> 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: 7h
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: 
shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
##
@@ -1197,6 +1241,112 @@ public boolean runDistCp(List srcPaths, Path dst, 
Configuration conf) thro
 }
   }
 
+  @Override
+  public boolean runDistCpWithSnapshots(String oldSnapshot, String 
newSnapshot, List 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 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 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:
   Done, Getting the value propagated from `DirCopyTask` itself




-- 
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: 598434)
Time Spent: 6h 50m  (was: 6h 40m)

> Add support for Snapshots during external table replication
> ---
>
> Key: HIVE-24852
> URL: https://issues.apache.org/jira/browse/HIVE-24852
>  

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: 
shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
##
@@ -1197,6 +1241,112 @@ public boolean runDistCp(List srcPaths, Path dst, 
Configuration conf) thro
 }
   }
 
+  @Override
+  public boolean runDistCpWithSnapshots(String oldSnapshot, String 
newSnapshot, List 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 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 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:
   No I meant can you not pass the value of the config to HadoopShims 
method of run distcp from whoever is calling this method




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

> Add support for Snapshots during external table replication
> ---
>
> Key: HIVE-24852
>   

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HIVE-24852?focusedWorklogId=598172=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 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 srcPaths, Path dst, 
Configuration conf) thro
 }
   }
 
+  @Override
+  public boolean runDistCpWithSnapshots(String oldSnapshot, String 
newSnapshot, List 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 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, 

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
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 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:
   Clean up the filter file after the test

##
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##
@@ -675,6 +675,16 @@ private static void populateLlapDaemonVarsSet(Set 
llapDaemonVarsSetLocal
 + " table or partition level. If hive.exec.parallel \n"
 + "is set to true then max worker threads created for copy can be 
hive.exec.parallel.thread.number(determines \n"
 + "number of copy tasks in parallel) * hive.repl.parallel.copy.tasks 
"),
+
REPL_SNAPSHOT_DIFF_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshotdiff.copy",
+false,"Use snapshot diff for copying data from source to "
++ "destination cluster for external table in distcp. If true it uses 
snapshot based distcp for all the paths "
++ "configured as part of hive.repl.external.warehouse.single.copy.task 
along with the external warehouse "
++ "default location."),
+
REPL_SNAPSHOT_OVERWRITE_TARGET_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshot.overwrite.target",

Review comment:
   where are you not taking the custom location paths?

##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
##
@@ -217,7 +224,12 @@ public int execute() {
   throw e;
 } catch (Exception e) {
   setException(e);
-  int errorCode = ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode();
+  int errorCode;
+  if 

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java
##
@@ -342,7 +343,12 @@ public static PathFilter getBootstrapDirectoryFilter(final 
FileSystem fs) {
 
   public static int handleException(boolean isReplication, Throwable e, String 
nonRecoverablePath,
 ReplicationMetricCollector 
metricCollector, String stageName, HiveConf conf){
-int errorCode = ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode();
+int errorCode;
+if (isReplication && e instanceof SnapshotException) {
+  errorCode = ErrorMsg.getErrorMsg("SNAPSHOT_ERROR").getErrorCode();

Review comment:
   Yes, It will be preserved. The entire stack trace is written here. 
https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java#L353
   
   and the exception is already set above, example:
   
https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java#L141
   
   Every code that calls this sets the exception before and calls this for 
error code to check if it is recoverable or non-recoverable.
   This logic is just to get the error code, which is used to decide whether 
the error is recoverable or non-recoverable.
   
   And the trace is also propagated back, example from `testFailureScenarios()`:
   
   ```
   Caused by: org.apache.hadoop.hdfs.protocol.SnapshotException: Nested 
snapshottable directories not allowed: 
path=/testFailureScenariossource1/tablesource, the ancestor 
/testFailureScenariossource1 is already a snapshottable directory.
   at 
org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.checkNestedSnapshottable(SnapshotManager.java:174)
   at 
org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.setSnapshottable(SnapshotManager.java:189)
   at 
org.apache.hadoop.hdfs.server.namenode.FSDirSnapshotOp.allowSnapshot(FSDirSnapshotOp.java:62)
   at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.allowSnapshot(FSNamesystem.java:6366)
   at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.allowSnapshot(NameNodeRpcServer.java:1842)
   at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.allowSnapshot(ClientNamenodeProtocolServerSideTranslatorPB.java:1211)
   at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
   at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:523)
   
   ```

##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
##
@@ -217,7 +224,12 @@ public int execute() {
   throw e;
 } catch (Exception e) {
   setException(e);
-  int errorCode = ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode();
+  int errorCode;
+  if (e instanceof SnapshotException) {
+errorCode = ErrorMsg.getErrorMsg("SNAPSHOT_ERROR").getErrorCode();

Review comment:
   Answered here:
   https://github.com/apache/hive/pull/2043#discussion_r633722120

##
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##
@@ -675,6 +675,16 @@ private static void populateLlapDaemonVarsSet(Set 
llapDaemonVarsSetLocal
 + " table or partition level. If hive.exec.parallel \n"
 + "is set to true then max worker threads created for copy can be 
hive.exec.parallel.thread.number(determines \n"
 + "number of copy tasks in parallel) * hive.repl.parallel.copy.tasks 
"),
+
REPL_SNAPSHOT_DIFF_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshotdiff.copy",
+false,"Use snapshot diff for copying data from source to "
++ "destination cluster for external table in distcp. If true it uses 
snapshot based distcp for all the paths "
++ "configured as part of hive.repl.external.warehouse.single.copy.task 
along with the external warehouse "
++ "default location."),
+
REPL_SNAPSHOT_OVERWRITE_TARGET_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshot.overwrite.target",

Review comment:
   It is being handled as part of the new config,
   ```
   
REPL_EXTERNAL_WAREHOUSE_SINGLE_COPY_TASK_PATHS("hive.repl.external.warehouse.single.copy.task.paths",
   ```




-- 
This is an automated message from the Apache 

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##
@@ -675,6 +675,16 @@ private static void populateLlapDaemonVarsSet(Set 
llapDaemonVarsSetLocal
 + " table or partition level. If hive.exec.parallel \n"
 + "is set to true then max worker threads created for copy can be 
hive.exec.parallel.thread.number(determines \n"
 + "number of copy tasks in parallel) * hive.repl.parallel.copy.tasks 
"),
+
REPL_SNAPSHOT_DIFF_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshotdiff.copy",
+false,"Use snapshot diff for copying data from source to "
++ "destination cluster for external table in distcp. If true it uses 
snapshot based distcp for all the paths "
++ "configured as part of hive.repl.external.warehouse.single.copy.task 
along with the external warehouse "
++ "default location."),
+
REPL_SNAPSHOT_OVERWRITE_TARGET_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshot.overwrite.target",

Review comment:
   It is being handled as part of the new config,
   ```
   
REPL_EXTERNAL_WAREHOUSE_SINGLE_COPY_TASK_PATHS("hive.repl.external.warehouse.single.copy.task.paths",
   ```




-- 
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: 597805)
Time Spent: 6h  (was: 5h 50m)

> 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: 6h
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java
##
@@ -342,7 +343,12 @@ public static PathFilter getBootstrapDirectoryFilter(final 
FileSystem fs) {
 
   public static int handleException(boolean isReplication, Throwable e, String 
nonRecoverablePath,
 ReplicationMetricCollector 
metricCollector, String stageName, HiveConf conf){
-int errorCode = ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode();
+int errorCode;
+if (isReplication && e instanceof SnapshotException) {
+  errorCode = ErrorMsg.getErrorMsg("SNAPSHOT_ERROR").getErrorCode();

Review comment:
   Yes, It will be preserved. The entire stack trace is written here. 
https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java#L353
   
   and the exception is already set above, example:
   
https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java#L141
   
   Every code that calls this sets the exception before and calls this for 
error code to check if it is recoverable or non-recoverable.
   This logic is just to get the error code, which is used to decide whether 
the error is recoverable or non-recoverable.
   
   And the trace is also propagated back, example from `testFailureScenarios()`:
   
   ```
   Caused by: org.apache.hadoop.hdfs.protocol.SnapshotException: Nested 
snapshottable directories not allowed: 
path=/testFailureScenariossource1/tablesource, the ancestor 
/testFailureScenariossource1 is already a snapshottable directory.
   at 
org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.checkNestedSnapshottable(SnapshotManager.java:174)
   at 
org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.setSnapshottable(SnapshotManager.java:189)
   at 
org.apache.hadoop.hdfs.server.namenode.FSDirSnapshotOp.allowSnapshot(FSDirSnapshotOp.java:62)
   at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.allowSnapshot(FSNamesystem.java:6366)
   at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.allowSnapshot(NameNodeRpcServer.java:1842)
   at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.allowSnapshot(ClientNamenodeProtocolServerSideTranslatorPB.java:1211)
   at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
   at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:523)
   
   ```




-- 
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: 597802)
Time Spent: 5h 40m  (was: 5.5h)

> 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: 5h 40m
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
##
@@ -217,7 +224,12 @@ public int execute() {
   throw e;
 } catch (Exception e) {
   setException(e);
-  int errorCode = ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode();
+  int errorCode;
+  if (e instanceof SnapshotException) {
+errorCode = ErrorMsg.getErrorMsg("SNAPSHOT_ERROR").getErrorCode();

Review comment:
   Answered here:
   https://github.com/apache/hive/pull/2043#discussion_r633722120




-- 
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: 597803)
Time Spent: 5h 50m  (was: 5h 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: 5h 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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: 
shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
##
@@ -1197,6 +1241,112 @@ public boolean runDistCp(List srcPaths, Path dst, 
Configuration conf) thro
 }
   }
 
+  @Override
+  public boolean runDistCpWithSnapshots(String oldSnapshot, String 
newSnapshot, List 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 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 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:
   have to be careful to not modify this constant.can you not pass the 
value of the conf




-- 
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: 597689)
Time Spent: 5.5h  (was: 5h 20m)

> Add support for Snapshots during external table replication
> ---
>
> Key: HIVE-24852
> URL: 

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java
##
@@ -342,7 +343,12 @@ public static PathFilter getBootstrapDirectoryFilter(final 
FileSystem fs) {
 
   public static int handleException(boolean isReplication, Throwable e, String 
nonRecoverablePath,
 ReplicationMetricCollector 
metricCollector, String stageName, HiveConf conf){
-int errorCode = ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode();
+int errorCode;
+if (isReplication && e instanceof SnapshotException) {
+  errorCode = ErrorMsg.getErrorMsg("SNAPSHOT_ERROR").getErrorCode();

Review comment:
   Is the actual error msg retained so that users can check that?




-- 
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: 597687)
Time Spent: 5h 20m  (was: 5h 10m)

> 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: 5h 20m
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
##
@@ -217,7 +224,12 @@ public int execute() {
   throw e;
 } catch (Exception e) {
   setException(e);
-  int errorCode = ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode();
+  int errorCode;
+  if (e instanceof SnapshotException) {
+errorCode = ErrorMsg.getErrorMsg("SNAPSHOT_ERROR").getErrorCode();

Review comment:
   why does snapshot error need to be treated specially?




-- 
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: 597681)
Time Spent: 5h 10m  (was: 5h)

> 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: 5h 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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
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 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:
   Clean up the filter file after the test

##
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##
@@ -675,6 +675,16 @@ private static void populateLlapDaemonVarsSet(Set 
llapDaemonVarsSetLocal
 + " table or partition level. If hive.exec.parallel \n"
 + "is set to true then max worker threads created for copy can be 
hive.exec.parallel.thread.number(determines \n"
 + "number of copy tasks in parallel) * hive.repl.parallel.copy.tasks 
"),
+
REPL_SNAPSHOT_DIFF_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshotdiff.copy",
+false,"Use snapshot diff for copying data from source to "
++ "destination cluster for external table in distcp. If true it uses 
snapshot based distcp for all the paths "
++ "configured as part of hive.repl.external.warehouse.single.copy.task 
along with the external warehouse "
++ "default location."),
+
REPL_SNAPSHOT_OVERWRITE_TARGET_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshot.overwrite.target",

Review comment:
   where are you not taking the custom location paths?




-- 
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:   

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##
@@ -668,6 +668,18 @@ private static void populateLlapDaemonVarsSet(Set 
llapDaemonVarsSetLocal
 + " table or partition level. If hive.exec.parallel \n"
 + "is set to true then max worker threads created for copy can be 
hive.exec.parallel.thread.number(determines \n"
 + "number of copy tasks in parallel) * hive.repl.parallel.copy.tasks 
"),
+
REPL_SNAPSHOT_DIFF_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshotdiff.copy",
+false,"Use snapshot diff for copying data from source to "
++ "destination cluster for external table in distcp"),
+
REPL_SNAPSHOT_OVERWRITE_TARGET_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshot.overwrite.target",
+true,"If this is enabled, in case the target is modified, when using 
snapshot for external table"
++ "data copy, the target data is overwritten and the modifications are 
removed and the copy is again "
++ "attempted using the snapshot based approach. If disabled, the 
replication will fail in case the target is "
++ "modified."),
+REPL_SNAPSHOT_EXTERNAL_TABLE_PATHS("hive.repl.externatable.snapshot.paths",

Review comment:
   Removed




-- 
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: 592959)
Time Spent: 4h 50m  (was: 4h 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: 4h 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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java
##
@@ -342,7 +343,12 @@ public static PathFilter getBootstrapDirectoryFilter(final 
FileSystem fs) {
 
   public static int handleException(boolean isReplication, Throwable e, String 
nonRecoverablePath,
 ReplicationMetricCollector 
metricCollector, String stageName, HiveConf conf){
-int errorCode = ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode();
+int errorCode;
+if (isReplication && e instanceof SnapshotException) {
+  errorCode = ErrorMsg.getErrorMsg("SNAPSHOT_ERROR").getErrorCode();

Review comment:
   Actually it should be like that but, in the ErrorMsg, it gets the error 
code based on the Exception message, but in case of Snapshot Exception, There 
can be a bunch of messages, Different for Nested Snapshot, Parent 
Snapshottable, No Snapshot Exists and couple of more cases. So Identifying all 
of such case and there corresponding Error Message won't be possible, So, I 
catch all SnapshotException and get one Error code for them.




-- 
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: 592957)
Time Spent: 4h 40m  (was: 4.5h)

> 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: 4h 40m
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/DirCopyTask.java
##
@@ -218,4 +219,62 @@ public String getName() {
   public boolean canExecuteInParallel() {
 return true;
   }
+
+  boolean copyUsingDistCpSnapshots(Path sourcePath, Path targetPath, 
UserGroupInformation proxyUser) throws IOException {
+
+DistributedFileSystem targetFs = SnapshotUtils.getDFS(targetPath, conf);
+boolean result = false;
+if 
(getWork().getCopyMode().equals(SnapshotUtils.SnapshotCopyMode.DIFF_COPY)) {
+  LOG.info("Using snapshot diff copy for source: {} and target: {}", 
sourcePath, targetPath);
+   result = FileUtils
+  .distCpWithSnapshot(firstSnapshot(work.getSnapshotPrefix()), 
secondSnapshot(work.getSnapshotPrefix()),
+  Collections.singletonList(sourcePath), targetPath, proxyUser,
+  conf, ShimLoader.getHadoopShims());
+   if(result) {
+ // Delete the older snapshot from last iteration.
+ targetFs.deleteSnapshot(targetPath, 
firstSnapshot(work.getSnapshotPrefix()));
+   } else {
+ throw new IOException(
+ "Can not successfully copy external table data using snapshot 
diff. source:" + sourcePath + " and target: "
+ + targetPath);
+   }
+} else if 
(getWork().getCopyMode().equals(SnapshotUtils.SnapshotCopyMode.INITIAL_COPY)) {
+  LOG.info("Using snapshot initial copy for source: {} and target: {}", 
sourcePath, targetPath);
+  // Get the path relative to the initial snapshot for copy.
+  Path snapRelPath =
+  new Path(sourcePath, HdfsConstants.DOT_SNAPSHOT_DIR + "/" + 
secondSnapshot(work.getSnapshotPrefix()));
+
+  // This is the first time we are copying, check if the target is 
snapshottable or not, if not attempt to allow
+  // snapshots.
+  SnapshotUtils.allowSnapshot(targetFs, targetPath, conf);
+  // Attempt to delete the snapshot, in case this is a bootstrap post a 
failed incremental, Since in case of
+  // bootstrap we go from start, so delete any pre-existing snapshot.
+  SnapshotUtils.deleteSnapshotSafe(targetFs, targetPath, 
firstSnapshot(work.getSnapshotPrefix()));

Review comment:
   Changed behaviour, We fail now




-- 
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: 592955)
Time Spent: 4.5h  (was: 4h 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: 4.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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/SnapshotUtils.java
##
@@ -0,0 +1,415 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.exec.repl.util;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.ql.exec.util.Retryable;
+import org.apache.hadoop.hive.ql.parse.EximUtil;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static 
org.apache.hadoop.hive.conf.HiveConf.ConfVars.REPL_SNAPSHOT_DIFF_FOR_EXTERNAL_TABLE_COPY;
+import static 
org.apache.hadoop.hive.conf.HiveConf.ConfVars.REPL_SNAPSHOT_EXTERNAL_TABLE_PATHS;
+import static 
org.apache.hadoop.hive.ql.exec.repl.ReplDumpTask.createTableFileList;
+import static 
org.apache.hadoop.hive.ql.exec.repl.ReplExternalTables.externalTableDataPath;
+import static 
org.apache.hadoop.hive.ql.exec.repl.ReplExternalTables.getExternalTableBaseDir;
+
+/**
+ * Utility class for snapshot related operations.
+ */
+public class SnapshotUtils {
+
+  private static final transient Logger LOG = 
LoggerFactory.getLogger(SnapshotUtils.class);
+
+  public static final String OLD_SNAPSHOT = "replOld";
+  public static final String NEW_SNAPSHOT = "replNew";
+
+  /**
+   * Gets a DistributedFileSystem object if possible from a path.
+   * @param path path from which DistributedFileSystem needs to be extracted.
+   * @param conf Hive Configuration.
+   * @return DFS or null.
+   */
+  public static DistributedFileSystem getDFS(Path path, HiveConf conf) throws 
IOException {
+FileSystem fs = path.getFileSystem(conf);
+if (fs instanceof DistributedFileSystem) {
+  return (DistributedFileSystem) fs;
+} else {
+  LOG.error("FileSystem for {} is not DistributedFileSystem", path);
+  throw new IOException("The filesystem for path {} is {}, The filesystem 
should be DistributedFileSystem to "
+  + "support snapshot based copy.");
+}
+  }
+
+  /**
+   *  Checks whether a given snapshot exists or not.
+   * @param dfs DistributedFileSystem.
+   * @param path path of snapshot.
+   * @param snapshotPrefix snapshot name prefix.
+   * @param snapshotName name of snapshot.
+   * @param conf Hive configuration.
+   * @return true if the snapshot exists.
+   * @throws IOException in case of any error.
+   */
+  public static boolean isSnapshotAvailable(DistributedFileSystem dfs, Path 
path, String snapshotPrefix,
+  String snapshotName, HiveConf conf) throws IOException {
+AtomicBoolean isSnapAvlb = new AtomicBoolean(false);
+Retryable retryable = 
Retryable.builder().withHiveConf(conf).withRetryOnException(IOException.class)
+.withFailOnException(SnapshotException.class).build();
+try {
+  retryable.executeCallable(() -> {
+isSnapAvlb
+.set(dfs.exists(new Path(path, HdfsConstants.DOT_SNAPSHOT_DIR + 
"/" + snapshotPrefix + snapshotName)));
+LOG.debug("Snapshot for path {} 

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: 
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/ReplicationTestUtils.java
##
@@ -544,12 +544,13 @@ public static void assertExternalFileList(List 
expected, String dumploca
 Set tableNames = new HashSet<>();
 for (String line = reader.readLine(); line != null; line = 
reader.readLine()) {
   String[] components = line.split(DirCopyWork.URI_SEPARATOR);
-  Assert.assertEquals("The file should have 
sourcelocation#targetlocation#tblName",
-  3, components.length);
+  Assert.assertEquals("The file should have 
sourcelocation#targetlocation#tblName#copymode", 5,

Review comment:
   For all external tables, Others we don't need. External Tables: Either 
will follow snapshot based repl(INITIAL/DIFF) or not(FALLBACK)




-- 
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: 592943)
Time Spent: 4h 10m  (was: 4h)

> 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: 4h 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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/DirCopyTask.java
##
@@ -218,4 +219,62 @@ public String getName() {
   public boolean canExecuteInParallel() {
 return true;
   }
+
+  boolean copyUsingDistCpSnapshots(Path sourcePath, Path targetPath, 
UserGroupInformation proxyUser) throws IOException {
+
+DistributedFileSystem targetFs = SnapshotUtils.getDFS(targetPath, conf);
+boolean result = false;
+if 
(getWork().getCopyMode().equals(SnapshotUtils.SnapshotCopyMode.DIFF_COPY)) {
+  LOG.info("Using snapshot diff copy for source: {} and target: {}", 
sourcePath, targetPath);
+   result = FileUtils
+  .distCpWithSnapshot(firstSnapshot(work.getSnapshotPrefix()), 
secondSnapshot(work.getSnapshotPrefix()),
+  Collections.singletonList(sourcePath), targetPath, proxyUser,
+  conf, ShimLoader.getHadoopShims());
+   if(result) {
+ // Delete the older snapshot from last iteration.
+ targetFs.deleteSnapshot(targetPath, 
firstSnapshot(work.getSnapshotPrefix()));
+   } else {
+ throw new IOException(
+ "Can not successfully copy external table data using snapshot 
diff. source:" + sourcePath + " and target: "
+ + targetPath);
+   }
+} else if 
(getWork().getCopyMode().equals(SnapshotUtils.SnapshotCopyMode.INITIAL_COPY)) {
+  LOG.info("Using snapshot initial copy for source: {} and target: {}", 
sourcePath, targetPath);
+  // Get the path relative to the initial snapshot for copy.
+  Path snapRelPath =
+  new Path(sourcePath, HdfsConstants.DOT_SNAPSHOT_DIR + "/" + 
secondSnapshot(work.getSnapshotPrefix()));
+
+  // This is the first time we are copying, check if the target is 
snapshottable or not, if not attempt to allow
+  // snapshots.
+  SnapshotUtils.allowSnapshot(targetFs, targetPath, conf);
+  // Attempt to delete the snapshot, in case this is a bootstrap post a 
failed incremental, Since in case of
+  // bootstrap we go from start, so delete any pre-existing snapshot.
+  SnapshotUtils.deleteSnapshotSafe(targetFs, targetPath, 
firstSnapshot(work.getSnapshotPrefix()));
+
+  // Copy from the initial snapshot path.
+  result = runFallbackDistCp(snapRelPath, targetPath, proxyUser);
+}
+
+// Create a new snapshot at target Filesystem. For the next iteration.
+if (result) {
+  SnapshotUtils.createSnapshot(targetFs, targetPath, 
firstSnapshot(work.getSnapshotPrefix()), conf);
+}
+return result;
+  }
+
+  private boolean runFallbackDistCp(Path sourcePath, Path targetPath, 
UserGroupInformation proxyUser)
+  throws IOException {
+ // do we create a new conf and only here provide this additional option 
so that we get away from

Review comment:
   This was there already, Showing up again due to refactor




-- 
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: 592942)
Time Spent: 4h  (was: 3h 50m)

> 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: 4h
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplExternalTables.java
##
@@ -162,7 +194,77 @@ 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).convertToString());
+fileList.add(new DirCopyWork(tableName, sourcePath, targetPath, copyMode, 
snapshotPrefix).convertToString());
+  }
+
+  private SnapshotUtils.SnapshotCopyMode createSnapshotsAtSource(Path 
sourcePath, String snapshotPrefix,
+  boolean isSnapshotEnabled, HiveConf conf, 
SnapshotUtils.ReplSnapshotCount replSnapshotCount, FileList snapPathFileList,
+  ArrayList 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.deleteSnapshotSafe(sourceDfs, sourcePath, 
firstSnapshot(snapshotPrefix));

Review comment:
   Changed behaviour, If it exists it will delete, and if the deletion 
fails, replication fails




-- 
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: 592940)
Time Spent: 3h 50m  (was: 3h 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: 3h 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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: 
shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
##
@@ -1197,6 +1224,103 @@ public boolean runDistCp(List srcPaths, Path dst, 
Configuration conf) thro
 }
   }
 
+  public boolean runDistCpWithSnapshots(String snap1, String snap2, List 
srcPaths, Path dst, Configuration conf)
+  throws IOException {
+DistCpOptions options = new DistCpOptions.Builder(srcPaths, 
dst).withSyncFolder(true).withUseDiff(snap1, snap2)
+
.preserve(FileAttribute.BLOCKSIZE).preserve(FileAttribute.XATTR).build();
+
+List params = constructDistCpWithSnapshotParams(srcPaths, dst, 
snap1, snap2, 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, snap1, snap2, 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, snap1)) {
+  LOG.warn("Copy failed due to target modified. Attempting to restore 
back the target. source: {} target: {} "
+  + "snapshot: {}", srcPaths, dst, snap1);
+  List rParams = constructDistCpWithSnapshotParams(srcPaths, 
dst, ".", snap1, 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, snap1);
+dst.getFileSystem(conf).deleteSnapshot(dst, snap1);
+dst.getFileSystem(conf).createSnapshot(dst, snap1);
+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, snap1, 
snap2, params, returnCode);
+  return false;
+}
+  }
+}
+  }
+} catch (Exception e) {
+  throw new IOException("Cannot execute DistCp process: " + e, e);

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: 592937)
Time Spent: 3.5h  (was: 3h 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: 3.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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-05-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: 
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbstractMessageHandler.java
##
@@ -22,11 +22,14 @@
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.parse.repl.load.UpdatedMetaDataTracker;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.HashSet;
 import java.util.Set;
 
 abstract class AbstractMessageHandler implements MessageHandler {
+  static final Logger LOG = 
LoggerFactory.getLogger(AbstractMessageHandler.class);

Review comment:
   Removed

##
File path: 
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/DropTableHandler.java
##
@@ -18,7 +18,9 @@
 package org.apache.hadoop.hive.ql.parse.repl.dump.events;
 
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.hadoop.hive.metastore.api.Table;

Review comment:
   Removed




-- 
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: 592938)
Time Spent: 3h 40m  (was: 3.5h)

> 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: 3h 40m
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-04-29 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 29/Apr/21 13:49
Start Date: 29/Apr/21 13:49
Worklog Time Spent: 10m 
  Work Description: aasha commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r619969098



##
File path: 
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/ReplicationTestUtils.java
##
@@ -544,12 +544,13 @@ public static void assertExternalFileList(List 
expected, String dumploca
 Set tableNames = new HashSet<>();
 for (String line = reader.readLine(); line != null; line = 
reader.readLine()) {
   String[] components = line.split(DirCopyWork.URI_SEPARATOR);
-  Assert.assertEquals("The file should have 
sourcelocation#targetlocation#tblName",
-  3, components.length);
+  Assert.assertEquals("The file should have 
sourcelocation#targetlocation#tblName#copymode", 5,

Review comment:
   do we need to write copymode for each table location?

##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java
##
@@ -342,7 +343,12 @@ public static PathFilter getBootstrapDirectoryFilter(final 
FileSystem fs) {
 
   public static int handleException(boolean isReplication, Throwable e, String 
nonRecoverablePath,
 ReplicationMetricCollector 
metricCollector, String stageName, HiveConf conf){
-int errorCode = ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode();
+int errorCode;
+if (isReplication && e instanceof SnapshotException) {
+  errorCode = ErrorMsg.getErrorMsg("SNAPSHOT_ERROR").getErrorCode();

Review comment:
   why is this different? Can't we compare based on error msg

##
File path: common/src/java/org/apache/hadoop/hive/common/FileUtils.java
##
@@ -691,6 +691,24 @@ public static boolean distCp(FileSystem srcFS, List 
srcPaths, Path dst,
 return copied;
   }
 
+  public static boolean distCpWithSnapshot(String snap1, String snap2, 
List srcPaths, Path dst,
+  UserGroupInformation proxyUser, HiveConf conf, HadoopShims shims) {
+boolean copied = false;
+try {
+  if (proxyUser == null) {
+copied = shims.runDistCpWithSnapshots(snap1, snap2, srcPaths, dst, 
conf);

Review comment:
   rename to snapshot new and old

##
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplExternalTables.java
##
@@ -162,7 +194,77 @@ 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).convertToString());
+fileList.add(new DirCopyWork(tableName, sourcePath, targetPath, copyMode, 
snapshotPrefix).convertToString());
+  }
+
+  private SnapshotUtils.SnapshotCopyMode createSnapshotsAtSource(Path 
sourcePath, String snapshotPrefix,
+  boolean isSnapshotEnabled, HiveConf conf, 
SnapshotUtils.ReplSnapshotCount replSnapshotCount, FileList snapPathFileList,
+  ArrayList 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.deleteSnapshotSafe(sourceDfs, sourcePath, 
firstSnapshot(snapshotPrefix));

Review comment:
   if deletion fails what happens?

##
File path: 
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AlterDatabaseHandler.java
##
@@ -24,7 +24,7 @@
 
 class AlterDatabaseHandler extends AbstractEventHandler {
 
-  AlterDatabaseHandler(NotificationEvent event) {
+  AlterDatabaseHandler(NotificationEvent event)  {

Review comment:
   remove the space

##
File path: 
shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
##
@@ -1197,6 +1224,103 @@ public boolean runDistCp(List srcPaths, Path dst, 
Configuration conf) thro
 }
   }
 
+  public boolean runDistCpWithSnapshots(String snap1, String snap2, List 
srcPaths, Path dst, Configuration conf)
+  throws IOException {
+DistCpOptions options = new DistCpOptions.Builder(srcPaths, 
dst).withSyncFolder(true).withUseDiff(snap1, snap2)
+
.preserve(FileAttribute.BLOCKSIZE).preserve(FileAttribute.XATTR).build();
+
+List params = constructDistCpWithSnapshotParams(srcPaths, dst, 

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 15:27
Start Date: 24/Mar/21 15:27
Worklog Time Spent: 10m 
  Work Description: ayushtkn commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600591368



##
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,

Review comment:
   This is little tough, `FileUtils` is in hive-common and we are in 
hive-exec, I have dependency on `SnapshotUtils` and `SnapshotUtils` have 
dependency on hive-exec methods from replication. So can't move `SnapshotUtils`
   Will it be ok if I move it to `SnapshotUtils`?




-- 
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: 571247)
Time Spent: 3h 10m  (was: 3h)

> 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: 3h 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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 15:25
Start Date: 24/Mar/21 15:25
Worklog Time Spent: 10m 
  Work Description: ayushtkn commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600589430



##
File path: common/src/java/org/apache/hadoop/hive/common/FileUtils.java
##
@@ -691,6 +690,31 @@ public static boolean distCp(FileSystem srcFS, List 
srcPaths, Path dst,
 return copied;
   }
 
+  public static boolean distCpWithSnapshot(FileSystem srcFs, String snap1,
+  String snap2, List srcPaths, Path dst,
+  UserGroupInformation proxyUser, HiveConf conf, HadoopShims shims) {
+boolean copied;
+try {
+  if (proxyUser == null) {
+copied =
+shims.runDistCpWithSnapshots(snap1, snap2, srcPaths, dst, conf);
+  } else {
+copied = shims
+.runDistCpWithSnapshotsAs(snap1, snap2, srcPaths, dst, conf,
+proxyUser);
+  }
+} catch (Exception e) {
+  LOG.error("Can not copy using snapshot from source: {}, target: {}",
+  srcPaths, dst);
+  copied = false;
+}
+if(copied)

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: 571242)
Time Spent: 3h  (was: 2h 50m)

> 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: 3h
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 15:25
Start Date: 24/Mar/21 15:25
Worklog Time Spent: 10m 
  Work Description: ayushtkn commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600589002



##
File path: 
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java
##
@@ -86,11 +97,45 @@
   // Only database object is updated
   updatedMetadata.set(context.dmd.getEventTo().toString(), actualDbName,
   null, null);
+  processSnapshots(context.hiveConf, oldDb, newDb);
   return Collections.singletonList(alterDbTask);
 } catch (Exception e) {
   throw (e instanceof SemanticException)
   ? (SemanticException) e
   : new SemanticException("Error reading message members", e);
 }
   }
+
+  private void processSnapshots(HiveConf conf, Database before,

Review comment:
   Moved to Snapshot Utils




-- 
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: 571241)
Time Spent: 2h 50m  (was: 2h 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: 2h 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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 15:24
Start Date: 24/Mar/21 15:24
Worklog Time Spent: 10m 
  Work Description: ayushtkn commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600588706



##
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 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 when the Source isn't dfs. for this `if`
   ```
   if (sourceDfs != null) {
   ```
   

##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java
##
@@ -453,4 +456,67 @@ public static Path getLatestDumpPath(Path dumpRoot, 
HiveConf conf) throws IOExce
 }
 return null;
   }
+
+  public static void createSnapshotSafe(FileSystem dfs,
+  Path snapshotPath, String snapshotName) {
+try {
+  dfs.createSnapshot(snapshotPath, snapshotName);
+} catch (IOException e) {
+  LOG.debug("Couldn't create the snapshot {} under path {}", snapshotName,
+  snapshotPath, e);
+}
+  }
+
+  public static void deleteSnapshotSafe(DistributedFileSystem dfs,
+  Path snapshotPath, String snapshotName) {
+try {
+  dfs.deleteSnapshot(snapshotPath, snapshotName);
+} catch (IOException e) {
+  LOG.debug("Couldn't delete the snapshot {} under path {}", snapshotName,
+  snapshotPath, e);
+}
+  }
+
+  public static void renameSnapshotSafe(DistributedFileSystem dfs,
+  Path snapshotPath, String oldSnapshotName, String newSnapshotName) {
+try {
+  dfs.renameSnapshot(snapshotPath, oldSnapshotName, newSnapshotName);
+} catch (IOException e) {
+  LOG.debug("Couldn't rename the snapshot {} to {} under path {}",
+  oldSnapshotName, newSnapshotName, snapshotPath, e);
+}
+  }
+
+  public static 

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HIVE-24852?focusedWorklogId=571230=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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

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_r600582028



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java
##
@@ -453,4 +456,67 @@ public static Path getLatestDumpPath(Path dumpRoot, 
HiveConf conf) throws IOExce
 }
 return null;
   }
+
+  public static void createSnapshotSafe(FileSystem dfs,
+  Path snapshotPath, String snapshotName) {
+try {
+  dfs.createSnapshot(snapshotPath, snapshotName);
+} catch (IOException e) {
+  LOG.debug("Couldn't create the snapshot {} under path {}", snapshotName,
+  snapshotPath, e);
+}
+  }
+
+  public static void deleteSnapshotSafe(DistributedFileSystem dfs,
+  Path snapshotPath, String snapshotName) {
+try {
+  dfs.deleteSnapshot(snapshotPath, snapshotName);
+} catch (IOException e) {
+  LOG.debug("Couldn't delete the snapshot {} under path {}", snapshotName,
+  snapshotPath, e);
+}
+  }
+
+  public static void renameSnapshotSafe(DistributedFileSystem dfs,
+  Path snapshotPath, String oldSnapshotName, String newSnapshotName) {
+try {
+  dfs.renameSnapshot(snapshotPath, oldSnapshotName, newSnapshotName);
+} catch (IOException e) {
+  LOG.debug("Couldn't rename the snapshot {} to {} under path {}",
+  oldSnapshotName, newSnapshotName, snapshotPath, e);
+}
+  }
+
+  public static boolean allowSanapshotSafe(DistributedFileSystem dfs,
+  Path snapshotPath) throws IOException {
+if (dfs.getFileStatus(snapshotPath).isSnapshotEnabled()) {
+  return true;
+} else {
+  try {
+dfs.allowSnapshot(snapshotPath);
+  } catch (Exception e) {
+LOG.error("Cannot allow snapshot on path {}", snapshotPath, e);
+throw e;
+  }
+}
+return true;
+  }
+
+  public static void disAllowSanapshotSafe(DistributedFileSystem dfs,

Review comment:
   Removed




-- 
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: 571231)
Time Spent: 2h 20m  (was: 2h 10m)

> 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 20m
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

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



##
File path: 
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java
##
@@ -86,11 +97,45 @@
   // Only database object is updated
   updatedMetadata.set(context.dmd.getEventTo().toString(), actualDbName,
   null, null);
+  processSnapshots(context.hiveConf, oldDb, newDb);
   return Collections.singletonList(alterDbTask);
 } catch (Exception e) {
   throw (e instanceof SemanticException)
   ? (SemanticException) e
   : new SemanticException("Error reading message members", e);
 }
   }
+
+  private void processSnapshots(HiveConf conf, Database before,

Review comment:
   Refactored to `SnapshotUtils`




-- 
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: 571232)
Time Spent: 2.5h  (was: 2h 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: 2.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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 15:16
Start Date: 24/Mar/21 15:16
Worklog Time Spent: 10m 
  Work Description: ayushtkn commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600581084



##
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##
@@ -667,6 +667,17 @@ private static void populateLlapDaemonVarsSet(Set 
llapDaemonVarsSetLocal
 + " table or partition level. If hive.exec.parallel \n"
 + "is set to true then max worker threads created for copy can be 
hive.exec.parallel.thread.number(determines \n"
 + "number of copy tasks in parallel) * hive.repl.parallel.copy.tasks 
"),
+
REPL_SNAPSHOT_DIFF_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshotdiff.copy",
+false,"Use snapshot diff for copying data from source to "
++ "destination cluster for external table in distcp"),
+
REPL_SNAPSHOT_PREFIX_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshot.prefix",

Review comment:
   Added validation in the code, Extended a test as well to confirm the 
validation. Making it user defined to prevent collision with any other distCp 
job with similar names.




-- 
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: 571228)
Time Spent: 2h  (was: 1h 50m)

> 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
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 15:15
Start Date: 24/Mar/21 15:15
Worklog Time Spent: 10m 
  Work Description: ayushtkn commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600579986



##
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
+Collections.singletonList(sourcePath), // list of source paths
+targetPath, false, proxyUser, conf, ShimLoader.getHadoopShims());
+  }
+} else if (getWork().getCopyMode()
+.equals(SnapshotUtils.SnapshotCopyMode.INITIAL_COPY)) {
+  LOG.info("Using snapshot initial copy for source: {} and target: {}",
+  sourcePath, targetPath);
+  // Try allowSnapshot at target first, if success or already allowed
+  // will use snapshots to copy else fallback.
+  boolean isTargetSnapshottable =
+  SnapshotUtils.allowSnapshot(targetFs, targetPath);
+  if (!isTargetSnapshottable) {
+// We can not allow creating snapshot at target, so no point moving
+// to snapshot mode of copy, fallback to normal copy.
+LOG.error("Can not copy from initial snapshot directory for source: {} 
"
++ "and target: {}. Since target is not snapshottable. Falling "
++ "back to normal copy.", sourcePath, targetPath);
+FileUtils.distCp(sourcePath.getFileSystem(conf), // source file system
+Collections.singletonList(sourcePath), // list of source paths
+targetPath, false, proxyUser, conf, ShimLoader.getHadoopShims());
+// Returning to avoid creating snapshots at target.
+return;
+  } else {
+// Get the path relative to the initial snapshot for copy.
+Path snapRelPath = new Path(sourcePath,
+HdfsConstants.DOT_SNAPSHOT_DIR + "/" + prefix + "initial");
+boolean result = FileUtils.distCp(sourcePath.getFileSystem(conf), //
+// source file system
+Collections.singletonList(snapRelPath), // source path relative to
+// snapshot
+targetPath, false, proxyUser, conf, ShimLoader.getHadoopShims());
+if (!result) {
+  LOG.error(
+  "Can not copy from initial snapshot directory for source: {} "
+  + "and target: {}. Falling back to normal copy.", 
snapRelPath,
+  targetPath);
+  FileUtils.distCp(sourcePath.getFileSystem(conf), // source file 
system
+  Collections.singletonList(sourcePath), // list of source paths
+  targetPath, false, proxyUser, conf, ShimLoader.getHadoopShims());
+}
+ 

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 15:14
Start Date: 24/Mar/21 15:14
Worklog Time Spent: 10m 
  Work Description: ayushtkn commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600578957



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/DirCopyTask.java
##
@@ -162,14 +165,21 @@ public int execute() {
   // do we create a new conf and only here provide this additional 
option so that we get away from
   // differences of data in two location for the same directories ?
   // basically add distcp.options.delete to hiveconf new object ?
-  FileUtils.distCp(
-sourcePath.getFileSystem(conf), // source file system
-Collections.singletonList(sourcePath),  // list of source paths
-targetPath,
-false,
-proxyUser,
-conf,
-ShimLoader.getHadoopShims());
+  if (!getWork().getCopyMode()
+  .equals(SnapshotUtils.SnapshotCopyMode.FALLBACK_COPY)) {
+LOG.info("Using Snapshot mode of copy for source: {} and target:"
++ " {}", sourcePath, targetPath);
+copyUsingDistCpSnapshots(sourcePath, targetPath, proxyUser);
+// Use distcp with snapshots for copy.

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: 571225)
Time Spent: 1h 40m  (was: 1.5h)

> 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: 1h 40m
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 15:14
Start Date: 24/Mar/21 15:14
Worklog Time Spent: 10m 
  Work Description: ayushtkn commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600578585



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/AckTask.java
##
@@ -47,6 +50,7 @@ public int execute() {
   }
   Path ackPath = work.getAckFilePath();
   Utils.create(ackPath, conf);
+//  createSnapshot();

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: 571224)
Time Spent: 1.5h  (was: 1h 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: 1.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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 15:13
Start Date: 24/Mar/21 15:13
Worklog Time Spent: 10m 
  Work Description: ayushtkn commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600578439



##
File path: common/src/java/org/apache/hadoop/hive/common/FileUtils.java
##
@@ -691,6 +690,31 @@ public static boolean distCp(FileSystem srcFS, List 
srcPaths, Path dst,
 return copied;
   }
 
+  public static boolean distCpWithSnapshot(FileSystem srcFs, String snap1,
+  String snap2, List srcPaths, Path dst,
+  UserGroupInformation proxyUser, HiveConf conf, HadoopShims shims) {
+boolean copied;
+try {
+  if (proxyUser == null) {
+copied =
+shims.runDistCpWithSnapshots(snap1, snap2, srcPaths, dst, conf);
+  } else {
+copied = shims
+.runDistCpWithSnapshotsAs(snap1, snap2, srcPaths, dst, conf,
+proxyUser);
+  }
+} catch (Exception e) {
+  LOG.error("Can not copy using snapshot from source: {}, target: {}",
+  srcPaths, dst);
+  copied = false;

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: 571222)
Time Spent: 1h 20m  (was: 1h 10m)

> 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: 1h 20m
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 08:02
Start Date: 24/Mar/21 08:02
Worklog Time Spent: 10m 
  Work Description: aasha commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600248724



##
File path: 
ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AlterDatabaseHandler.java
##
@@ -86,11 +97,45 @@
   // Only database object is updated
   updatedMetadata.set(context.dmd.getEventTo().toString(), actualDbName,
   null, null);
+  processSnapshots(context.hiveConf, oldDb, newDb);
   return Collections.singletonList(alterDbTask);
 } catch (Exception e) {
   throw (e instanceof SemanticException)
   ? (SemanticException) e
   : new SemanticException("Error reading message members", e);
 }
   }
+
+  private void processSnapshots(HiveConf conf, Database before,

Review comment:
   can use a common method. done at 2 places.




-- 
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: 570987)
Time Spent: 1h 10m  (was: 1h)

> 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: 1h 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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 08:01
Start Date: 24/Mar/21 08:01
Worklog Time Spent: 10m 
  Work Description: aasha commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600248356



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java
##
@@ -453,4 +456,67 @@ public static Path getLatestDumpPath(Path dumpRoot, 
HiveConf conf) throws IOExce
 }
 return null;
   }
+
+  public static void createSnapshotSafe(FileSystem dfs,
+  Path snapshotPath, String snapshotName) {
+try {
+  dfs.createSnapshot(snapshotPath, snapshotName);
+} catch (IOException e) {
+  LOG.debug("Couldn't create the snapshot {} under path {}", snapshotName,
+  snapshotPath, e);
+}
+  }
+
+  public static void deleteSnapshotSafe(DistributedFileSystem dfs,
+  Path snapshotPath, String snapshotName) {
+try {
+  dfs.deleteSnapshot(snapshotPath, snapshotName);
+} catch (IOException e) {
+  LOG.debug("Couldn't delete the snapshot {} under path {}", snapshotName,
+  snapshotPath, e);
+}
+  }
+
+  public static void renameSnapshotSafe(DistributedFileSystem dfs,
+  Path snapshotPath, String oldSnapshotName, String newSnapshotName) {
+try {
+  dfs.renameSnapshot(snapshotPath, oldSnapshotName, newSnapshotName);
+} catch (IOException e) {
+  LOG.debug("Couldn't rename the snapshot {} to {} under path {}",
+  oldSnapshotName, newSnapshotName, snapshotPath, e);
+}
+  }
+
+  public static boolean allowSanapshotSafe(DistributedFileSystem dfs,
+  Path snapshotPath) throws IOException {
+if (dfs.getFileStatus(snapshotPath).isSnapshotEnabled()) {
+  return true;
+} else {
+  try {
+dfs.allowSnapshot(snapshotPath);
+  } catch (Exception e) {
+LOG.error("Cannot allow snapshot on path {}", snapshotPath, e);
+throw e;
+  }
+}
+return true;
+  }
+
+  public static void disAllowSanapshotSafe(DistributedFileSystem dfs,

Review comment:
   same methods are there in snapshot utils also




-- 
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: 570986)
Time Spent: 1h  (was: 50m)

> 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: 1h
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HIVE-24852?focusedWorklogId=570985=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 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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 06:20
Start Date: 24/Mar/21 06:20
Worklog Time Spent: 10m 
  Work Description: aasha commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600202144



##
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:
   this method is called for each !result. Can be cleaned up




-- 
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: 570963)
Time Spent: 40m  (was: 0.5h)

> 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: 40m
>  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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 24/Mar/21 06:15
Start Date: 24/Mar/21 06:15
Worklog Time Spent: 10m 
  Work Description: aasha commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r600199925



##
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##
@@ -667,6 +667,17 @@ private static void populateLlapDaemonVarsSet(Set 
llapDaemonVarsSetLocal
 + " table or partition level. If hive.exec.parallel \n"
 + "is set to true then max worker threads created for copy can be 
hive.exec.parallel.thread.number(determines \n"
 + "number of copy tasks in parallel) * hive.repl.parallel.copy.tasks 
"),
+
REPL_SNAPSHOT_DIFF_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshotdiff.copy",
+false,"Use snapshot diff for copying data from source to "
++ "destination cluster for external table in distcp"),
+
REPL_SNAPSHOT_PREFIX_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshot.prefix",

Review comment:
   can we have some default value here? Or if its mandatory value, we 
should fail at policy creation itself




-- 
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: 570960)
Time Spent: 0.5h  (was: 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: 0.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)


[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-23 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 23/Mar/21 17:28
Start Date: 23/Mar/21 17:28
Worklog Time Spent: 10m 
  Work Description: aasha commented on a change in pull request #2043:
URL: https://github.com/apache/hive/pull/2043#discussion_r596568836



##
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/AckTask.java
##
@@ -47,6 +50,7 @@ public int execute() {
   }
   Path ackPath = work.getAckFilePath();
   Utils.create(ackPath, conf);
+//  createSnapshot();

Review comment:
   remove this

##
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:
   repeated code. can this be added to a common place

##
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
+Collections.singletonList(sourcePath), // list of source paths
+targetPath, false, proxyUser, conf, ShimLoader.getHadoopShims());
+  }
+} else if (getWork().getCopyMode()
+.equals(SnapshotUtils.SnapshotCopyMode.INITIAL_COPY)) {
+  LOG.info("Using snapshot initial copy for source: {} and target: {}",
+  sourcePath, targetPath);
+  // Try allowSnapshot at target first, if success or already allowed
+  // will use snapshots to copy else fallback.
+  boolean isTargetSnapshottable =
+  SnapshotUtils.allowSnapshot(targetFs, targetPath);
+  if (!isTargetSnapshottable) {
+// We can not allow creating snapshot at target, so no point moving
+// to snapshot mode of copy, fallback to normal copy.
+LOG.error("Can not copy from initial snapshot directory for source: {} 
"
++ "and target: {}. Since target is not snapshottable. Falling "
++ "back to normal copy.", sourcePath, targetPath);
+FileUtils.distCp(sourcePath.getFileSystem(conf), // source file system
+Collections.singletonList(sourcePath), // list of source paths
+targetPath, false, proxyUser, conf, ShimLoader.getHadoopShims());
+// Returning to avoid creating snapshots at target.
+return;
+  } else {
+// Get the path relative to the initial snapshot for copy.
+Path snapRelPath = new Path(sourcePath,
+HdfsConstants.DOT_SNAPSHOT_DIR + "/" + prefix + "initial");
+

[jira] [Work logged] (HIVE-24852) Add support for Snapshots during external table replication

2021-03-07 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on HIVE-24852:
-

Author: ASF GitHub Bot
Created on: 07/Mar/21 19:15
Start Date: 07/Mar/21 19:15
Worklog Time Spent: 10m 
  Work Description: ayushtkn opened a new pull request #2043:
URL: https://github.com/apache/hive/pull/2043


   



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: 562027)
Remaining Estimate: 0h
Time Spent: 10m

> 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
>  Time Spent: 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)