Skip to content
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

When call openLedgerOp, make the timeout ex is a separate error code #3562

Merged
merged 8 commits into from
Oct 27, 2022

Conversation

poorbarcode
Copy link
Contributor

@poorbarcode poorbarcode commented Oct 20, 2022

Descriptions of the changes in this PR:

Motivation

When we execute bkClient.openLedger(ledgerId), the execution flow is as follows:

  1. start opening the ledger
  2. get ledger meta
  3. read the last confirmed entry
  4. open ledger success

If we get the correct ledgerMeta at step 2, this means that this ledger has not been deleted. If step 3 times out, we should try again to make sure the ledger exists until we get a clear response from the BK server.(High light)However, in the current implementation, the timeout exception is rewritten as a LedgerRecoveryException, making it impossible to determine whether we should retry.

Log:

Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] ERROR org.apache.bookkeeper.client.ReadLastConfirmedOp - While readLastConfirmed ledger: 59158316 did not hear success responses from all quorums, QuorumCoverage(e:2,w:2,a:2) = [-23, -23]
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [order/org-217/persistent/p1010-tx] Opened ledger 59158316 for consumer order-service. rc=-10

Looking at the ledger metadata:

LedgerMetadata{formatVersion=3, ensembleSize=2, writeQuorumSize=2, ackQuorumSize=2, state=IN_RECOVERY, digestType=CRC32C, password=base64:, ensembles={0=[***:3181, ***:3181]}, customMetadata={component=***, pulsar/managed-ledger=***, pulsar/cursor=***, application=***}}

see also: apache/pulsar#18123

Changes

  • When calling openLedgerOp, do not rewritten TiemoutException as a LedgerRecoveryException
  • add the dependency: junit4-dataprovider
  • use @DataProvider to simpler the test case "testOpenLedgerRecover" & "testOpenLedgerNoRecover"

@poorbarcode
Copy link
Contributor Author

ping @zymap @dlg99 @eolivelli @hangc0276 @shoothzj PTAL. Thanks.

@poorbarcode poorbarcode force-pushed the improve/open_ledger_op_timeout branch 2 times, most recently from 62b42f5 to 7eeb397 Compare October 20, 2022 14:05
@@ -25,6 +25,7 @@
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.*;
Copy link
Contributor

Choose a reason for hiding this comment

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

Please avoid the star import.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Already remove the star import

Comment on lines 378 to 431
@Test
public void testOpenLedgerNoRecoveryWithTimeoutEx() throws Exception {
mockReadEntryTimeout();
LedgerMetadata ledgerMetadata = generateLedgerMetadata(ensembleSize,
writeQuorumSize, ackQuorumSize, password, customMetadata);
registerMockLedgerMetadata(ledgerId, ledgerMetadata);
ledgerMetadata.getAllEnsembles().values().forEach(bookieAddressList -> {
bookieAddressList.forEach(bookieAddress -> {
registerMockEntryForRead(ledgerId, BookieProtocol.LAST_ADD_CONFIRMED, bookieAddress, entryData, -1);
registerMockEntryForRead(ledgerId, 0, bookieAddress, entryData, -1);
});
});
try {
result(newOpenLedgerOp()
.withPassword(ledgerMetadata.getPassword())
.withDigestType(DigestType.CRC32)
.withLedgerId(ledgerId)
.withRecovery(false)
.execute());
fail("Expect timeout error");
} catch (BKException.BKTimeoutException timeoutException) {
// Expect timeout error.
}
// Reset bk client.
resetBKClient();
}

@Test
public void testOpenLedgerRecoveryWithTimeoutEx() throws Exception {
mockReadEntryTimeout();
LedgerMetadata ledgerMetadata = generateLedgerMetadata(ensembleSize,
writeQuorumSize, ackQuorumSize, password, customMetadata);
registerMockLedgerMetadata(ledgerId, ledgerMetadata);

ledgerMetadata.getAllEnsembles().values().forEach(bookieAddressList -> {
bookieAddressList.forEach(bookieAddress -> {
registerMockEntryForRead(ledgerId, BookieProtocol.LAST_ADD_CONFIRMED, bookieAddress, entryData, -1);
registerMockEntryForRead(ledgerId, 0, bookieAddress, entryData, -1);
});
});
try {
result(newOpenLedgerOp()
.withPassword(ledgerMetadata.getPassword())
.withDigestType(DigestType.CRC32)
.withLedgerId(ledgerId)
.withRecovery(true)
.execute());
fail("Expect timeout error");
} catch (BKException.BKTimeoutException timeoutException) {
// Expect timeout error.
}
// Reset bk client.
resetBKClient();
}
Copy link
Contributor

Choose a reason for hiding this comment

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

