This is an automated email from the ASF dual-hosted git repository.
okumin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/master by this push:
new 6c6fb28d5c8 HIVE-28608: Resolve Issues in ReplicationMigrationTool and
Activate Test Suite (#5595) (Indhumathi Muthumurugesh, reviewed by Shohei
Okumiya)
6c6fb28d5c8 is described below
commit 6c6fb28d5c85d38ffb51757681ed1200190addd3
Author: Indhumathi <[email protected]>
AuthorDate: Thu Jan 16 16:41:19 2025 +0530
HIVE-28608: Resolve Issues in ReplicationMigrationTool and Activate Test
Suite (#5595) (Indhumathi Muthumurugesh, reviewed by Shohei Okumiya)
---
.../ql/parse/TestReplicationMigrationTool.java | 25 +++++++++-------------
.../ql/exec/repl/ReplicationMigrationTool.java | 6 ++++--
2 files changed, 14 insertions(+), 17 deletions(-)
diff --git
a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationMigrationTool.java
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationMigrationTool.java
index dccd5316bb0..0f140230949 100644
---
a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationMigrationTool.java
+++
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationMigrationTool.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.util.ToolRunner;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
-import org.junit.Ignore;
import org.junit.Test;
import java.io.ByteArrayOutputStream;
@@ -46,8 +45,6 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
-// Enable the test post fixing HIVE-28608
-@Ignore
public class TestReplicationMigrationTool extends
BaseReplicationAcrossInstances {
String extraPrimaryDb;
@@ -250,16 +247,18 @@ public class TestReplicationMigrationTool extends
BaseReplicationAcrossInstances
// Open a file and check if it gets caught by the verify open file option.
FSDataOutputStream stream = fs.append(new Path(externalTableLocationa,
"filea1.txt"));
- ToolRunner.run(conf, replTool,
+ int result = ToolRunner.run(conf, replTool,
new String[] { "-dumpFilePath", tuple.dumpLocation, "-fileLevelCheck",
"-verifyOpenFiles" });
- fail("Script didn't fail despite having an open file.");
+ if (result != -1) {
+ fail("Script didn't fail despite having an open file.");
+ }
// Make sure we get the exception.
assertTrue(err.toString(), err.toString().contains("There are open
files"));
- err.reset();
- out.reset();
// Confirm that successful message is not printed, and failure message is
printed.
assertFalse(out.toString(), out.toString().contains("Completed
verification. Source & Target are in Sync."));
assertTrue(out.toString(), out.toString().contains("Completed
verification. Source & Target are not in Sync."));
+ err.reset();
+ out.reset();
// Close the file and check, the script should return success.
stream.close();
@@ -334,9 +333,11 @@ public class TestReplicationMigrationTool extends
BaseReplicationAcrossInstances
private void validateChecksumValidationFails(ReplicationMigrationTool
replTool, WarehouseInstance.Tuple tuple,
String message) throws Exception {
- ToolRunner.run(conf, replTool,
+ int result = ToolRunner.run(conf, replTool,
new String[] { "-dumpFilePath", tuple.dumpLocation, "-fileLevelCheck",
"-verifyChecksum" });
- fail("Script didn't fail despite having an extra file.");
+ if (result != -1) {
+ fail("Script didn't fail despite having an extra file.");
+ }
// Make sure we get the exception.
assertTrue(err.toString(), err.toString().contains(message));
// Confirm that successful message is not printed.
@@ -388,24 +389,18 @@ public class TestReplicationMigrationTool extends
BaseReplicationAcrossInstances
assertEquals(0,
ToolRunner.run(conf, replTool, new String[] { "-dumpFilePath",
tuple.dumpLocation +"/hive/_file_list_external", "-dirLevelCheck" }));
assertTrue(out.toString().contains("Completed verification"));
- assertFalse(err.toString(), err.toString().isEmpty());
out.reset();
- err.reset();
// Verify at file level.
assertEquals(0, ToolRunner
.run(conf, replTool, new String[] { "-dumpFilePath",
tuple.dumpLocation + "/hive", "-fileLevelCheck" }));
assertTrue(out.toString().contains("Completed verification."));
- assertFalse(err.toString(), err.toString().isEmpty());
out.reset();
- err.reset();
// Verify at file level, with checksum.
assertEquals(0, ToolRunner.run(conf, replTool,
new String[] { "-dumpFilePath", tuple.dumpLocation, "-fileLevelCheck",
"-verifyChecksum" }));
assertTrue(out.toString().contains("Completed verification. Source &
Target are in Sync."));
- assertFalse(err.toString(), err.toString().isEmpty());
out.reset();
- err.reset();
}
}
diff --git
a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplicationMigrationTool.java
b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplicationMigrationTool.java
index 29d336d9012..02531567faa 100644
---
a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplicationMigrationTool.java
+++
b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplicationMigrationTool.java
@@ -155,6 +155,9 @@ public class ReplicationMigrationTool implements Tool {
System.out.println("Completed verification. Source & Target are " +
(failed == 0 ? "in Sync." : "not in Sync."));
System.out.println("Time Taken: " + (System.currentTimeMillis() -
startTime) + " ms");
+ if (failed != 0) {
+ return -1;
+ }
} catch (UnsupportedOperationException e) {
System.err.println(e.getMessage());
System.err.println(help);
@@ -256,7 +259,7 @@ public class ReplicationMigrationTool implements Tool {
// If there is even single open file we can abort.
if (srcDFS.listOpenFiles(EnumSet.of(ALL_OPEN_FILES),
Path.getPathWithoutSchemeAndAuthority(srcPath).toString())
.hasNext()) {
- System.out.println("There are open files in " + srcPath);
+ System.err.println("There are open files in " + srcPath);
return false;
} else {
LOG.error("Open file check is ignored since the source filesystem is
not of type of "
@@ -359,7 +362,6 @@ public class ReplicationMigrationTool implements Tool {
LocatedFileStatus sourceFile = srcListing.next();
if (filtersPattern != null && !isCopied(sourceFile.getPath(),
filtersPattern)) {
LOG.info("Entry: {} is filtered.", sourceFile.getPath());
- continue;
} else {
System.err.println("Extra entry at source: " + sourceFile.getPath());
return false;