diff --git a/hadoop-ozone/dist/src/main/smoketest/debug/ozone-debug-lease-recovery.robot b/hadoop-ozone/dist/src/main/smoketest/debug/ozone-debug-lease-recovery.robot index 1f9646579dc1..a721f2acbbe6 100644 --- a/hadoop-ozone/dist/src/main/smoketest/debug/ozone-debug-lease-recovery.robot +++ b/hadoop-ozone/dist/src/main/smoketest/debug/ozone-debug-lease-recovery.robot @@ -36,12 +36,12 @@ Create volume bucket and put key *** Test Cases *** Test ozone debug recover for o3fs ${result} = Execute Lease recovery cli o3fs://${BUCKET}.${VOLUME}.om/${TESTFILE} - Should Contain ${result} Key: ${TESTFILE} is already closed + Should Contain ${result} Lease recovery SUCCEEDED ${result} = Execute Lease recovery cli o3fs://${BUCKET}.${VOLUME}.om/randomfile Should Contain ${result} not found Test ozone debug recover for ofs ${result} = Execute Lease recovery cli ofs://om/${VOLUME}/${BUCKET}/${TESTFILE} - Should Contain ${result} Key: ${TESTFILE} is already closed + Should Contain ${result} Lease recovery SUCCEEDED ${result} = Execute Lease recovery cli ofs://om/${VOLUME}/${BUCKET}/randomfile Should Contain ${result} not found diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java index 1cd8a7e2b4d1..a16114671832 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestLeaseRecovery.java @@ -150,6 +150,9 @@ public void testRecovery() throws Exception { } // The lease should have been recovered. assertTrue("File should be closed", fs.isFileClosed(file)); + + // A second call to recoverLease should succeed too. + assertTrue(fs.recoverLease(file)); } finally { closeIgnoringKeyNotFound(stream); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java index 1351d028b724..ef4fe1456fd2 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/debug/TestLeaseRecoverer.java @@ -146,5 +146,9 @@ public void testCLI() throws IOException { // make sure length remains the same fileStatus = fs.getFileStatus(file); assertEquals(dataSize, fileStatus.getLen()); + + // recover the same file second time should succeed + cmd.execute(args); + assertEquals("", stderr.toString()); } } diff --git a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java index 8f977167a2d3..1369fcc32105 100644 --- a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java +++ b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.annotation.InterfaceStability; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.security.token.DelegationTokenIssuer; @@ -132,7 +133,16 @@ public boolean recoverLease(Path f) throws IOException { LOG.trace("recoverLease() path:{}", f); Path qualifiedPath = makeQualified(f); String key = pathToKey(qualifiedPath); - List infoList = getAdapter().recoverFilePrepare(key); + List infoList = null; + try { + infoList = getAdapter().recoverFilePrepare(key); + } catch (OMException e) { + if (e.getResult() == OMException.ResultCodes.KEY_ALREADY_CLOSED) { + // key is already closed, let's just return success + return true; + } + throw e; + } // TODO: query DN to get the final block length OmKeyInfo keyInfo = infoList.get(0); OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(keyInfo.getVolumeName()) diff --git a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java index 0530b606b299..71d55f434835 100644 --- a/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java +++ b/hadoop-ozone/ozonefs-hadoop3/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java @@ -29,6 +29,7 @@ import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.security.token.DelegationTokenIssuer; @@ -138,7 +139,16 @@ public boolean recoverLease(final Path f) throws IOException { LOG.trace("recoverLease() path:{}", f); Path qualifiedPath = makeQualified(f); String key = pathToKey(qualifiedPath); - List infoList = getAdapter().recoverFilePrepare(key); + List infoList = null; + try { + infoList = getAdapter().recoverFilePrepare(key); + } catch (OMException e) { + if (e.getResult() == OMException.ResultCodes.KEY_ALREADY_CLOSED) { + // key is already closed, let's just return success + return true; + } + throw e; + } // TODO: query DN to get the final block length OmKeyInfo keyInfo = infoList.get(0); OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(keyInfo.getVolumeName()) diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java index a31a07cf6763..488d45da1c19 100644 --- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java +++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.annotation.InterfaceStability; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.security.token.DelegationTokenIssuer; @@ -132,7 +133,16 @@ public boolean recoverLease(Path f) throws IOException { LOG.trace("isFileClosed() path:{}", f); Path qualifiedPath = makeQualified(f); String key = pathToKey(qualifiedPath); - List infoList = getAdapter().recoverFilePrepare(key); + List infoList = null; + try { + infoList = getAdapter().recoverFilePrepare(key); + } catch (OMException e) { + if (e.getResult() == OMException.ResultCodes.KEY_ALREADY_CLOSED) { + // key is already closed, let's just return success + return true; + } + throw e; + } // TODO: query DN to get the final block length OmKeyInfo keyInfo = infoList.get(0); OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(keyInfo.getVolumeName()) diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java index ed62f608b4ae..c501e0652eaa 100644 --- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java +++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/RootedOzoneFileSystem.java @@ -29,6 +29,7 @@ import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.security.token.DelegationTokenIssuer; @@ -131,7 +132,16 @@ public boolean recoverLease(final Path f) throws IOException { LOG.trace("recoverLease() path:{}", f); Path qualifiedPath = makeQualified(f); String key = pathToKey(qualifiedPath); - List infoList = getAdapter().recoverFilePrepare(key); + List infoList = null; + try { + infoList = getAdapter().recoverFilePrepare(key); + } catch (OMException e) { + if (e.getResult() == OMException.ResultCodes.KEY_ALREADY_CLOSED) { + // key is already closed, let's just return success + return true; + } + throw e; + } // TODO: query DN to get the final block length OmKeyInfo keyInfo = infoList.get(0); OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(keyInfo.getVolumeName())