It's better to add a data provider to reduce the duplicated code.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good idea

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Already use @DataProvider to simpler the test case

Copy link
Contributor

@hangc0276 hangc0276 left a comment

Choose a reason for hiding this comment

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

Nice catch!

if (ex != null) {
LOG.error("Ledger {} read timeout", ledgerId, ex);
}
openComplete(rc, null);
});
} else {
openComplete(bk.getReturnRc(BKException.Code.LedgerRecoveryException), null);
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we need to close the ledgerHandle when encountering other exceptions?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good suggestion. Already fixed

Copy link
Member

@wenbingshen wenbingshen left a comment

Choose a reason for hiding this comment

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

@poorbarcode Good work. I have only add some minor comments. PTAL.

Comment on lines 212 to 218
} else if (rc == BKException.Code.TimeoutException) {
closeLedgerHandleAsync().whenComplete((r, ex) -> {
if (ex != null) {
LOG.error("Ledger {} read timeout", ledgerId, ex);
}
openComplete(rc, null);
});
Copy link
Member

Choose a reason for hiding this comment

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

LOG.error("Ledger {} read timeout", ledgerId, ex);

should be replaced with the following:

LOG.error("Ledger {} close failed", ledgerId, ex);

the ex is not the read timeout exception.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

You are right, already fixed

Comment on lines 219 to 218
} else {
openComplete(bk.getReturnRc(BKException.Code.LedgerRecoveryException), null);
closeLedgerHandleAsync().whenComplete((r, ex) -> {
openComplete(bk.getReturnRc(BKException.Code.LedgerRecoveryException), null);
});
}
}
Copy link
Member

Choose a reason for hiding this comment

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

also need log error:

                            if (ex != null) {
                                LOG.error("Ledger {} close failed", ledgerId, ex);
                            }

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Already fixed

@poorbarcode poorbarcode requested review from wenbingshen and removed request for hangc0276 October 21, 2022 03:56
Copy link
Member

@wenbingshen wenbingshen left a comment

Choose a reason for hiding this comment

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

nice

@hangc0276
Copy link
Contributor

@poorbarcode Would you please take a look at the failed CI?
https://github.com/apache/bookkeeper/actions/runs/3294663301/jobs/5433755230

@poorbarcode poorbarcode requested review from eolivelli and removed request for dlg99, merlimat and zymap October 26, 2022 11:24
Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

Lgtm

@hangc0276
Copy link
Contributor

@poorbarcode Would you please rebase the master? thanks.

@@ -331,47 +330,6 @@ public void testOpenLedgerClientClosed() throws Exception {
.execute());
}

@Test
Copy link
Member

Choose a reason for hiding this comment

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

Why remove them?

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

Copy link
Contributor Author

Choose a reason for hiding this comment

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

These two tests has been removed into BookKeeperBuildersOpenLedgerTest.java. see #3562 (comment)

@poorbarcode
Copy link
Contributor Author

rerun failure checks

@hangc0276
Copy link
Contributor

@zymap Would you please help take a look? thanks.

@hangc0276 hangc0276 merged commit ef31c7a into apache:master Oct 27, 2022
@poorbarcode poorbarcode deleted the improve/open_ledger_op_timeout branch October 27, 2022 07:17
hangc0276 pushed a commit to streamnative/bookkeeper-achieved that referenced this pull request Oct 28, 2022
…pache#3562)

Descriptions of the changes in this PR:

### Motivation
When we execute `bkClient.openLedger(ledgerId)`, the execution flow is as follows:

1. start opening the ledger
2. get ledger meta
3. read the last confirmed entry
4. open ledger success

If we get the correct ledgerMeta at step 2, this means that this ledger has not been deleted. If step 3 times out, we should try again to make sure the ledger exists until we get a clear response from the BK server.<strong>(High light)</strong>However, in the current implementation, the timeout exception is rewritten as a `LedgerRecoveryException`, making it impossible to determine whether we should retry.

Log:
```
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] ERROR org.apache.bookkeeper.client.ReadLastConfirmedOp - While readLastConfirmed ledger: 59158316 did not hear success responses from all quorums, QuorumCoverage(e:2,w:2,a:2) = [-23, -23]
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [order/org-217/persistent/p1010-tx] Opened ledger 59158316 for consumer order-service. rc=-10
```

Looking at the ledger metadata:
```
LedgerMetadata{formatVersion=3, ensembleSize=2, writeQuorumSize=2, ackQuorumSize=2, state=IN_RECOVERY, digestType=CRC32C, password=base64:, ensembles={0=[***:3181, ***:3181]}, customMetadata={component=***, pulsar/managed-ledger=***, pulsar/cursor=***, application=***}}
```

