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

Add ensemble check to over-replicated ledger GC #2813

Merged

Conversation

shustsud
Copy link
Contributor

@shustsud shustsud commented Oct 5, 2021

Motivation

  • Issue is as described in PR#2797.

In one day, zookeepers became high cpu usage and disk full.
The cause of this is bookie's gc of overreplicated ledgers.
Gc created/deleted zk nodes under /ledgers/underreplication/locks very frequently and some bookies ran gc at same time.
As a result, zookeepers created a lot of snapshots and became disk full.

  • I want to reduce the number of lock node creations and deletions in ZK.

Changes

  • Add an ensemble check before creating the lock node.
    This is to reduce the number of lock node creations and deletions in ZK.

  • If PR#2797 was merged, this PR needs to be fixed.

Copy link
Contributor

@nicoloboschi nicoloboschi left a comment

Choose a reason for hiding this comment

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

LGTM, I left a non blocker comment

}
}
latch.await();
LOG.info("Finished pre-check over-replicated ledgers. candidateOverReplicatedLedgersSize={}",
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe it can be interesting to log how many ledgers are over replicated over the total number of active ledgers (bkActiveLedgers)
something like

Over-replicated ledgers pre-check count: {}/{}, candidateOverReplicatedLedgers.size(), bkActiveLedgers.size()

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed.

Copy link
Contributor

@nicoloboschi nicoloboschi left a comment

Choose a reason for hiding this comment

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

LGTM

@dlg99
Copy link
Contributor

dlg99 commented Oct 8, 2021

zookeepers created a lot of snapshots and became disk full

take a look at autopurge.snapRetainCount and autopurge.purgeInterval at http://zookeeper.apache.org/doc/r3.6.3/zookeeperAdmin.html#sc_advancedConfiguration

@dlg99 dlg99 requested a review from eolivelli October 8, 2021 15:52
Copy link
Contributor

@dlg99 dlg99 left a comment

Choose a reason for hiding this comment

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

Left couple of comments.

@shustsud
Copy link
Contributor Author

take a look at autopurge.snapRetainCount and autopurge.purgeInterval at http://zookeeper.apache.org/doc/r3.6.3/zookeeperAdmin.html#sc_advancedConfiguration

Thank you for your comment.
I take a look at the URL above.

Copy link
Contributor

@dlg99 dlg99 left a comment

Choose a reason for hiding this comment

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

LGTM. Thank you.
@eolivelli please take a look

@shustsud shustsud changed the title Add pre-check to over-replicated ledger GC Add ensemble check to over-replicated ledger GC Oct 12, 2021
@shustsud
Copy link
Contributor Author

rerun failure checks

1 similar comment
@shustsud
Copy link
Contributor Author

rerun failure checks

@zymap
Copy link
Member

zymap commented Oct 15, 2021

@eolivelli Would you like to review this PR? @dlg99 want to include this PR into the release/4.14.3 so I want to merge this PR

@zymap
Copy link
Member

zymap commented Oct 18, 2021

ping @eolivelli

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
I left one request, PTAL

Happy to merge

// this is to reduce the number of lock node creations and deletions in ZK.
// the ensemble check is done again after the lock node is created.
// also, check if the ledger is being replicated already by the replication worker
if (!isNotBookieIncludedInLedgerEnsembles(ledgerManager.readLedgerMetadata(ledgerId).get())
Copy link
Contributor

Choose a reason for hiding this comment

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

readLedgerMetadata is a costly operation,
Can you please move it in a dedicated line, out of the 'if' statement?
This way is it more evident while reading the 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.

@eolivelli
Thank you for your comment.
I fixed it as above.

@zymap
Copy link
Member

zymap commented Oct 20, 2021

@shustsud Could you please resolve the conflicts?

@shustsud shustsud force-pushed the add_pre-check_to_over-replicated_ledger_gc branch from 766eb36 to 728fbba Compare October 21, 2021 02:34
@shustsud
Copy link
Contributor Author

@zymap
I resolved this PR conflict.

@shustsud
Copy link
Contributor Author

rerun failure checks

@shustsud
Copy link
Contributor Author

@zymap
I'm sorry. I have created the following PR.
 #2844

I think this PR should be included in the 4.14.3 release.

zymap pushed a commit that referenced this pull request Oct 25, 2021
…#2844)

### Motivation
For each ledger whose metadata is not in ZK, following stack trace will be output:

```
15:30:17.925 [GarbageCollectorThread-11-1] ERROR o.a.b.b.ScanAndCompareGarbageCollector - Exception when iterating through the ledgers to check for over-replication
java.util.concurrent.ExecutionException: org.apache.bookkeeper.client.BKException$BKNoSuchLedgerExistsException: No such ledger exists
        at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
        at org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector.removeOverReplicatedledgers(ScanAndCompareGarbageCollector.java:199)
        at org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector.gc(ScanAndCompareGarbageCollector.java:120)
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.doGcLedgers(GarbageCollectorThread.java:372)
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.runWithFlags(GarbageCollectorThread.java:323)
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(GarbageCollectorThread.java:301)
        at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.bookkeeper.client.BKException$BKNoSuchLedgerExistsException: No such ledger exists
        at org.apache.bookkeeper.meta.AbstractZkLedgerManager$3.processResult(AbstractZkLedgerManager.java:397)
        at org.apache.bookkeeper.zookeeper.ZooKeeperClient$19$1.processResult(ZooKeeperClient.java:994)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:575)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

It is noisy, makes the size of log files large and finally causes OOM during log rotation.
So we should suppress the stacktrace.

(This problem is due to [#2813](#2813).)

### Changes
Add error handling to readLedgerMetadata in over-replicated ledger GC in order to suppress the stacktrace.
zymap pushed a commit that referenced this pull request Oct 26, 2021
### Motivation
- Issue is as described in [PR#2797](#2797).
> In one day, zookeepers became high cpu usage and disk full.
> The cause of this is bookie's gc of overreplicated ledgers.
> Gc created/deleted zk nodes under /ledgers/underreplication/locks very frequently and some bookies ran gc at same time.
> As a result, zookeepers created a lot of snapshots and became disk full.

- I want to reduce the number of lock node creations and deletions in ZK.

### Changes
- Add an ensemble check before creating the lock node.
This is to reduce the number of lock node creations and deletions in ZK.

- ~~If [PR#2797](#2797) was merged, this PR needs to be fixed.~~

(cherry picked from commit 53954ca)
zymap pushed a commit that referenced this pull request Oct 26, 2021
…#2844)

### Motivation
For each ledger whose metadata is not in ZK, following stack trace will be output:

```
15:30:17.925 [GarbageCollectorThread-11-1] ERROR o.a.b.b.ScanAndCompareGarbageCollector - Exception when iterating through the ledgers to check for over-replication
java.util.concurrent.ExecutionException: org.apache.bookkeeper.client.BKException$BKNoSuchLedgerExistsException: No such ledger exists
        at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
        at org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector.removeOverReplicatedledgers(ScanAndCompareGarbageCollector.java:199)
        at org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector.gc(ScanAndCompareGarbageCollector.java:120)
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.doGcLedgers(GarbageCollectorThread.java:372)
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.runWithFlags(GarbageCollectorThread.java:323)
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(GarbageCollectorThread.java:301)
        at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.bookkeeper.client.BKException$BKNoSuchLedgerExistsException: No such ledger exists
        at org.apache.bookkeeper.meta.AbstractZkLedgerManager$3.processResult(AbstractZkLedgerManager.java:397)
        at org.apache.bookkeeper.zookeeper.ZooKeeperClient$19$1.processResult(ZooKeeperClient.java:994)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:575)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

It is noisy, makes the size of log files large and finally causes OOM during log rotation.
So we should suppress the stacktrace.

(This problem is due to [#2813](#2813).)

### Changes
Add error handling to readLedgerMetadata in over-replicated ledger GC in order to suppress the stacktrace.

(cherry picked from commit bd5c50b)
hsaputra pushed a commit that referenced this pull request Oct 29, 2021
### Motivation

In order to complete migration to Gradle we must build all the subprojects.

### Changes

- Enabled `sh` integration tests with gradle, located in `tests/scripts/src/test/bash/gradle`
- Added these modules to the build
    - `bookkeeper-http:servlet-http-server` 
    - `metadata-drivers:etcd`
    - `tests:backward-compat:*`
    - `tests:shaded:*`
    - `stream:bk-grpc-name-resolver`
- DL shading process is now performed (before it didn't build any jar)
- Groovy tests (`tests:backward-compat:*`) now are triggered by the build/tests itself; with Maven, there is a "runner" project (`tests/integration-tests-base-groovy`); in Gradle is useless so it is skipped


### Test

- Both `bin/bookkeper standalone` and `bin/bookkeper_gradle standalone` work locally
- Tests are passing locally  

Master Issue: #2849



Reviewers: Henry Saputra <[email protected]>, Prashant Kumar <None>

This closes #2850 from nicoloboschi/fix/2849/gradle and squashes the following commits:

00b49f4 [Nicolò Boschi] Fix common_gradle.sh regex
bd739fd [Nicolò Boschi] fix sh tests
43230ba [Nicolò Boschi] revert sh files. Avoid to modify maven files, create gradle versions to faciltate migration
d1f95e4 [Nicolò Boschi] fix shaded deps
bcab40d [Nicolò Boschi] fix build
5fd0341 [Nicolò Boschi] fix build
0082e0e [Nicolò Boschi] fix build
2c32ac1 [Nicolò Boschi] fixes
3bc0b26 [Nicolò Boschi] bookkeeper-server-shaded-tests
ba89132 [Nicolò Boschi] shaded tests
6d39e33 [Nicolò Boschi] sh tests
e0032bc [Nicolò Boschi] actually run arquillian groovy tests
08dcc39 [Nicolò Boschi] backwards
2361f79 [Nicolò Boschi] hierarchical-ledger-manager
8388e11 [Nicolò Boschi] current-server-old-clients
6a24344 [Nicolò Boschi] bc-non-fips
2faca01 [Nicolò Boschi] bk-grpc-name-resolver
991bc11 [Nicolò Boschi] servlet-http-server
675ef7b [Nicolò Boschi] etcd
b1d5e14 [ZhangJian He] A empty implement in EtcdLedgerManagerFactory to let the project can compile (#2845)
bd5c50b [shustsud] Add error handling to readLedgerMetadata in over-replicated ledger GC (#2844)
746f9f6 [Matteo Merli] Remove direct ZK access for Auditor (#2842)
4117200 [ZhangJian He] the compare should be >= instead of > (#2782)
14ef56f [Prashant Kumar] BookieId can not be cast to BookieSocketAddress (#2843)
e10f3fe [ZhangJian He] Forget to close preAllocator log on shutdown (#2819)
53954ca [shustsud] Add ensemble check to over-replicated ledger GC (#2813)
919fdd3 [Prashant Kumar] Issue:2840 Create bookie shellscript for gradle (#2841)
031d168 [gaozhangmin] fix-npe-when-pulsar-ZkBookieRackAffinityMapping-getBookieAddressResolver (#2788)
3dd671c [Prashant Kumar] Migrate bookkeepr-server:test to gradle run unit tests excepts org.apache.bookkeeper.bookie. org.apache.bookkeeper.client org.apache.bookkeeper.replication org.apache.bookkeeper.tls. (#2812)
f6903b8 [Jack Vanlightly] BP-44 USE metrics
a4afaa4 [Matteo Merli] Eliminate direct ZK access in ScanAndCompareGarbageCollector (#2833)
a9b576d [Yunze Xu] Release semaphore when addEntry accepts the same entries (#2832)
148bf22 [Yun Tang] Ensure to release cache during KeyValueStorageRocksDB#closec (#2821)
4dc4260 [gaozhangmin] Heap memory leak problem when ledger replication failed (#2794)
a522fa3 [Raúl Gracia] Issue 2815: Upgrade to log4j2 to get rid of CVE-2019-17571 (#2816)
0465052 [Nicolò Boschi] Upgrade httpclient from 4.5.5 to 4.5.13 (#2793)
594a056 [Raúl Gracia] Issue 2795: Bookkeeper upgrade using Bookie ID may fail due to cookie mismatch (#2796)
354cf37 [Raúl Gracia] Upgraded dependencies with CVEs (#2792)
e413c70 [Raúl Gracia] Issue 2728: Entry Log GC may get blocked when using entryLogPerLedgerEnabled option (#2779)
883231e [pradeepbn] Building bookkeeper with gradle on java11
@rdhabalia rdhabalia assigned shustsud and unassigned shustsud Feb 9, 2023
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
### Motivation
- Issue is as described in [PR#2797](apache#2797).
> In one day, zookeepers became high cpu usage and disk full.
> The cause of this is bookie's gc of overreplicated ledgers.
> Gc created/deleted zk nodes under /ledgers/underreplication/locks very frequently and some bookies ran gc at same time.
> As a result, zookeepers created a lot of snapshots and became disk full.

- I want to reduce the number of lock node creations and deletions in ZK.

### Changes
- Add an ensemble check before creating the lock node.
This is to reduce the number of lock node creations and deletions in ZK.

- ~~If [PR#2797](apache#2797) was merged, this PR needs to be fixed.~~
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
…apache#2844)

### Motivation
For each ledger whose metadata is not in ZK, following stack trace will be output:

```
15:30:17.925 [GarbageCollectorThread-11-1] ERROR o.a.b.b.ScanAndCompareGarbageCollector - Exception when iterating through the ledgers to check for over-replication
java.util.concurrent.ExecutionException: org.apache.bookkeeper.client.BKException$BKNoSuchLedgerExistsException: No such ledger exists
        at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
        at org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector.removeOverReplicatedledgers(ScanAndCompareGarbageCollector.java:199)
        at org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector.gc(ScanAndCompareGarbageCollector.java:120)
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.doGcLedgers(GarbageCollectorThread.java:372)
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.runWithFlags(GarbageCollectorThread.java:323)
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(GarbageCollectorThread.java:301)
        at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.bookkeeper.client.BKException$BKNoSuchLedgerExistsException: No such ledger exists
        at org.apache.bookkeeper.meta.AbstractZkLedgerManager$3.processResult(AbstractZkLedgerManager.java:397)
        at org.apache.bookkeeper.zookeeper.ZooKeeperClient$19$1.processResult(ZooKeeperClient.java:994)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:575)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

It is noisy, makes the size of log files large and finally causes OOM during log rotation.
So we should suppress the stacktrace.

(This problem is due to [apache#2813](apache#2813).)

### Changes
Add error handling to readLedgerMetadata in over-replicated ledger GC in order to suppress the stacktrace.
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
### Motivation

In order to complete migration to Gradle we must build all the subprojects.

### Changes

- Enabled `sh` integration tests with gradle, located in `tests/scripts/src/test/bash/gradle`
- Added these modules to the build
    - `bookkeeper-http:servlet-http-server` 
    - `metadata-drivers:etcd`
    - `tests:backward-compat:*`
    - `tests:shaded:*`
    - `stream:bk-grpc-name-resolver`
- DL shading process is now performed (before it didn't build any jar)
- Groovy tests (`tests:backward-compat:*`) now are triggered by the build/tests itself; with Maven, there is a "runner" project (`tests/integration-tests-base-groovy`); in Gradle is useless so it is skipped


### Test

- Both `bin/bookkeper standalone` and `bin/bookkeper_gradle standalone` work locally
- Tests are passing locally  

Master Issue: apache#2849



Reviewers: Henry Saputra <[email protected]>, Prashant Kumar <None>

This closes apache#2850 from nicoloboschi/fix/2849/gradle and squashes the following commits:

00b49f4 [Nicolò Boschi] Fix common_gradle.sh regex
bd739fd [Nicolò Boschi] fix sh tests
43230ba [Nicolò Boschi] revert sh files. Avoid to modify maven files, create gradle versions to faciltate migration
d1f95e4 [Nicolò Boschi] fix shaded deps
bcab40d [Nicolò Boschi] fix build
5fd0341 [Nicolò Boschi] fix build
0082e0e [Nicolò Boschi] fix build
2c32ac1 [Nicolò Boschi] fixes
3bc0b26 [Nicolò Boschi] bookkeeper-server-shaded-tests
ba89132 [Nicolò Boschi] shaded tests
6d39e33 [Nicolò Boschi] sh tests
e0032bc [Nicolò Boschi] actually run arquillian groovy tests
08dcc39 [Nicolò Boschi] backwards
2361f79 [Nicolò Boschi] hierarchical-ledger-manager
8388e11 [Nicolò Boschi] current-server-old-clients
6a24344 [Nicolò Boschi] bc-non-fips
2faca01 [Nicolò Boschi] bk-grpc-name-resolver
991bc11 [Nicolò Boschi] servlet-http-server
675ef7b [Nicolò Boschi] etcd
b1d5e14 [ZhangJian He] A empty implement in EtcdLedgerManagerFactory to let the project can compile (apache#2845)
bd5c50b [shustsud] Add error handling to readLedgerMetadata in over-replicated ledger GC (apache#2844)
746f9f6 [Matteo Merli] Remove direct ZK access for Auditor (apache#2842)
4117200 [ZhangJian He] the compare should be >= instead of > (apache#2782)
14ef56f [Prashant Kumar] BookieId can not be cast to BookieSocketAddress (apache#2843)
e10f3fe [ZhangJian He] Forget to close preAllocator log on shutdown (apache#2819)
53954ca [shustsud] Add ensemble check to over-replicated ledger GC (apache#2813)
919fdd3 [Prashant Kumar] Issue:2840 Create bookie shellscript for gradle (apache#2841)
031d168 [gaozhangmin] fix-npe-when-pulsar-ZkBookieRackAffinityMapping-getBookieAddressResolver (apache#2788)
3dd671c [Prashant Kumar] Migrate bookkeepr-server:test to gradle run unit tests excepts org.apache.bookkeeper.bookie. org.apache.bookkeeper.client org.apache.bookkeeper.replication org.apache.bookkeeper.tls. (apache#2812)
f6903b8 [Jack Vanlightly] BP-44 USE metrics
a4afaa4 [Matteo Merli] Eliminate direct ZK access in ScanAndCompareGarbageCollector (apache#2833)
a9b576d [Yunze Xu] Release semaphore when addEntry accepts the same entries (apache#2832)
148bf22 [Yun Tang] Ensure to release cache during KeyValueStorageRocksDB#closec (apache#2821)
4dc4260 [gaozhangmin] Heap memory leak problem when ledger replication failed (apache#2794)
a522fa3 [Raúl Gracia] Issue 2815: Upgrade to log4j2 to get rid of CVE-2019-17571 (apache#2816)
0465052 [Nicolò Boschi] Upgrade httpclient from 4.5.5 to 4.5.13 (apache#2793)
594a056 [Raúl Gracia] Issue 2795: Bookkeeper upgrade using Bookie ID may fail due to cookie mismatch (apache#2796)
354cf37 [Raúl Gracia] Upgraded dependencies with CVEs (apache#2792)
e413c70 [Raúl Gracia] Issue 2728: Entry Log GC may get blocked when using entryLogPerLedgerEnabled option (apache#2779)
883231e [pradeepbn] Building bookkeeper with gradle on java11
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.

5 participants