-
Notifications
You must be signed in to change notification settings - Fork 907
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
Issue 2728: Entry Log GC may get blocked when using entryLogPerLedgerEnabled option #2779
Issue 2728: Entry Log GC may get blocked when using entryLogPerLedgerEnabled option #2779
Conversation
Signed-off-by: Raúl Gracia <[email protected]>
Signed-off-by: Raúl <[email protected]>
Signed-off-by: Raúl <[email protected]>
Signed-off-by: Raúl <[email protected]>
Signed-off-by: Raúl <[email protected]>
// to a new one. We scan entry logs as follows: | ||
// - entryLogPerLedgerEnabled is false: Extract it for every entry log except for the current one (un-flushed). | ||
// - entryLogPerLedgerEnabled is true: Scan all flushed entry logs up to the highest known id. | ||
Supplier<Long> finalEntryLog = () -> conf.isEntryLogPerLedgerEnabled() ? entryLogger.getLastLogId() : |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thinking out loud:
is it possible to add some method in entryLogger
(here and below) instead of referring to the BookieConfiguration ?
I find adding these conf.isEntryLogPerLedgerEnabled()
quite difficult to maintain in the future.
Probably we can keep inside EntryLogger the fact that we have one EntryLogger per ledger.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, I can do that, no worries.
But I have a more fundamental question about this change that I would like to hear from (all of) you. I will try to summarize it:
- By default, entry logs are all flushed with the exception of the current one.
- With
entryLogPerLedgerEnabled
, there could be multiple unflushed entry logs across the id space.
Now, let's imagine that the Bookie restarts and there is unflushed data/metadata of entry logs. In the default case, this situation would impact only on the last entry log. In the case of entryLogPerLedgerEnabled
, there could be multiple entry logs impacted (i.e., data was in memory but not synced to disk).
The question is: when a Bookie starts and replays the journal, would it take care of all the unflushed entry log data irrespective of whether it belongs to the last entry log or multiple entry logs?
My understanding is that the replay of the journal takes care of all the unflushed data of all entry logs (but I would like to confirm this with you). If this is true, then I think that this PR is safe. In addition, our experiments inducing multiple Bookie restarts in purpose show that no unflushed entry log was wrongly garbage collected after the Bookie was up again.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
When the journal is replayed it writes the entries in the same way as an add entry request received from a client.
I think at some point it might be a good idea to hide the GC and compaction code entirely inside the ledger storage implementations. Different storage engines work in different ways and we're trying to make each conform to a standard set of interfaces/behaviours. ELPL would then be one more ledger storage engine that is free to do GC/compaction in any way it likes without worrying about impacting other storage engines or conforming to a common interface.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think at some point it might be a good idea to hide the GC and compaction code entirely inside the ledger storage implementations
this is a good idea (for the future, not for this patch I mean)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @Vanlightly, then I understand that this change is safe.
I also like the considerations regarding the refactoring, we have been also discussing this with @fpj. But I wonder if it would be better to first have entryLogPerLedgerEnabled
option working correctly (goal of this PR), and think in parallel about the refactoring of GC logic and ledger storage separately, as it appears to be a larger change. Does it sound reasonable?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1
please follow up with the refactoring work
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good.
BP-44: USE metrics. A proposal for improving BookKeeper metrics so that operators can employ the USE method for diagnosing performance issues. Reviewers: Henry Saputra <[email protected]>, Andrey Yegorov <None>, Enrico Olivelli <[email protected]> This closes #2835 from Vanlightly/BP-44-use-metrics and squashes the following commits: 8d9baab [Jack Vanlightly] Added link to USE method and listed each term of USE 5a0f67d [Jack Vanlightly] BP-44 USE metrics 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
### 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
BP-44: USE metrics. A proposal for improving BookKeeper metrics so that operators can employ the USE method for diagnosing performance issues. Reviewers: Henry Saputra <[email protected]>, Andrey Yegorov <None>, Enrico Olivelli <[email protected]> This closes apache#2835 from Vanlightly/BP-44-use-metrics and squashes the following commits: 8d9baab [Jack Vanlightly] Added link to USE method and listed each term of USE 5a0f67d [Jack Vanlightly] BP-44 USE metrics 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
### 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
Motivation
Fix Bookkeeper entry log GC when entryLogPerLedgerEnabled is activated, so we prevent Bookies to run out of ledger storage capacity.
Changes
The objective of this PR is lo leave the GC algorithm in the default case untouched (
entryLogPerLedgerEnabled=false
), and fix the GC whenentryLogPerLedgerEnabled=true
. By "fix", we mean that the GC process can garbage collect all the available rotated entry log files that belong to ledgers that have been deleted already.Context: By default (
entryLogPerLedgerEnabled=false
), meaning that entry log files may contain entries from multiple ledgers. In the normal case, entries are being added to the current entry log file and when it reaches a size limit, it is flushed and rotated so a new entry log file is created to store the next entries. In this sense, the logic for creating entry logs assumes that the initial id is 0 and each new entry log file will have an id which is +1 compared to the previous entry log id. This logic entails that all entry log files with an id lower than the current one have been already flushed and rotated, so they can be safely considered as candidates for garbage collection in case that they are not related to any active ledger.Problem: The problem appears when
entryLogPerLedgerEnabled=true
, as it breaks some of the assumptions of the whole approach to create entry logs and garbage collect them. With this option, each entry log file will be related exclusively to one ledger id. While the entry log id scheme remains intact (i.e., ids of entry log files are produced sequentially starting from 0), now the assumption of entry logs being rotated linearly does not hold anymore. We can have a series of flushed and unflushed entry logs due to the differences in write throughput of the ledgers. However, the GC algorithm will only scan the metadata of entry logs that are below thegetLeastUnflushedEntryLog()
:bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
Line 589 in c8d4086
Therefore, if one entry log file takes a long time to be flushed and rotated, it will be prevent the GC algorithm to garbage collect entry log ids beyond that id. If entry logs are getting produced at high rate beyond the
getLeastUnflushedEntryLog()
the Bookie may run out of ledger storage. This happens even though most of the entry logs beyondgetLeastUnflushedEntryLog()
can be candidates for garbage collection.Solution: The current proposal tries to deal with the
entryLogPerLedgerEnabled=true
case, while leaving as is the default case (entryLogPerLedgerEnabled=false
). The changes in this sense, are the following:entryLogPerLedgerEnabled=true
we will attempt to scan entry log files up to the highest id. That is, the algorithm will go beyond thegetLeastUnflushedEntryLog()
.RecentEntryLogsStatus
that meet the following condition:entryLogsStatusMap.containsKey(entryLogId) && entryLogsStatusMap.get(entryLogId) || entryLogId < leastUnflushedLogId
. Note thatentryLogsStatusMap
is an in-memory map that may be lost after a Bookie restart. The assumption is that after a Bookie restart, the ledger metadata of all entry logs is restored from the journal, so it contains the last state they had in memory before the crash.How to verify it
At the moment, we have done a 2 day workload with
entryLogPerLedgerEnabled=true
writing 1GBps. The original issue (ledger storage getting full sporadically in some Bookies) has not been reproduced.We are in the process of analyzing logs from these experiments and create more unit tests to validate corner cases of this change.
Master Issue: #2728