Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -496,6 +496,54 @@ public void testOfsHSync(boolean incrementalChunkList) throws Exception {
}
}

@Test
public void testHSyncOpenKeyCommitAfterExpiry() throws Exception {
// Set the fs.defaultFS
final String rootPath = String.format("%s://%s/",
OZONE_OFS_URI_SCHEME, CONF.get(OZONE_OM_ADDRESS_KEY));
CONF.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath);

final Path key1 = new Path("hsync-key");
final Path key2 = new Path("hsync-key1");

try (FileSystem fs = FileSystem.get(CONF)) {
// Create key1
try (FSDataOutputStream os = fs.create(key1, true)) {
os.write(1);
os.hsync();
// Create key2
try (FSDataOutputStream os1 = fs.create(key2, true)) {
os1.write(1);
os1.hsync();
// There should be 2 key in openFileTable
assertThat(2 == getOpenKeyInfo(BUCKET_LAYOUT).size());
assertThat(2 == getKeyInfo(BUCKET_LAYOUT).size());

// Resume openKeyCleanupService
openKeyCleanupService.resume();
GenericTestUtils.waitFor(() -> 0 == getOpenKeyInfo(BUCKET_LAYOUT).size(), 1000, 12000);

// Verify entry from openKey gets committed eventually
GenericTestUtils.waitFor(() ->
0 == getOpenKeyInfo(BUCKET_LAYOUT).size(), 1000, 12000);
// Verify key is still present
assertThat(2 == getKeyInfo(BUCKET_LAYOUT).size());

// Clean up
assertTrue(fs.delete(key1, false));
assertTrue(fs.delete(key2, false));
waitForEmptyDeletedTable();
} catch (OMException ex) {
assertEquals(OMException.ResultCodes.KEY_NOT_FOUND, ex.getResult());
Copy link
Contributor

@ChenSammi ChenSammi Dec 10, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ashishkumar50 , if key2 is not explicitly deleted, will os1 still throws KEY_NOT_FOUND?

Given the issue to fix, I would suggest this new test to have one hsynced key and one normal key to cover the case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added one hsync and one normal key.

if key2 is not explicitly deleted, will os1 still throws KEY_NOT_FOUND?

Yes because when os1 is geting closed it will try to commit key and which it will not find and throws KEY_NOT_FOUND error.

}
} catch (OMException ex) {
assertEquals(OMException.ResultCodes.KEY_NOT_FOUND, ex.getResult());
} finally {
openKeyCleanupService.suspend();
}
}
}

@Test
public void testHSyncDeletedKey() throws Exception {
// Verify that a key can't be successfully hsync'ed again after it's deleted,
Expand Down Expand Up @@ -595,6 +643,21 @@ private List<OmKeyInfo> getOpenKeyInfo(BucketLayout bucketLayout) {
return omKeyInfo;
}

private List<OmKeyInfo> getKeyInfo(BucketLayout bucketLayout) {
List<OmKeyInfo> omKeyInfo = new ArrayList<>();

Table<String, OmKeyInfo> openFileTable =
cluster.getOzoneManager().getMetadataManager().getKeyTable(bucketLayout);
try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
iterator = openFileTable.iterator()) {
while (iterator.hasNext()) {
omKeyInfo.add(iterator.next().getValue());
}
} catch (Exception e) {
}
return omKeyInfo;
}

@Test
public void testUncommittedBlocks() throws Exception {
waitForEmptyDeletedTable();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.hadoop.hdds.utils.BackgroundTask;
import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
import org.apache.hadoop.ozone.ClientVersion;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.om.ExpiredOpenKeys;
import org.apache.hadoop.ozone.om.KeyManager;
Expand Down Expand Up @@ -78,6 +79,7 @@ public class OpenKeyCleanupService extends BackgroundService {
private final int cleanupLimitPerTask;
private final AtomicLong submittedOpenKeyCount;
private final AtomicLong runCount;
private final AtomicLong callIdCount;
private final AtomicBoolean suspended;

public OpenKeyCleanupService(long interval, TimeUnit unit, long timeout,
Expand Down Expand Up @@ -113,6 +115,7 @@ public OpenKeyCleanupService(long interval, TimeUnit unit, long timeout,

this.submittedOpenKeyCount = new AtomicLong(0);
this.runCount = new AtomicLong(0);
this.callIdCount = new AtomicLong(0);
this.suspended = new AtomicBoolean(false);
}

Expand Down Expand Up @@ -244,6 +247,7 @@ private OMRequest createCommitKeyRequest(
.setCmdType(Type.CommitKey)
.setCommitKeyRequest(request)
.setClientId(clientId.toString())
.setVersion(ClientVersion.CURRENT_VERSION)
.build();
}

Expand All @@ -265,7 +269,7 @@ private OMRequest createDeleteOpenKeysRequest(

private OMResponse submitRequest(OMRequest omRequest) {
try {
return OzoneManagerRatisUtils.submitRequest(ozoneManager, omRequest, clientId, runCount.get());
return OzoneManagerRatisUtils.submitRequest(ozoneManager, omRequest, clientId, callIdCount.incrementAndGet());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

With the callId used, runCount is of no use now. runCount can be removed.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

runCount is used by multiple tests to get how many times service has run like in testCleanupExpiredOpenKeys.
So i used different name to distinguish it, What is your suggestion?

Copy link
Contributor

@ChenSammi ChenSammi Dec 10, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can testCleanupExpiredOpenKeys be refactored to use callId where runCount is used?

Copy link
Contributor Author

@ashishkumar50 ashishkumar50 Dec 12, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have updated with callId. Also in tests we are already waiting for SubmittedOpenKeyCount which ensures openKeyCleanupService has finished executing, so I think we need not wait again for runCount.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Then can we remove the getRunCount()?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed it now, just was referenced in test logs.

} catch (ServiceException e) {
LOG.error("Open key " + omRequest.getCmdType()
+ " request failed. Will retry at next run.", e);
Expand Down
Loading