-
Notifications
You must be signed in to change notification settings - Fork 589
HDDS-11807. Make callId different for each request in openKeyCleanupService #7551
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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()); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added one hsync and one normal key.
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, | ||
|
|
@@ -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(); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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; | ||
|
|
@@ -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; | ||
ashishkumar50 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| private final AtomicBoolean suspended; | ||
|
|
||
| public OpenKeyCleanupService(long interval, TimeUnit unit, long timeout, | ||
|
|
@@ -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); | ||
| } | ||
|
|
||
|
|
@@ -244,6 +247,7 @@ private OMRequest createCommitKeyRequest( | |
| .setCmdType(Type.CommitKey) | ||
| .setCommitKeyRequest(request) | ||
| .setClientId(clientId.toString()) | ||
| .setVersion(ClientVersion.CURRENT_VERSION) | ||
| .build(); | ||
| } | ||
|
|
||
|
|
@@ -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()); | ||
|
||
| } catch (ServiceException e) { | ||
| LOG.error("Open key " + omRequest.getCmdType() | ||
| + " request failed. Will retry at next run.", e); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.