see also: apache/pulsar#18123

### Changes

- When calling openLedgerOp, do not rewritten `TiemoutException` as a `LedgerRecoveryException`
- add the dependency: `junit4-dataprovider`
- use `@DataProvider` to simpler the test case "testOpenLedgerRecover" & "testOpenLedgerNoRecover"

(cherry picked from commit ef31c7a)
zymap pushed a commit that referenced this pull request Nov 3, 2022
…3562)

Descriptions of the changes in this PR:

### Motivation
When we execute `bkClient.openLedger(ledgerId)`, the execution flow is as follows:

1. start opening the ledger
2. get ledger meta
3. read the last confirmed entry
4. open ledger success

If we get the correct ledgerMeta at step 2, this means that this ledger has not been deleted. If step 3 times out, we should try again to make sure the ledger exists until we get a clear response from the BK server.<strong>(High light)</strong>However, in the current implementation, the timeout exception is rewritten as a `LedgerRecoveryException`, making it impossible to determine whether we should retry.

Log:
```
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] ERROR org.apache.bookkeeper.client.ReadLastConfirmedOp - While readLastConfirmed ledger: 59158316 did not hear success responses from all quorums, QuorumCoverage(e:2,w:2,a:2) = [-23, -23]
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [order/org-217/persistent/p1010-tx] Opened ledger 59158316 for consumer order-service. rc=-10
```

Looking at the ledger metadata:
```
LedgerMetadata{formatVersion=3, ensembleSize=2, writeQuorumSize=2, ackQuorumSize=2, state=IN_RECOVERY, digestType=CRC32C, password=base64:, ensembles={0=[***:3181, ***:3181]}, customMetadata={component=***, pulsar/managed-ledger=***, pulsar/cursor=***, application=***}}
```

see also: apache/pulsar#18123

### Changes

- When calling openLedgerOp, do not rewritten `TiemoutException` as a `LedgerRecoveryException`
- add the dependency: `junit4-dataprovider`
- use `@DataProvider` to simpler the test case "testOpenLedgerRecover" & "testOpenLedgerNoRecover"

(cherry picked from commit ef31c7a)
hangc0276 pushed a commit to hangc0276/bookkeeper that referenced this pull request Nov 7, 2022
…pache#3562)

Descriptions of the changes in this PR:

### Motivation
When we execute `bkClient.openLedger(ledgerId)`, the execution flow is as follows:

1. start opening the ledger
2. get ledger meta
3. read the last confirmed entry
4. open ledger success

If we get the correct ledgerMeta at step 2, this means that this ledger has not been deleted. If step 3 times out, we should try again to make sure the ledger exists until we get a clear response from the BK server.<strong>(High light)</strong>However, in the current implementation, the timeout exception is rewritten as a `LedgerRecoveryException`, making it impossible to determine whether we should retry.

Log:
```
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] ERROR org.apache.bookkeeper.client.ReadLastConfirmedOp - While readLastConfirmed ledger: 59158316 did not hear success responses from all quorums, QuorumCoverage(e:2,w:2,a:2) = [-23, -23]
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [order/org-217/persistent/p1010-tx] Opened ledger 59158316 for consumer order-service. rc=-10
```

Looking at the ledger metadata:
```
LedgerMetadata{formatVersion=3, ensembleSize=2, writeQuorumSize=2, ackQuorumSize=2, state=IN_RECOVERY, digestType=CRC32C, password=base64:, ensembles={0=[***:3181, ***:3181]}, customMetadata={component=***, pulsar/managed-ledger=***, pulsar/cursor=***, application=***}}
```

see also: apache/pulsar#18123

### Changes

- When calling openLedgerOp, do not rewritten `TiemoutException` as a `LedgerRecoveryException`
- add the dependency: `junit4-dataprovider`
- use `@DataProvider` to simpler the test case "testOpenLedgerRecover" & "testOpenLedgerNoRecover"

(cherry picked from commit ef31c7a)
hangc0276 pushed a commit to hangc0276/bookkeeper that referenced this pull request Nov 7, 2022
…pache#3562)

Descriptions of the changes in this PR:

### Motivation
When we execute `bkClient.openLedger(ledgerId)`, the execution flow is as follows:

1. start opening the ledger
2. get ledger meta
3. read the last confirmed entry
4. open ledger success

If we get the correct ledgerMeta at step 2, this means that this ledger has not been deleted. If step 3 times out, we should try again to make sure the ledger exists until we get a clear response from the BK server.<strong>(High light)</strong>However, in the current implementation, the timeout exception is rewritten as a `LedgerRecoveryException`, making it impossible to determine whether we should retry.

