ayushtkn commented on a change in pull request #2539:
URL: https://github.com/apache/hive/pull/2539#discussion_r724067080
##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
##########
@@ -1105,17 +1105,13 @@ Long bootStrapDump(Path dumpRoot, DumpMetaData dmd,
Path cmRoot, Hive hiveDb)
boolean isExternalTablePresent = false;
String snapshotPrefix = dbName.toLowerCase();
- ArrayList<String> prevSnaps = new ArrayList<>(); // Will stay empty
in case of bootstrap
+ ArrayList<String> prevSnaps = new ArrayList<>();
if (isSnapshotEnabled) {
- // Delete any old existing snapshot file, We always start fresh in
case of bootstrap.
-
FileUtils.deleteIfExists(getDFS(SnapshotUtils.getSnapshotFileListPath(dumpRoot),
conf),
- new Path(SnapshotUtils.getSnapshotFileListPath(dumpRoot),
- EximUtil.FILE_LIST_EXTERNAL_SNAPSHOT_CURRENT));
-
FileUtils.deleteIfExists(getDFS(SnapshotUtils.getSnapshotFileListPath(dumpRoot),
conf),
Review comment:
Check if we need to preserve
##########
File path:
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosUsingSnapshots.java
##########
@@ -201,7 +218,10 @@ public void testBasicReplicationWithSnapshots() throws
Throwable {
public void testBasicStartFromIncrementalReplication() throws Throwable {
// Run a cycle of dump & load with snapshot disabled.
- ArrayList<String> withClause = new ArrayList<>(1);
+ ArrayList<String> withClause = new ArrayList<>(3);
+ ArrayList<String> withClause2 = new ArrayList<>(3);
+ withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" +
primary.repldDir + "'");
+ withClause2.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" +
primary.repldDir + "'");
Review comment:
Give a check why twice?
##########
File path:
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplExternalTables.java
##########
@@ -192,64 +196,138 @@ private void dirLocationToCopy(String tableName,
FileList fileList, Path sourceP
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<String> prevSnaps, boolean isBootstrap) throws IOException {
+ private Map<String, SnapshotUtils.SnapshotCopyMode>
createSnapshotsAtSource(Path sourcePath, Path targetPath, String snapshotPrefix,
+
boolean isSnapshotEnabled, HiveConf conf, SnapshotUtils.ReplSnapshotCount
replSnapshotCount, FileList snapPathFileList,
+
ArrayList<String> prevSnaps, boolean isBootstrap) throws IOException {
+ Map<String, SnapshotUtils.SnapshotCopyMode> ret = new HashMap<>();
+ ret.put(snapshotPrefix, FALLBACK_COPY);
if (!isSnapshotEnabled) {
LOG.info("Snapshot copy not enabled for path {} Will use normal distCp
for copying data.", sourcePath);
- return FALLBACK_COPY;
+ return ret;
}
+ String prefix = snapshotPrefix;
+ SnapshotUtils.SnapshotCopyMode copyMode = FALLBACK_COPY;
DistributedFileSystem sourceDfs = SnapshotUtils.getDFS(sourcePath, conf);
try {
- if(isBootstrap) {
- // Delete any pre existing snapshots.
- SnapshotUtils.deleteSnapshotIfExists(sourceDfs, sourcePath,
firstSnapshot(snapshotPrefix), conf);
- SnapshotUtils.deleteSnapshotIfExists(sourceDfs, sourcePath,
secondSnapshot(snapshotPrefix), conf);
- allowAndCreateInitialSnapshot(sourcePath, snapshotPrefix, conf,
replSnapshotCount, snapPathFileList, sourceDfs);
- return INITIAL_COPY;
+ if(conf.getBoolVar(HiveConf.ConfVars.REPL_REUSE_SNAPSHOTS)) {
+ try {
+ FileStatus[] listing = sourceDfs.listStatus(new Path(sourcePath,
".snapshot"));
+ for (FileStatus elem : listing) {
+ String snapShotName = elem.getPath().getName();
+ if (snapShotName.contains(OLD_SNAPSHOT)) {
+ prefix = snapShotName.substring(0,
snapShotName.lastIndexOf(OLD_SNAPSHOT));
+ break;
+ }
+ if (snapShotName.contains(NEW_SNAPSHOT)) {
+ prefix = snapShotName.substring(0,
snapShotName.lastIndexOf(NEW_SNAPSHOT));
+ break;
+ }
+ }
+ ret.clear();
+ ret.put(prefix, copyMode);
+ snapshotPrefix = prefix;
+ } catch (SnapshotException e) {
+ //dir not snapshottable, continue
+ }
}
+ boolean firstSnapAvailable =
+ SnapshotUtils.isSnapshotAvailable(sourceDfs, sourcePath,
snapshotPrefix, OLD_SNAPSHOT, conf);
+ boolean secondSnapAvailable =
+ SnapshotUtils.isSnapshotAvailable(sourceDfs, sourcePath,
snapshotPrefix, NEW_SNAPSHOT, conf);
+ //While resuming a failed replication
if (prevSnaps.contains(sourcePath.toString())) {
// We already created a snapshot for this, just refresh the latest
snapshot and leave.
- sourceDfs.deleteSnapshot(sourcePath, secondSnapshot(snapshotPrefix));
+ // In case of reverse replication after failover, in some paths,
second snapshot may not be present.
+ SnapshotUtils.deleteSnapshotIfExists(sourceDfs, sourcePath,
secondSnapshot(snapshotPrefix), conf);
replSnapshotCount.incrementNumDeleted();
SnapshotUtils.createSnapshot(sourceDfs, sourcePath,
secondSnapshot(snapshotPrefix), conf);
- replSnapshotCount.incrementNumCreated();
Review comment:
revert this
##########
File path:
ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/SnapshotUtils.java
##########
@@ -275,17 +275,17 @@ public static void renameSnapshot(FileSystem fs, Path
snapshotPath, String sourc
/**
* Deletes the snapshots present in the list.
- * @param dfs DistributedFileSystem.
* @param diffList Elements to be deleted.
* @param prefix Prefix used in snapshot names,
* @param snapshotCount snapshot counter to track the number of snapshots
deleted.
* @param conf the Hive Configuration.
* @throws IOException in case of any error.
*/
- private static void cleanUpSnapshots(DistributedFileSystem dfs,
ArrayList<String> diffList, String prefix,
+ private static void cleanUpSnapshots(ArrayList<String> diffList, String
prefix,
ReplSnapshotCount snapshotCount, HiveConf conf) throws IOException {
for (String path : diffList) {
Path snapshotPath = new Path(path);
+ DistributedFileSystem dfs = (DistributedFileSystem)
snapshotPath.getFileSystem(conf);
Review comment:
It is creating DFS in loop, can make it create only once
##########
File path: common/src/java/org/apache/hadoop/hive/common/FileUtils.java
##########
@@ -705,6 +705,16 @@ public static boolean distCpWithSnapshot(String
oldSnapshot, String newSnapshot,
oldSnapshot, newSnapshot);
} catch (IOException e) {
LOG.error("Can not copy using snapshot from source: {}, target: {}",
srcPaths, dst);
+ try {
+ // in case overwriteTarget is set to false, and we encounter an
exception due to targetFs getting
+ // changed since last snapshot, then fallback to initial copy
+ if (!overwriteTarget && !e.getCause().getMessage().contains("changed
since snapshot")) {
+ LOG.warn("Diff copy failed due to changed target filesystem, falling
back to normal distcp.");
+ return distCp(srcPaths.get(0).getFileSystem(conf), srcPaths, dst,
false, proxyUser, conf, shims);
Review comment:
use the snapshot to copy, you can keep it where rdiff is being done
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]