Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<OmKeyInfo> infoList = getAdapter().recoverFilePrepare(key);
List<OmKeyInfo> 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())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<OmKeyInfo> infoList = getAdapter().recoverFilePrepare(key);
List<OmKeyInfo> 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())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<OmKeyInfo> infoList = getAdapter().recoverFilePrepare(key);
List<OmKeyInfo> 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())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<OmKeyInfo> infoList = getAdapter().recoverFilePrepare(key);
List<OmKeyInfo> 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())
Expand Down