Log:
```
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] ERROR org.apache.bookkeeper.client.ReadLastConfirmedOp - While readLastConfirmed ledger: 59158316 did not hear success responses from all quorums, QuorumCoverage(e:2,w:2,a:2) = [-23, -23]
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [order/org-217/persistent/p1010-tx] Opened ledger 59158316 for consumer order-service. rc=-10
```

Looking at the ledger metadata:
```
LedgerMetadata{formatVersion=3, ensembleSize=2, writeQuorumSize=2, ackQuorumSize=2, state=IN_RECOVERY, digestType=CRC32C, password=base64:, ensembles={0=[***:3181, ***:3181]}, customMetadata={component=***, pulsar/managed-ledger=***, pulsar/cursor=***, application=***}}
```

see also: apache/pulsar#18123

### Changes

- When calling openLedgerOp, do not rewritten `TiemoutException` as a `LedgerRecoveryException`
- add the dependency: `junit4-dataprovider`
- use `@DataProvider` to simpler the test case "testOpenLedgerRecover" & "testOpenLedgerNoRecover"

(cherry picked from commit ef31c7a)
nicoloboschi pushed a commit to datastax/bookkeeper that referenced this pull request Jan 11, 2023
…pache#3562)

Descriptions of the changes in this PR:

### Motivation
When we execute `bkClient.openLedger(ledgerId)`, the execution flow is as follows:

1. start opening the ledger
2. get ledger meta
3. read the last confirmed entry
4. open ledger success

If we get the correct ledgerMeta at step 2, this means that this ledger has not been deleted. If step 3 times out, we should try again to make sure the ledger exists until we get a clear response from the BK server.<strong>(High light)</strong>However, in the current implementation, the timeout exception is rewritten as a `LedgerRecoveryException`, making it impossible to determine whether we should retry.

Log:
```
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] ERROR org.apache.bookkeeper.client.ReadLastConfirmedOp - While readLastConfirmed ledger: 59158316 did not hear success responses from all quorums, QuorumCoverage(e:2,w:2,a:2) = [-23, -23]
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [order/org-217/persistent/p1010-tx] Opened ledger 59158316 for consumer order-service. rc=-10
```

Looking at the ledger metadata:
```
LedgerMetadata{formatVersion=3, ensembleSize=2, writeQuorumSize=2, ackQuorumSize=2, state=IN_RECOVERY, digestType=CRC32C, password=base64:, ensembles={0=[***:3181, ***:3181]}, customMetadata={component=***, pulsar/managed-ledger=***, pulsar/cursor=***, application=***}}
```

see also: apache/pulsar#18123

### Changes

- When calling openLedgerOp, do not rewritten `TiemoutException` as a `LedgerRecoveryException`
- add the dependency: `junit4-dataprovider`
- use `@DataProvider` to simpler the test case "testOpenLedgerRecover" & "testOpenLedgerNoRecover"

(cherry picked from commit ef31c7a)
(cherry picked from commit cee32aa)
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
…pache#3562)

Descriptions of the changes in this PR:

### Motivation
When we execute `bkClient.openLedger(ledgerId)`, the execution flow is as follows:

1. start opening the ledger
2. get ledger meta
3. read the last confirmed entry
4. open ledger success

If we get the correct ledgerMeta at step 2, this means that this ledger has not been deleted. If step 3 times out, we should try again to make sure the ledger exists until we get a clear response from the BK server.<strong>(High light)</strong>However, in the current implementation, the timeout exception is rewritten as a `LedgerRecoveryException`, making it impossible to determine whether we should retry.

Log:
```
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] ERROR org.apache.bookkeeper.client.ReadLastConfirmedOp - While readLastConfirmed ledger: 59158316 did not hear success responses from all quorums, QuorumCoverage(e:2,w:2,a:2) = [-23, -23]
Oct 17, 2022 22:54:05.818 [BookKeeperClientWorker-OrderedExecutor-16-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [order/org-217/persistent/p1010-tx] Opened ledger 59158316 for consumer order-service. rc=-10
```

Looking at the ledger metadata:
```
LedgerMetadata{formatVersion=3, ensembleSize=2, writeQuorumSize=2, ackQuorumSize=2, state=IN_RECOVERY, digestType=CRC32C, password=base64:, ensembles={0=[***:3181, ***:3181]}, customMetadata={component=***, pulsar/managed-ledger=***, pulsar/cursor=***, application=***}}
```

see also: apache/pulsar#18123

### Changes

- When calling openLedgerOp, do not rewritten `TiemoutException` as a `LedgerRecoveryException`
- add the dependency: `junit4-dataprovider`
- use `@DataProvider` to simpler the test case "testOpenLedgerRecover" & "testOpenLedgerNoRecover"
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants