This is an automated email from the ASF dual-hosted git repository.
tomscut pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git
The following commit(s) were added to refs/heads/trunk by this push:
new dd8fd7e4f3f HDFS-17821. Fix SBN repeatedly do checkpoint after fsimage
transfer failure to part of the multiple NNs (#7876)
dd8fd7e4f3f is described below
commit dd8fd7e4f3fc36d5f5679b911e3fca279870d8f0
Author: caozhiqiang <[email protected]>
AuthorDate: Wed Oct 22 17:03:01 2025 +0800
HDFS-17821. Fix SBN repeatedly do checkpoint after fsimage transfer failure
to part of the multiple NNs (#7876)
Reviewed-by: Tao Li <[email protected]>
Signed-off-by: Tao Li <[email protected]>
---
.../server/namenode/ha/StandbyCheckpointer.java | 2 +-
.../server/namenode/ha/TestStandbyCheckpoints.java | 47 +++++++++++++++++++++-
2 files changed, 46 insertions(+), 3 deletions(-)
diff --git
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
index 8426bbe3302..e240921f670 100644
---
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
+++
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
@@ -342,7 +342,7 @@ public TransferFsImage.TransferResult call()
throw ie;
}
- if (!ioes.isEmpty()) {
+ if (ioes.size() > activeNNAddresses.size() / 2) {
throw MultipleIOException.createIOException(ioes);
}
}
diff --git
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
index 01d461ade4d..6f36d5e19bc 100644
---
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
+++
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
@@ -721,8 +721,51 @@ private void doCreate() throws IOException {
out.write(42);
out.close();
}
-
-
+
+ /**
+ * Test checkpoint still succeeds when no more than half of the fsimages
upload failed.
+ */
+ @Test
+ @Timeout(value = 300)
+ public void testPutFsimagePartFailed() throws Exception {
+ for (int i = 1; i < NUM_NNS; i++) {
+ cluster.shutdownNameNode(i);
+
+ // Make true checkpoint for DFS_NAMENODE_CHECKPOINT_PERIOD_KEY
+
cluster.getConfiguration(i).setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY,
3);
+
cluster.getConfiguration(i).setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY,
1000);
+ }
+ doEdits(0, 10);
+ cluster.transitionToStandby(0);
+
+ for (int i = 1; i < NUM_NNS; i++) {
+ cluster.restartNameNode(i, false);
+ }
+ cluster.waitClusterUp();
+ setNNs();
+
+ for (int i = 0; i < NUM_NNS; i++) {
+ // Once the standby catches up, it should do a checkpoint
+ // and save to local directories.
+ HATestUtil.waitForCheckpoint(cluster, i, ImmutableList.of(12));
+ }
+
+ long snnCheckpointTime1 =
nns[1].getNamesystem().getStandbyLastCheckpointTime();
+ cluster.transitionToActive(0);
+ cluster.transitionToObserver(2);
+ cluster.shutdownNameNode(2);
+
+ doEdits(11, 20);
+ nns[0].getRpcServer().rollEditLog();
+ HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(23));
+
+ long snnCheckpointTime2 =
nns[1].getNamesystem().getStandbyLastCheckpointTime();
+
+ // Make sure that standby namenode checkpoint success and update the
lastCheckpointTime
+ // even though it send fsimage to nn2 failed because nn2 is shut down.
+ assertTrue(snnCheckpointTime2 > snnCheckpointTime1);
+ }
+
/**
* A codec which just slows down the saving of the image significantly
* by sleeping a few milliseconds on every write. This makes it easy to
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]