[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13260691#comment-13260691 ] Matteo Merli commented on BOOKKEEPER-220: - Thanks Flavio for reviewing the code. # I'm sure there are similarities since we're trying to achieve the same goal :) I already took a look at the BookkeeperPersistentManager in Hedwig, but I need some more time to fully understand its design. # Yes, this point is not clear yet. I think there are a few options : ** Make the ManagedLedger acquire a lock on zookeeper when opening ** Enforce the single writer within the same VM and _trust_ that no other machine is using it. The reason I'm considering this is because we already need to have a separate lock to ensure that a single topic/queue is only served by one broker at any time. # You're right, for now I've implemented all the async call using an executor in a background thread pool. I know this is far from ideal from an async point of view, but please note that the implementation is still very incomplete, I just put it there so that Chiaming can start working in parallel on the broker, using the ManagedLedger async api with something that for a client perspective "somewhat works". > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13262177#comment-13262177 ] Matteo Merli commented on BOOKKEEPER-220: - # How should we proceed on this? Who is going to list the Hedwig differences from ML? I am perfectly fine in extending the ML subset to something that can be convenient to both us and Hedwig. So far I just included the minimal set of requirements that we identified. Also, I am perfectly ok in changing any terminology used for the ML. # I don't know how the passord/master-key work in BK, need to dig more into it. So far, I just made that you can choose the password for the entire ML. The same password will be used when opening ledgers for reading/writing. Right now, if 2 different clients try to open the same ML and start writing, they will write into different BK ledgers, and possibly corrupt the ML state. Same thing for reading, if ML is running on 2 boxes, the cursor list will not be accurate leading to wrong ledgers deletions. I think the password have to be the same (within the same ML), because ledgers created on one box will need to be read from others in case of a failover. # Ok # Queue semantics: the original idea was to leave the topic/queue implementation in the broker/hub layer. In this case, when the broker wants to use the ML for a queue, it will simply use one single cursor. Again, if it makes sense we could possibly move that code into the ML. > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-181) Scale hedwig
[ https://issues.apache.org/jira/browse/BOOKKEEPER-181?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13262223#comment-13262223 ] Matteo Merli commented on BOOKKEEPER-181: - We've been going through the delete ledger operation h4. Deletion sequence # Client calls {{bookkeeperClient.deleteLedger(id)}} ## We know the {{N}} bookies that have this ledger {{(bk1, ..., bkn)}} ## Add {{id}} to the list of {{to_delete}} ledgers for {{bk1}} ## ... ## Add {{id}} to the list of {{to_delete}} ledgers for {{bkn}} ## Remove ledger {{id}} from main list (eg. '/ledgers/Lxx') ## Return success to the client # Each bookie, in the GC thread will scan its own list of ledgers from the {{to_delete}} queue and remove them from the list when the deletion has been finalized. h4. Handling failures When the client deletes a ledger, it'd have to do {{N+1}} separated steps. Each of them can potentially fail, and the client will receive an exception in this case. If the client receives an exception, it has to assume that the ledger has not been deleted properly and retry the {{deleteLedger()}} call, although the operation may have actually succeeded (or partially succeeded). In the subsequent {{deleteLedger()}} call, the same steps will re-executed. The worst-case scenario is that a bookie could see the same ledger more than once in its own {{to_delete}} queue, and hence try to delete it twice, but this should be easy to handle. > Scale hedwig > > > Key: BOOKKEEPER-181 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-181 > Project: Bookkeeper > Issue Type: Improvement > Components: bookkeeper-server, hedwig-server >Reporter: Sijie Guo >Assignee: Sijie Guo > Fix For: 4.2.0 > > Attachments: hedwigscale.pdf, hedwigscale.pdf > > > Current implementation of Hedwig and BookKeeper is designed to scale to > hundreds of thousands of topics, but now we are looking at scaling them to > tens to hundreds of millions of topics, using a scalable key/value store such > as HBase. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-181) Scale hedwig
[ https://issues.apache.org/jira/browse/BOOKKEEPER-181?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13262716#comment-13262716 ] Matteo Merli commented on BOOKKEEPER-181: - Yes, you're right, with the above scheme you can end with half-deleted ledgers. I think it could be fixed by adding another step to the sequence. Although I would not call this a _data loss_, since the client explicitly asked for deletion, meaning the data is no longer relevant. Anyway, the client must keep track of its own ledger somewhere, and it's its responsibility to make sure the ledgers are deleted (and to retry when the deleteLedger() fails). Otherwise, even with the current implemented solution, there is a resource leak in the system, with unused dangling ledgers that stick around forever. > Scale hedwig > > > Key: BOOKKEEPER-181 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-181 > Project: Bookkeeper > Issue Type: Improvement > Components: bookkeeper-server, hedwig-server >Reporter: Sijie Guo >Assignee: Sijie Guo > Fix For: 4.2.0 > > Attachments: hedwigscale.pdf, hedwigscale.pdf > > > Current implementation of Hedwig and BookKeeper is designed to scale to > hundreds of thousands of topics, but now we are looking at scaling them to > tens to hundreds of millions of topics, using a scalable key/value store such > as HBase. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13266354#comment-13266354 ] Matteo Merli commented on BOOKKEEPER-220: - In our design, hubs (brokers) manage groups of topics/queues. Each group is owned by one broker at once and it's reassigned when doing failover. A single topic/queue maps (1:1) with a ML, so there is only one broker that works with a ML at any time, both for reading/writing. This is exactly the same as in Hedwig case. Anyway, it would be interesting to have the ability of having many readers running on different brokers that consumes from the same ML, although this complicates the design. The major problem that I see is synchronizing all the instances of the ML running on different boxes to decide which ledgers can be deleted. For the ML, the goal we have is to build something like a persistent log appender, with multiple persistent reader marks that can be failed over to other servers when crashing/rebalancing. This is the minimal set of features that we were thinking of. I believe this is very similar to what is currently used for Hedwig. > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-273) LedgerHandle.deleteLedger() should be idempotent
Matteo Merli created BOOKKEEPER-273: --- Summary: LedgerHandle.deleteLedger() should be idempotent Key: BOOKKEEPER-273 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-273 Project: Bookkeeper Issue Type: Bug Components: bookkeeper-client Affects Versions: 4.1.0 Reporter: Matteo Merli Priority: Minor Deleting a non-existing ledger should silently succeed. Current behavior is to raise a ZKException, but it's not possible to know whether there was some error or the ledger does not exists anymore. This scenario will happen when a previous deleteLedger() call succeeded but the client crashed before updating its own ledger list. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-273) LedgerHandle.deleteLedger() should be idempotent
[ https://issues.apache.org/jira/browse/BOOKKEEPER-273?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-273: Attachment: 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch > LedgerHandle.deleteLedger() should be idempotent > > > Key: BOOKKEEPER-273 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-273 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-client >Affects Versions: 4.1.0 >Reporter: Matteo Merli >Priority: Minor > Fix For: 4.1.0 > > Attachments: > 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch > > > Deleting a non-existing ledger should silently succeed. > Current behavior is to raise a ZKException, but it's not possible to know > whether there was some error or the ledger does not exists anymore. > This scenario will happen when a previous deleteLedger() call succeeded but > the client crashed before updating its own ledger list. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-273) LedgerHandle.deleteLedger() should be idempotent
[ https://issues.apache.org/jira/browse/BOOKKEEPER-273?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-273: Attachment: 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch > LedgerHandle.deleteLedger() should be idempotent > > > Key: BOOKKEEPER-273 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-273 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-client >Affects Versions: 4.1.0 >Reporter: Matteo Merli >Priority: Minor > Fix For: 4.1.0 > > Attachments: > 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch, > 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch > > > Deleting a non-existing ledger should silently succeed. > Current behavior is to raise a ZKException, but it's not possible to know > whether there was some error or the ledger does not exists anymore. > This scenario will happen when a previous deleteLedger() call succeeded but > the client crashed before updating its own ledger list. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-273) LedgerHandle.deleteLedger() should be idempotent
[ https://issues.apache.org/jira/browse/BOOKKEEPER-273?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13285701#comment-13285701 ] Matteo Merli commented on BOOKKEEPER-273: - Updated the patch to throw BKNoSuchLedgerExistsException when the ledger doesn't exist. > LedgerHandle.deleteLedger() should be idempotent > > > Key: BOOKKEEPER-273 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-273 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-client >Affects Versions: 4.1.0 >Reporter: Matteo Merli >Priority: Minor > Fix For: 4.1.0 > > Attachments: > 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch, > 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch > > > Deleting a non-existing ledger should silently succeed. > Current behavior is to raise a ZKException, but it's not possible to know > whether there was some error or the ledger does not exists anymore. > This scenario will happen when a previous deleteLedger() call succeeded but > the client crashed before updating its own ledger list. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-273) LedgerHandle.deleteLedger() should be idempotent
[ https://issues.apache.org/jira/browse/BOOKKEEPER-273?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13285711#comment-13285711 ] Matteo Merli commented on BOOKKEEPER-273: - Updated the patch with formatting and exception full name. > LedgerHandle.deleteLedger() should be idempotent > > > Key: BOOKKEEPER-273 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-273 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-client >Affects Versions: 4.1.0 >Reporter: Matteo Merli >Priority: Minor > Fix For: 4.1.0 > > Attachments: > 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch, > 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch, > 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch > > > Deleting a non-existing ledger should silently succeed. > Current behavior is to raise a ZKException, but it's not possible to know > whether there was some error or the ledger does not exists anymore. > This scenario will happen when a previous deleteLedger() call succeeded but > the client crashed before updating its own ledger list. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-273) LedgerHandle.deleteLedger() should be idempotent
[ https://issues.apache.org/jira/browse/BOOKKEEPER-273?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-273: Attachment: 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch > LedgerHandle.deleteLedger() should be idempotent > > > Key: BOOKKEEPER-273 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-273 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-client >Affects Versions: 4.1.0 >Reporter: Matteo Merli >Priority: Minor > Fix For: 4.1.0 > > Attachments: > 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch, > 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch, > 0001-BOOKKEEPER-273-LedgerHandle.deleteLedger-should-be-i.patch > > > Deleting a non-existing ledger should silently succeed. > Current behavior is to raise a ZKException, but it's not possible to know > whether there was some error or the ledger does not exists anymore. > This scenario will happen when a previous deleteLedger() call succeeded but > the client crashed before updating its own ledger list. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-220: Fix Version/s: 4.2.0 > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-220: Attachment: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13286241#comment-13286241 ] Matteo Merli commented on BOOKKEEPER-220: - I have updated the github repository with the latest working code. Also a patch against bookkeeper trunk has been attached. > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13286623#comment-13286623 ] Matteo Merli commented on BOOKKEEPER-220: - @Sijie: thanks for the thorough review bq. 1. ManagedLedgerFactoryImpl with zookeeper quorum string created its owned zookeeper handle and bookkeeper handle but doesn't close them. Good point. Will fix that in factory.shutdown() bq. 2. You did a lot of work in the constructor of ManagedLedgerImpl, which is not a good practice. I think the race condition could be fixed by "syncronyzing" the ManagedLedgerFactoryImpl.open(), that way at most one instance will be created for the same ManagedLedger bq. 3. async & sync operations. seems that you using executor to change sync operation to async. why not leverage the async interface provided by bookkeeper directly? And implement the sync operation based async operation. As of now, on ManagedLedger.asyncAddEntry uses the bookkeeper asyncAddEntry directly and only when adding the entry doesn't involve closing current ledger and opening a new one. The reason for using the executor is to maintain the code simpler and it definitely was easier to write upfront. The problem with ManagedLedger is that it normally needs to do several (blocking) steps like: talk with BK, talk with ZK,... And rolling out an async version means breaking all the steps in different functions. Probably at some point it will be convenient to do that. By now I thought that the only method that could greatly benefit (in terms of performance) from being truly async was the addEntry(). bq. 4. in line 183 of ManagedLedgerImpl, you use lastLedger.getLastAddConfirmed() as entry id to return the position. If concurrent addEntry operations arrives, you would read wrong entry id. why not use the entryid returned by lastLedger#addEntry. Besides that, I think getLastAddConfirmed doesn't equal to entryid you added for a writable ledger. ManagedLedgerImpl.addEntry is synchronized so there's should be no race condition there, but the problem was that the sync LedgerHandle.addEntry() doesn't return the entryId (like the asyncAddEntry does) bq. 5. for asyncAddEntry, addEntry in ManagedLedgerImpl, you don't change ledger when addEntry failed. The ledger would be closed when adding entry failed, you could not add more entries into it even the system goes back. You could refer BOOKKEEPER-74. Sure, I missed that bq. 6. trimConsumedLedgers would be triggered immediately if necessary when updating cursors. It might take time to deleteLedger & updateLedgersId. Is is possible to trim the ledgers lazily, e.g. running in background? Yes, I think it could be easily done in background using the executor. Thanks, I will shortly put the changes in the review board. > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-220: Attachment: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13286753#comment-13286753 ] Matteo Merli commented on BOOKKEEPER-220: - Updated to address Sijie comments 1, 2 & 5 4. I need to have LedgerHandle.addEntry() to return an entryId before changing it 6. Trimming in background: the code need some refactoring to do that because the trimming itself is a synchronized method, so even if we do it in background it will hold the lock delaying other operations. Updated patch / github / review board ( https://reviews.apache.org/r/5320/ ) > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-280) LedgerHandle.addEntry() should return an entryId
Matteo Merli created BOOKKEEPER-280: --- Summary: LedgerHandle.addEntry() should return an entryId Key: BOOKKEEPER-280 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-280 Project: Bookkeeper Issue Type: Bug Components: bookkeeper-client Affects Versions: 4.2.0 Reporter: Matteo Merli Priority: Minor Fix For: 4.2.0 LedgerHandle.asyncAddEntry callback provides the entryId of the newly added entry, but the synchronous version return void. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-280) LedgerHandle.addEntry() should return an entryId
[ https://issues.apache.org/jira/browse/BOOKKEEPER-280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-280: Attachment: 0001-BOOKKEEPER-280-LedgerHandle.addEntry-should-return-a.patch > LedgerHandle.addEntry() should return an entryId > > > Key: BOOKKEEPER-280 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-280 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-client >Affects Versions: 4.2.0 >Reporter: Matteo Merli >Priority: Minor > Fix For: 4.2.0 > > Attachments: > 0001-BOOKKEEPER-280-LedgerHandle.addEntry-should-return-a.patch > > > LedgerHandle.asyncAddEntry callback provides the entryId of the newly added > entry, but the synchronous version return void. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-280) LedgerHandle.addEntry() should return an entryId
[ https://issues.apache.org/jira/browse/BOOKKEEPER-280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-280: Attachment: 0001-BOOKKEEPER-280-LedgerHandle.addEntry-should-return-a.patch > LedgerHandle.addEntry() should return an entryId > > > Key: BOOKKEEPER-280 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-280 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-client >Affects Versions: 4.2.0 >Reporter: Matteo Merli >Priority: Minor > Fix For: 4.2.0 > > Attachments: > 0001-BOOKKEEPER-280-LedgerHandle.addEntry-should-return-a.patch, > 0001-BOOKKEEPER-280-LedgerHandle.addEntry-should-return-a.patch > > > LedgerHandle.asyncAddEntry callback provides the entryId of the newly added > entry, but the synchronous version return void. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13293311#comment-13293311 ] Matteo Merli commented on BOOKKEEPER-220: - @Ivan bq. I've just taken a look a the code. I like it a lot. It's very nice, neat and readable. I like the direction it's going. I have a few concerns though. Thank you for reviewing it. bq. My biggest concern is that the implementation doesn't take precautions so that split brain does not occur. If a managed ledger is being used as a write ahead log for node A, and node B falsely suspects that A is down, it can start writing to the the write ahead log for A and there's nothing to stop them. This isn't very difficult to prevent, but it needs to be done in your code. You need to use versioned zookeeper writes in the metastore, and ensure that when you create a ledger, all previous ledgers are closed. Yes, there is actually no enforcement that makes sure the a managed ledger is running only on one box and it would be definitely better to have some kind of protection. I wasn't sure on how to implement it (my first thought was to have a lock on zk), and the other reason is that from our side we are already protecting this using a zk lock (a topic maps to one managed ledger and one topic is served by one broker). Regarding versioned writes, what should be the behavior when a versioned write fails (meaning someone else is messing up with the managed ledger) ? Versioned writes would make the state consistent but there would still be problems in running the managed ledgers from different machines (consumer not getting messages, And, how can I verify that a ledger is closed for writing (I think I'd need to check only the last ledger in the list) ? bq. Another potential issue in the future is that the async and sync versions of addEntry do not share a code path. This could lead to bugs in the future which get fixed on one and not on the other. It would be better to have a completely async version, and then implement the sync version by calling the async and waiting on a countdown latch. Also, I think it would be nicer to have asyncAddEntry using the async calls and callbacks rather than a worker thread OR if using the worker threads, using the actor pattern. I prefer the former, as it will perform better as more requests can be outstanding at a time, but the latter works too. I'm all for making (at least) asyncAddEntry truly async and unifying the sync/async path. My only concern is how to deal when having to call more than one BK async method. For example, asyncAddEntry can require to create a new ledger if the current one is full. That would mean to call bk.asyncCreateLedger() and in the callback use the newly created ledger with ledger.asyncAddEntry(). Is this feasable? I think (maybe I'm wrong) that it's not safe to call BK api methods from inside a BK callback. Current implementation is cumbersome, but it's truly async in the most common case of addEntry, when not crossing ledger boundaries. bq. Instead of having managed cursor return a list of entries, how about having managed cursor itself present a iterator type interface where you can keep calling next() to get the next entry? Yes, I initially made the readEntries() that returns a list because I thought it was easier to get and easier to use for batch sending and let the client to pick up the batch size, but I'm open to change it. What I would'n want is to have both readEntries and the iterator style because that would be confusing. bq. One last small thing, is that it would be better to use a custom exception, rather than java.lang.Exception. Ok, I agree with that. What would be more appropriate, to use BKException or some other one (MLException, ...) ? > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please conta
[jira] [Updated] (BOOKKEEPER-280) LedgerHandle.addEntry() should return an entryId
[ https://issues.apache.org/jira/browse/BOOKKEEPER-280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-280: Attachment: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > LedgerHandle.addEntry() should return an entryId > > > Key: BOOKKEEPER-280 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-280 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-client >Affects Versions: 4.2.0 >Reporter: Matteo Merli >Assignee: Matteo Merli >Priority: Minor > Fix For: 4.2.0 > > Attachments: > 0001-BOOKKEEPER-280-LedgerHandle.addEntry-should-return-a.patch, > 0001-BOOKKEEPER-280-LedgerHandle.addEntry-should-return-a.patch > > > LedgerHandle.asyncAddEntry callback provides the entryId of the newly added > entry, but the synchronous version return void. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-280) LedgerHandle.addEntry() should return an entryId
[ https://issues.apache.org/jira/browse/BOOKKEEPER-280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-280: Attachment: (was: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch) > LedgerHandle.addEntry() should return an entryId > > > Key: BOOKKEEPER-280 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-280 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-client >Affects Versions: 4.2.0 >Reporter: Matteo Merli >Assignee: Matteo Merli >Priority: Minor > Fix For: 4.2.0 > > Attachments: > 0001-BOOKKEEPER-280-LedgerHandle.addEntry-should-return-a.patch, > 0001-BOOKKEEPER-280-LedgerHandle.addEntry-should-return-a.patch > > > LedgerHandle.asyncAddEntry callback provides the entryId of the newly added > entry, but the synchronous version return void. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-220: Attachment: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13401063#comment-13401063 ] Matteo Merli commented on BOOKKEEPER-220: - I've updated the attached patch to include the following : 1. Unified addEntry/asyncAddEntry always using the async api 2. Fence the ManagedLedger when the addEntry throws a FencedException 3. Use version when writing the list of ledgers to ZK 4. Use ManagedLedgerException instead of Exception List of individual commits : https://github.com/merlimat/managed-ledger/commits/master Review board: https://reviews.apache.org/r/5320/ > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-220: Attachment: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-220: Attachment: (was: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch) > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13401064#comment-13401064 ] Matteo Merli commented on BOOKKEEPER-220: - @Ivan, I haven't include the ManagedCursor.next() iterator style. I'm not sure how the corresponding async version should look like. Also, I think that the client choosing a big batch number of entries when reading should only affect the client, since managed ledger is running as a library on client side. > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13412447#comment-13412447 ] Matteo Merli commented on BOOKKEEPER-220: - I've addressed the issues pointed out by Ivan and incorporated its changes for the asyncOpen() method. Updated on github: https://github.com/merlimat/managed-ledger/commits/master Changelog : * Improved code coverage * Updated Position javadoc to not mention ledgerId/entryId * Splitted complete/failed methods in async callbacks * Fixed problems reported by FindBugs * Marked public interfaces with @Beta * Extracting interface for Position * Removed (non-Javadoc) automatically inserted comments * Removed protected from package-private items * Removed executor for opening managed ledger * Fixed deadlock caused by updated the list of ledgers while it was already being updated asynchronously. Few notes : bq. What happens if one reader marks delete on a position, which another reader on the same ledger hasn't read yet? A ledger is only deleted from BK when all the attached readers in the ML have consumed/markdeleted all the entries contained in that specific ledger. Position class: I've changed that to an (emptpy) interface to hide ledgerId/entryId. Sijie, this won't exactly address your issue, because the current implementation of ML assumes a specific type of position (PositionImpl) Protobuf: it would make sense to use protobuf for ML once all BK switches to it. For now I just thought the metadata required it's pretty simple and writing simple text content makes debugging easier. > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13412902#comment-13412902 ] Matteo Merli commented on BOOKKEEPER-220: - Ok, I thought it was meant to be included in a future release and with the binary encoding. I'll change the metadata. > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13412907#comment-13412907 ] Matteo Merli commented on BOOKKEEPER-220: - On a slightly related topic, we've been thinking about the ManagedCursor.markDelete() method. Its goal is to persist the last-message acknowledged by a client and it's implemented updating a z-node with the current acked-position. As we previously discussed, this generates a lot of updates into the cursors z-nodes on ZK. To circumvent this issue, we thought it may be possible to keep the cursors mark-deleted persisted positions directly in BK. That would mean using a separate BK ledger for each cursor and that markDelete() just appends the new position to the end of that ledger (after N updates the ledger will be deleted and the cursor will append to a new one). When doing a recovery (when the ML is reopened), we'd just need to read the last entry in each of the cursors ledgers. The only drawback would be in having more BK ledgers. Another schema could be to use a single BK ledger for all the cursors inside the same ML, and periodically snapshot all the cursors to avoid having to rescan the whole ledger when there is a slow consumer. Do you see any potential problem with these approach? Or a better way to do the same? > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13412993#comment-13412993 ] Matteo Merli commented on BOOKKEEPER-220: - Yes, there will be always the possibility of receiving at least one duplicate entry. However there could be differences in how applications can handle duplicates, namely in the how many duplicates... And one guarantee must be that we never re-deliver a successfully mark-deleted message. To summarize: applications can be lazy (according to their requirements) and call markDelete() every once in a while, but when it's called we need to make sure the position it's persisted appropriately. > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13413839#comment-13413839 ] Matteo Merli commented on BOOKKEEPER-220: - Ivan, HBase was what we always though as the solution for the metadata updates. The above was an idea to remove "the need" for Hbase (at least for the write throughput part), by storing the updates directly in BK. At the end of the day, even HBase has to log the cursor update as a new entry in its journal and every once in a while it will perform the compaction. This woud not be much different in BK, the cursor update is appended to the ledger and after a while (let say 1M updates) we create a new ledger and delete the old one. > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-351) asyncAddEntry should not throw an exception
[ https://issues.apache.org/jira/browse/BOOKKEEPER-351?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-351: Attachment: 0001-BOOKKEEPER-351-asyncAddEntry-should-not-throw-an-exc.patch 2nd version of the patch > asyncAddEntry should not throw an exception > --- > > Key: BOOKKEEPER-351 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-351 > Project: Bookkeeper > Issue Type: Bug >Affects Versions: 4.2.0 >Reporter: Matteo Merli >Assignee: Matteo Merli >Priority: Minor > Fix For: 4.2.0 > > Attachments: > 0001-BOOKKEEPER-351-asyncAddEntry-should-not-throw-an-exc.patch, > 0001-BOOKKEEPER-351-asyncAddEntry-should-not-throw-an-exc.patch > > > There are cases where LedgerHandle.asyncAddEntry() fails with a > RuntimeException that is thrown by executor.submit(). > It should better invoke the callback with a failure result. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-351) asyncAddEntry should not throw an exception
[ https://issues.apache.org/jira/browse/BOOKKEEPER-351?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13508441#comment-13508441 ] Matteo Merli commented on BOOKKEEPER-351: - [~hustlmsp] I've updated the test case with the above suggestions. Thanks for reminding this, I had it forgotten. > asyncAddEntry should not throw an exception > --- > > Key: BOOKKEEPER-351 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-351 > Project: Bookkeeper > Issue Type: Bug >Affects Versions: 4.2.0 >Reporter: Matteo Merli >Assignee: Matteo Merli >Priority: Minor > Fix For: 4.2.0 > > Attachments: > 0001-BOOKKEEPER-351-asyncAddEntry-should-not-throw-an-exc.patch, > 0001-BOOKKEEPER-351-asyncAddEntry-should-not-throw-an-exc.patch > > > There are cases where LedgerHandle.asyncAddEntry() fails with a > RuntimeException that is thrown by executor.submit(). > It should better invoke the callback with a failure result. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-524) Bookie journal filesystem gets full after SyncThread is terminated with exception
Matteo Merli created BOOKKEEPER-524: --- Summary: Bookie journal filesystem gets full after SyncThread is terminated with exception Key: BOOKKEEPER-524 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-524 Project: Bookkeeper Issue Type: Bug Components: bookkeeper-server Affects Versions: 4.2.0 Reporter: Matteo Merli The SyncThread get a NPE while the rest of the bookie is still running. This causes the journal gc to be stopped and the filesystem get full. Tue Dec 18 17:01:18 2012: Exception in thread "SyncThread" java.lang.NullPointerException Tue Dec 18 17:01:18 2012: at org.apache.bookkeeper.bookie.LedgerCacheImpl.getLedgerEntryPage(LedgerCacheImpl.java:153) Tue Dec 18 17:01:18 2012: at org.apache.bookkeeper.bookie.LedgerCacheImpl.flushLedger(LedgerCacheImpl.java:421) Tue Dec 18 17:01:18 2012: at org.apache.bookkeeper.bookie.LedgerCacheImpl.flushLedger(LedgerCacheImpl.java:363) Tue Dec 18 17:01:18 2012: at org.apache.bookkeeper.bookie.InterleavedLedgerStorage.flush(InterleavedLedgerStorage.java:148) Tue Dec 18 17:01:18 2012: at org.apache.bookkeeper.bookie.Bookie$SyncThread.run(Bookie.java:221) -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-524) Bookie journal filesystem gets full after SyncThread is terminated with exception
[ https://issues.apache.org/jira/browse/BOOKKEEPER-524?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-524: Attachment: 0001-BOOKKEEPER-524-Bookie-journal-filesystem-gets-full-a.patch I'm not particularly familiar with the LedgerCache code but I think returning null should be correct when the page is not found. > Bookie journal filesystem gets full after SyncThread is terminated with > exception > - > > Key: BOOKKEEPER-524 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-524 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-server >Affects Versions: 4.2.0 >Reporter: Matteo Merli > Attachments: > 0001-BOOKKEEPER-524-Bookie-journal-filesystem-gets-full-a.patch > > > The SyncThread get a NPE while the rest of the bookie is still running. This > causes the journal gc to be stopped and the filesystem get full. > Tue Dec 18 17:01:18 2012: Exception in thread "SyncThread" > java.lang.NullPointerException > Tue Dec 18 17:01:18 2012: at > org.apache.bookkeeper.bookie.LedgerCacheImpl.getLedgerEntryPage(LedgerCacheImpl.java:153) > Tue Dec 18 17:01:18 2012: at > org.apache.bookkeeper.bookie.LedgerCacheImpl.flushLedger(LedgerCacheImpl.java:421) > Tue Dec 18 17:01:18 2012: at > org.apache.bookkeeper.bookie.LedgerCacheImpl.flushLedger(LedgerCacheImpl.java:363) > Tue Dec 18 17:01:18 2012: at > org.apache.bookkeeper.bookie.InterleavedLedgerStorage.flush(InterleavedLedgerStorage.java:148) > Tue Dec 18 17:01:18 2012: at > org.apache.bookkeeper.bookie.Bookie$SyncThread.run(Bookie.java:221) -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-524) Bookie journal filesystem gets full after SyncThread is terminated with exception
[ https://issues.apache.org/jira/browse/BOOKKEEPER-524?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-524: Attachment: 0001-BOOKKEEPER-524-Bookie-journal-filesystem-gets-full-a.patch Added try/catch for the SyncThread. I've been trying BK with this patch in the same stress tests and the problem didn't reproduce anymore. > Bookie journal filesystem gets full after SyncThread is terminated with > exception > - > > Key: BOOKKEEPER-524 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-524 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-server >Affects Versions: 4.2.0 >Reporter: Matteo Merli >Priority: Blocker > Fix For: 4.2.0 > > Attachments: > 0001-BOOKKEEPER-524-Bookie-journal-filesystem-gets-full-a.patch, > 0001-BOOKKEEPER-524-Bookie-journal-filesystem-gets-full-a.patch > > > The SyncThread get a NPE while the rest of the bookie is still running. This > causes the journal gc to be stopped and the filesystem get full. > Tue Dec 18 17:01:18 2012: Exception in thread "SyncThread" > java.lang.NullPointerException > Tue Dec 18 17:01:18 2012: at > org.apache.bookkeeper.bookie.LedgerCacheImpl.getLedgerEntryPage(LedgerCacheImpl.java:153) > Tue Dec 18 17:01:18 2012: at > org.apache.bookkeeper.bookie.LedgerCacheImpl.flushLedger(LedgerCacheImpl.java:421) > Tue Dec 18 17:01:18 2012: at > org.apache.bookkeeper.bookie.LedgerCacheImpl.flushLedger(LedgerCacheImpl.java:363) > Tue Dec 18 17:01:18 2012: at > org.apache.bookkeeper.bookie.InterleavedLedgerStorage.flush(InterleavedLedgerStorage.java:148) > Tue Dec 18 17:01:18 2012: at > org.apache.bookkeeper.bookie.Bookie$SyncThread.run(Bookie.java:221) -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-49) bookkeeper - parallel async read same entry of same ledger will fail
[ https://issues.apache.org/jira/browse/BOOKKEEPER-49?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-49: --- Attachment: 0001-BOOKKEEPER-49-bookkeeper-parallel-async-read-same-en.patch I have this patch to fix the issue by maintaining a Multimap of ReadCompletion. The read completion is not associated with the response (there is no request id), it just ensures that the callback is called for each request. Not sure if this is the best solution in the long run but it's fixing the problem for ManagedLedger use cases. > bookkeeper - parallel async read same entry of same ledger will fail > > > Key: BOOKKEEPER-49 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-49 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-client >Affects Versions: 4.0.0, 4.1.0 >Reporter: Sijie Guo >Assignee: Sijie Guo > Fix For: 4.3.0 > > Attachments: > 0001-BOOKKEEPER-49-bookkeeper-parallel-async-read-same-en.patch > > > all ledgers shared a PerChannelBookieClient. > PerChannelBookieClient put all the read requests in a > ConcurrentHashMap map called readCompletions, > which is indexed by CompletionKey. If two read requests have same entryId and > same ledgerId, they have the same CompletionKey, the latter one will > overwrite the previous one. So a read request's callback will not be invoked. > we may need to chain the callbacks for same completion keys. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-558) change bookkeeper request/response protocol to use protobuf
[ https://issues.apache.org/jira/browse/BOOKKEEPER-558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13562570#comment-13562570 ] Matteo Merli commented on BOOKKEEPER-558: - Is some sort of compatibility taken into account on this issue? It would be nice to be able to upgrade to the protobuf enabled bookie without having to shutdown the service. Would it be possible/reasonable to have the bookie understand both protocol versions during a release cycle? > change bookkeeper request/response protocol to use protobuf > --- > > Key: BOOKKEEPER-558 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-558 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client, bookkeeper-server >Reporter: Sijie Guo >Assignee: Aniruddha > Fix For: 4.3.0 > > > Change the protocol to protobuf, which is easy to extend the protocol in > future. > [~i0exception] already did some works for it. he would generate the patch > later. created this jira in case of duplicated works. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-220) Managed Ledger proposal
[ https://issues.apache.org/jira/browse/BOOKKEEPER-220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-220: Attachment: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch I resubmit the ManagedLedger patch for review. Lots of bugfixes and testing has been done since the last patch. The full list of changes can be found on the github repository : https://github.com/merlimat/managed-ledger/commits/master > Managed Ledger proposal > --- > > Key: BOOKKEEPER-220 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-220 > Project: Bookkeeper > Issue Type: New Feature > Components: bookkeeper-client >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.3.0 > > Attachments: 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch, > 0001-BOOKKEEPER-220-Managed-Ledger-proposal.patch > > > The ManagedLedger design is based on our need to manage a set of ledgers, > with a single writer (at any point in time) and a set on consumers that read > entries from it. > The ManagedLedger also takes care of periodically closing ledgers to have a > "reasonable" sized sets of ledgers that can individually deleted when no more > needed. > I've put on github the interface proposal (along with an early WIP > implementation) > http://github.com/merlimat/managed-ledger -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-49) bookkeeper - parallel async read same entry of same ledger will fail
[ https://issues.apache.org/jira/browse/BOOKKEEPER-49?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13575867#comment-13575867 ] Matteo Merli commented on BOOKKEEPER-49: Rakesh, I think that there should be an Id on the request and that should be addressed in BOOKKEEPER-558. Once that jira is resolved we'll just need to keep the map with the key + reqId. > bookkeeper - parallel async read same entry of same ledger will fail > > > Key: BOOKKEEPER-49 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-49 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-client >Affects Versions: 4.0.0, 4.1.0 >Reporter: Sijie Guo >Assignee: Sijie Guo > Fix For: 4.3.0 > > Attachments: > 0001-BOOKKEEPER-49-bookkeeper-parallel-async-read-same-en.patch > > > all ledgers shared a PerChannelBookieClient. > PerChannelBookieClient put all the read requests in a > ConcurrentHashMap map called readCompletions, > which is indexed by CompletionKey. If two read requests have same entryId and > same ledgerId, they have the same CompletionKey, the latter one will > overwrite the previous one. So a read request's callback will not be invoked. > we may need to chain the callbacks for same completion keys. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-568) NPE during GC with HierarchicalLedgerManager
Matteo Merli created BOOKKEEPER-568: --- Summary: NPE during GC with HierarchicalLedgerManager Key: BOOKKEEPER-568 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-568 Project: Bookkeeper Issue Type: Bug Components: bookkeeper-server Affects Versions: 4.2.0 Reporter: Matteo Merli Priority: Minor {noformat} 2013-02-11 14:06:28,904 - WARN - [GarbageCollectorThread:ScanAndCompareGarbageCollector@103] - Exception when iterating over the metadata {} java.io.IOException: Error when check more elements at org.apache.bookkeeper.meta.HierarchicalLedgerManager$HierarchicalLedgerRangeIterator.hasNext(HierarchicalLedgerManager.java:423) at org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector.gc(ScanAndCompareGarbageCollector.java:75) at org.apache.bookkeeper.bookie.GarbageCollectorThread.doGcLedgers(GarbageCollectorThread.java:302) at org.apache.bookkeeper.bookie.GarbageCollectorThread.run(GarbageCollectorThread.java:271) Caused by: java.lang.NullPointerException at org.apache.bookkeeper.meta.HierarchicalLedgerManager$HierarchicalLedgerRangeIterator.hasNext(HierarchicalLedgerManager.java:419) ... 3 more {noformat} In the code below, l2NodesIter appears to be null. {code} public boolean hasNext() throws IOException { try { if (l1NodesIter == null) { l1NodesIter = zk.getChildren(ledgerRootPath, null).iterator(); hasMoreElement = nextL1Node(); } else if (!l2NodesIter.hasNext()) { hasMoreElement = nextL1Node(); } } catch (Exception e) { throw new IOException("Error when check more elements", e); } return hasMoreElement; } {code} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-568) NPE during GC with HierarchicalLedgerManager
[ https://issues.apache.org/jira/browse/BOOKKEEPER-568?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13576182#comment-13576182 ] Matteo Merli commented on BOOKKEEPER-568: - I'm not sure if doing {code} l2NodesIter == null || !l2NodesIter.hasNext() {code} would be the correct check. > NPE during GC with HierarchicalLedgerManager > > > Key: BOOKKEEPER-568 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-568 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-server >Affects Versions: 4.2.0 >Reporter: Matteo Merli >Priority: Minor > > {noformat} > 2013-02-11 14:06:28,904 - WARN - > [GarbageCollectorThread:ScanAndCompareGarbageCollector@103] - Exception when > iterating over the metadata {} > java.io.IOException: Error when check more elements > at > org.apache.bookkeeper.meta.HierarchicalLedgerManager$HierarchicalLedgerRangeIterator.hasNext(HierarchicalLedgerManager.java:423) > at > org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector.gc(ScanAndCompareGarbageCollector.java:75) > at > org.apache.bookkeeper.bookie.GarbageCollectorThread.doGcLedgers(GarbageCollectorThread.java:302) > at > org.apache.bookkeeper.bookie.GarbageCollectorThread.run(GarbageCollectorThread.java:271) > Caused by: java.lang.NullPointerException > at > org.apache.bookkeeper.meta.HierarchicalLedgerManager$HierarchicalLedgerRangeIterator.hasNext(HierarchicalLedgerManager.java:419) > ... 3 more > {noformat} > In the code below, l2NodesIter appears to be null. > {code} > public boolean hasNext() throws IOException { >try { > if (l1NodesIter == null) { > l1NodesIter = zk.getChildren(ledgerRootPath, null).iterator(); > hasMoreElement = nextL1Node(); > } else if (!l2NodesIter.hasNext()) { > hasMoreElement = nextL1Node(); > } >} catch (Exception e) { > throw new IOException("Error when check more elements", e); >} >return hasMoreElement; > } > {code} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-568) NPE during GC with HierarchicalLedgerManager
[ https://issues.apache.org/jira/browse/BOOKKEEPER-568?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-568: Attachment: 0001-BOOKKEEPER-568-NPE-during-GC-with-HierarchicalLedger.patch Here's the patch with a very simple test case that would throw exception without the fix > NPE during GC with HierarchicalLedgerManager > > > Key: BOOKKEEPER-568 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-568 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-server >Affects Versions: 4.2.0 >Reporter: Matteo Merli >Priority: Minor > Attachments: > 0001-BOOKKEEPER-568-NPE-during-GC-with-HierarchicalLedger.patch > > > {noformat} > 2013-02-11 14:06:28,904 - WARN - > [GarbageCollectorThread:ScanAndCompareGarbageCollector@103] - Exception when > iterating over the metadata {} > java.io.IOException: Error when check more elements > at > org.apache.bookkeeper.meta.HierarchicalLedgerManager$HierarchicalLedgerRangeIterator.hasNext(HierarchicalLedgerManager.java:423) > at > org.apache.bookkeeper.bookie.ScanAndCompareGarbageCollector.gc(ScanAndCompareGarbageCollector.java:75) > at > org.apache.bookkeeper.bookie.GarbageCollectorThread.doGcLedgers(GarbageCollectorThread.java:302) > at > org.apache.bookkeeper.bookie.GarbageCollectorThread.run(GarbageCollectorThread.java:271) > Caused by: java.lang.NullPointerException > at > org.apache.bookkeeper.meta.HierarchicalLedgerManager$HierarchicalLedgerRangeIterator.hasNext(HierarchicalLedgerManager.java:419) > ... 3 more > {noformat} > In the code below, l2NodesIter appears to be null. > {code} > public boolean hasNext() throws IOException { >try { > if (l1NodesIter == null) { > l1NodesIter = zk.getChildren(ledgerRootPath, null).iterator(); > hasMoreElement = nextL1Node(); > } else if (!l2NodesIter.hasNext()) { > hasMoreElement = nextL1Node(); > } >} catch (Exception e) { > throw new IOException("Error when check more elements", e); >} >return hasMoreElement; > } > {code} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-578) LedgerCacheImpl is reserving 1/3 of Heap size but allocates NonHeap memory
Matteo Merli created BOOKKEEPER-578: --- Summary: LedgerCacheImpl is reserving 1/3 of Heap size but allocates NonHeap memory Key: BOOKKEEPER-578 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-578 Project: Bookkeeper Issue Type: Bug Components: bookkeeper-server Reporter: Matteo Merli Priority: Minor By default the page limit parameter is set to -1, which means to assign 1/3 of Heap space to the LedgerCache. Each LedgerEntryPage is then allocating the memory outside the heap (ByteBuffer.allocateDirect()). This makes BK to use more memory than the -XmxNN configured setting. Is there any particular reason for the LedgerEntryPage buffer to be allocated outside the java heap? Could that be changed? -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-578) LedgerCacheImpl is reserving 1/3 of Heap size but allocates NonHeap memory
[ https://issues.apache.org/jira/browse/BOOKKEEPER-578?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13592528#comment-13592528 ] Matteo Merli commented on BOOKKEEPER-578: - Yes, I've seen the buffers are read with NIO from disk so it makes sense to direct buffers. The only thing, it's that using the default on 1/3 heap for that is misleading, because one assumes that memory will be allocated from inside the java heap. Say, if I configure the bookie jvm with -Xmx20g then the total amount of memory that the jvm will be using is 27Gb (20 + 20*1/3). > LedgerCacheImpl is reserving 1/3 of Heap size but allocates NonHeap memory > -- > > Key: BOOKKEEPER-578 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-578 > Project: Bookkeeper > Issue Type: Bug > Components: bookkeeper-server >Reporter: Matteo Merli >Priority: Minor > > By default the page limit parameter is set to -1, which means to assign 1/3 > of Heap space to the LedgerCache. Each LedgerEntryPage is then allocating the > memory outside the heap (ByteBuffer.allocateDirect()). > This makes BK to use more memory than the -XmxNN configured setting. Is there > any particular reason for the LedgerEntryPage buffer to be allocated outside > the java heap? Could that be changed? -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-595) Crash of inprocess autorecovery daemon should not take down the bookie
[ https://issues.apache.org/jira/browse/BOOKKEEPER-595?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13616620#comment-13616620 ] Matteo Merli commented on BOOKKEEPER-595: - +1 Looks goot to me along with BOOKKEEPER-594. Thanks > Crash of inprocess autorecovery daemon should not take down the bookie > -- > > Key: BOOKKEEPER-595 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-595 > Project: Bookkeeper > Issue Type: Sub-task >Reporter: Ivan Kelly >Assignee: Ivan Kelly > Fix For: 4.3.0 > > Attachments: > 0001-BOOKKEEPER-595-Crash-of-inprocess-autorecovery-daemo.patch > > > Autorecovery is a new feature. We should not bind the stability of the > relatively mature bookie daemon to this new daemon. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-605) Use static Logger objects everywhere
Matteo Merli created BOOKKEEPER-605: --- Summary: Use static Logger objects everywhere Key: BOOKKEEPER-605 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-605 Project: Bookkeeper Issue Type: Improvement Reporter: Matteo Merli Priority: Minor There are some classes where the SLF4J Logger is not declared as static. Some of them are classes that will have many instances LedgerEntry, PendingReadOp : {noformat} $ git grep LoggerFactory.getLogger | grep -v static bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerEntry.java: Logger LOG = LoggerFactory.getLogger(LedgerEntry.class); bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java: Logger LOG = LoggerFactory.getLogger(PendingReadOp.class); bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/NIOServerFactory.java: Logger LOG = LoggerFactory.getLogger(NIOServerFactory.class); bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerInputStream.java: Logger LOG = LoggerFactory.getLogger(LedgerInputStream.class); bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerOutputStream.java: Logger LOG = LoggerFactory.getLogger(LedgerOutputStream.class); bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LoopbackClient.java: Logger LOG = LoggerFactory.getLogger(LoopbackClient.class); hedwig-client/src/main/java/org/apache/hedwig/client/conf/ClientConfiguration.java: Logger logger = LoggerFactory.getLogger(ClientConfiguration.class); hedwig-client/src/main/java/org/apache/hedwig/client/handlers/CloseSubscriptionResponseHandler.java: LoggerFactory.getLogger(CloseSubscriptionResponseHandler.class); hedwig-client/src/main/java/org/apache/hedwig/client/netty/impl/AbstractSubscribeResponseHandler.java: LoggerFactory.getLogger(AbstractSubscribeResponseHandler.class); hedwig-client/src/main/java/org/apache/hedwig/client/netty/impl/multiplex/MultiplexSubscribeResponseHandler.java: LoggerFactory.getLogger(MultiplexSubscribeResponseHandler.class); hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookkeeperPersistenceManagerWhiteBox.java: LoggerFactory.getLogger(TestBookkeeperPersistenceManagerWhiteBox.class); {noformat} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-637) NoSuchEntry exception when reading an entry from a bookie should not print ERROR level message
Matteo Merli created BOOKKEEPER-637: --- Summary: NoSuchEntry exception when reading an entry from a bookie should not print ERROR level message Key: BOOKKEEPER-637 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-637 Project: Bookkeeper Issue Type: Improvement Affects Versions: 4.2.1, 4.3.0 Reporter: Matteo Merli Assignee: Matteo Merli Priority: Trivial The NoSuchEntry is an internal error that is recoverable within the BK client library by issuing a read to a different bookie. I think that INFO level should be more appropriate. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-637) NoSuchEntry exception when reading an entry from a bookie should not print ERROR level message
[ https://issues.apache.org/jira/browse/BOOKKEEPER-637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-637: Attachment: BOOKKEEPER-637.diff > NoSuchEntry exception when reading an entry from a bookie should not print > ERROR level message > -- > > Key: BOOKKEEPER-637 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-637 > Project: Bookkeeper > Issue Type: Improvement >Affects Versions: 4.2.1, 4.3.0 >Reporter: Matteo Merli >Assignee: Matteo Merli >Priority: Trivial > Attachments: BOOKKEEPER-637.diff > > > The NoSuchEntry is an internal error that is recoverable within the BK client > library by issuing a read to a different bookie. > I think that INFO level should be more appropriate. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-605) Use static Logger objects everywhere
[ https://issues.apache.org/jira/browse/BOOKKEEPER-605?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-605: Attachment: BOOKKEEPER-605.diff > Use static Logger objects everywhere > > > Key: BOOKKEEPER-605 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-605 > Project: Bookkeeper > Issue Type: Improvement >Reporter: Matteo Merli >Priority: Minor > Fix For: 4.3.0 > > Attachments: BOOKKEEPER-605.diff > > > There are some classes where the SLF4J Logger is not declared as static. Some > of them are classes that will have many instances LedgerEntry, PendingReadOp > : > {noformat} > $ git grep LoggerFactory.getLogger | grep -v static > bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerEntry.java: > Logger LOG = LoggerFactory.getLogger(LedgerEntry.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java: > Logger LOG = LoggerFactory.getLogger(PendingReadOp.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/NIOServerFactory.java: > Logger LOG = LoggerFactory.getLogger(NIOServerFactory.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerInputStream.java: > Logger LOG = LoggerFactory.getLogger(LedgerInputStream.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerOutputStream.java: > Logger LOG = LoggerFactory.getLogger(LedgerOutputStream.class); > bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LoopbackClient.java: > Logger LOG = LoggerFactory.getLogger(LoopbackClient.class); > hedwig-client/src/main/java/org/apache/hedwig/client/conf/ClientConfiguration.java: > Logger logger = LoggerFactory.getLogger(ClientConfiguration.class); > hedwig-client/src/main/java/org/apache/hedwig/client/handlers/CloseSubscriptionResponseHandler.java: > LoggerFactory.getLogger(CloseSubscriptionResponseHandler.class); > hedwig-client/src/main/java/org/apache/hedwig/client/netty/impl/AbstractSubscribeResponseHandler.java: > LoggerFactory.getLogger(AbstractSubscribeResponseHandler.class); > hedwig-client/src/main/java/org/apache/hedwig/client/netty/impl/multiplex/MultiplexSubscribeResponseHandler.java: > LoggerFactory.getLogger(MultiplexSubscribeResponseHandler.class); > hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookkeeperPersistenceManagerWhiteBox.java: > > LoggerFactory.getLogger(TestBookkeeperPersistenceManagerWhiteBox.class); > {noformat} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-605) Use static Logger objects everywhere
[ https://issues.apache.org/jira/browse/BOOKKEEPER-605?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13695072#comment-13695072 ] Matteo Merli commented on BOOKKEEPER-605: - Sure, that would be better. I didn't intend to generate a big diff, but it makes sense > Use static Logger objects everywhere > > > Key: BOOKKEEPER-605 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-605 > Project: Bookkeeper > Issue Type: Improvement >Reporter: Matteo Merli >Priority: Minor > Fix For: 4.3.0 > > Attachments: BOOKKEEPER-605.diff > > > There are some classes where the SLF4J Logger is not declared as static. Some > of them are classes that will have many instances LedgerEntry, PendingReadOp > : > {noformat} > $ git grep LoggerFactory.getLogger | grep -v static > bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerEntry.java: > Logger LOG = LoggerFactory.getLogger(LedgerEntry.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java: > Logger LOG = LoggerFactory.getLogger(PendingReadOp.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/NIOServerFactory.java: > Logger LOG = LoggerFactory.getLogger(NIOServerFactory.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerInputStream.java: > Logger LOG = LoggerFactory.getLogger(LedgerInputStream.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerOutputStream.java: > Logger LOG = LoggerFactory.getLogger(LedgerOutputStream.class); > bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LoopbackClient.java: > Logger LOG = LoggerFactory.getLogger(LoopbackClient.class); > hedwig-client/src/main/java/org/apache/hedwig/client/conf/ClientConfiguration.java: > Logger logger = LoggerFactory.getLogger(ClientConfiguration.class); > hedwig-client/src/main/java/org/apache/hedwig/client/handlers/CloseSubscriptionResponseHandler.java: > LoggerFactory.getLogger(CloseSubscriptionResponseHandler.class); > hedwig-client/src/main/java/org/apache/hedwig/client/netty/impl/AbstractSubscribeResponseHandler.java: > LoggerFactory.getLogger(AbstractSubscribeResponseHandler.class); > hedwig-client/src/main/java/org/apache/hedwig/client/netty/impl/multiplex/MultiplexSubscribeResponseHandler.java: > LoggerFactory.getLogger(MultiplexSubscribeResponseHandler.class); > hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookkeeperPersistenceManagerWhiteBox.java: > > LoggerFactory.getLogger(TestBookkeeperPersistenceManagerWhiteBox.class); > {noformat} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-605) Use static Logger objects everywhere
[ https://issues.apache.org/jira/browse/BOOKKEEPER-605?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-605: Attachment: BOOKKEEPER-605.diff Added final to all loggers > Use static Logger objects everywhere > > > Key: BOOKKEEPER-605 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-605 > Project: Bookkeeper > Issue Type: Improvement >Reporter: Matteo Merli >Priority: Minor > Fix For: 4.3.0 > > Attachments: BOOKKEEPER-605.diff, BOOKKEEPER-605.diff > > > There are some classes where the SLF4J Logger is not declared as static. Some > of them are classes that will have many instances LedgerEntry, PendingReadOp > : > {noformat} > $ git grep LoggerFactory.getLogger | grep -v static > bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerEntry.java: > Logger LOG = LoggerFactory.getLogger(LedgerEntry.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java: > Logger LOG = LoggerFactory.getLogger(PendingReadOp.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/NIOServerFactory.java: > Logger LOG = LoggerFactory.getLogger(NIOServerFactory.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerInputStream.java: > Logger LOG = LoggerFactory.getLogger(LedgerInputStream.class); > bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerOutputStream.java: > Logger LOG = LoggerFactory.getLogger(LedgerOutputStream.class); > bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LoopbackClient.java: > Logger LOG = LoggerFactory.getLogger(LoopbackClient.class); > hedwig-client/src/main/java/org/apache/hedwig/client/conf/ClientConfiguration.java: > Logger logger = LoggerFactory.getLogger(ClientConfiguration.class); > hedwig-client/src/main/java/org/apache/hedwig/client/handlers/CloseSubscriptionResponseHandler.java: > LoggerFactory.getLogger(CloseSubscriptionResponseHandler.class); > hedwig-client/src/main/java/org/apache/hedwig/client/netty/impl/AbstractSubscribeResponseHandler.java: > LoggerFactory.getLogger(AbstractSubscribeResponseHandler.class); > hedwig-client/src/main/java/org/apache/hedwig/client/netty/impl/multiplex/MultiplexSubscribeResponseHandler.java: > LoggerFactory.getLogger(MultiplexSubscribeResponseHandler.class); > hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookkeeperPersistenceManagerWhiteBox.java: > > LoggerFactory.getLogger(TestBookkeeperPersistenceManagerWhiteBox.class); > {noformat} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-604) Ledger storage can log an exception if GC happens concurrently.
[ https://issues.apache.org/jira/browse/BOOKKEEPER-604?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-604: Attachment: BOOKKEEPER-604.diff Catching the NoLedgerException and printing a log message when the race condition occours > Ledger storage can log an exception if GC happens concurrently. > --- > > Key: BOOKKEEPER-604 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-604 > Project: Bookkeeper > Issue Type: Bug >Reporter: Ivan Kelly > Fix For: 4.3.0 > > Attachments: BOOKKEEPER-604.diff > > > If a ledger is flushing, and part way through,GC kicks in, it can delete the > index file before we try and flush it. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-649) Race condition in sync ZKUtils.createFullPathOptimistic()
Matteo Merli created BOOKKEEPER-649: --- Summary: Race condition in sync ZKUtils.createFullPathOptimistic() Key: BOOKKEEPER-649 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-649 Project: Bookkeeper Issue Type: Bug Reporter: Matteo Merli Assignee: Matteo Merli Priority: Trivial Fix For: 4.3.0 If multiple threads are calling createFullPathOptimistic() there will be race conditions on creating the intermediate nodes. If the intermediate nodes have been created since we last check, we can just ignore the exception and continue creating the child nodes. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-649) Race condition in sync ZKUtils.createFullPathOptimistic()
[ https://issues.apache.org/jira/browse/BOOKKEEPER-649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-649: Attachment: 0001-BOOKKEEPER-649-Race-condition-in-sync-ZKUtils.create.patch > Race condition in sync ZKUtils.createFullPathOptimistic() > - > > Key: BOOKKEEPER-649 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-649 > Project: Bookkeeper > Issue Type: Bug >Reporter: Matteo Merli >Assignee: Matteo Merli >Priority: Trivial > Fix For: 4.3.0 > > Attachments: > 0001-BOOKKEEPER-649-Race-condition-in-sync-ZKUtils.create.patch > > > If multiple threads are calling createFullPathOptimistic() there will be race > conditions on creating the intermediate nodes. If the intermediate nodes have > been created since we last check, we can just ignore the exception and > continue creating the child nodes. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-649) Race condition in sync ZKUtils.createFullPathOptimistic()
[ https://issues.apache.org/jira/browse/BOOKKEEPER-649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13714203#comment-13714203 ] Matteo Merli commented on BOOKKEEPER-649: - [~rakeshr] In your patch you cathing the NodeExistsException in the zk.create(), but that would hide the error of creating the leaf node when it already exists. I think we should only catch the NodeExistsException in the recursive call because at that point we are creating empty nodes. > Race condition in sync ZKUtils.createFullPathOptimistic() > - > > Key: BOOKKEEPER-649 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-649 > Project: Bookkeeper > Issue Type: Bug >Reporter: Matteo Merli >Assignee: Matteo Merli >Priority: Trivial > Fix For: 4.3.0 > > Attachments: > 0001-BOOKKEEPER-649-Race-condition-in-sync-ZKUtils.create.patch, > 0002-BOOKKEEPER-649.patch, 0002-BOOKKEEPER-649.patch > > > If multiple threads are calling createFullPathOptimistic() there will be race > conditions on creating the intermediate nodes. If the intermediate nodes have > been created since we last check, we can just ignore the exception and > continue creating the child nodes. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-660) Logs too noisy on NIOServerFactory when client drops a connection
[ https://issues.apache.org/jira/browse/BOOKKEEPER-660?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-660: Attachment: BOOKKEEPER-660.diff > Logs too noisy on NIOServerFactory when client drops a connection > - > > Key: BOOKKEEPER-660 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-660 > Project: Bookkeeper > Issue Type: Bug >Affects Versions: 4.2.0 >Reporter: Ivan Kelly >Assignee: Matteo Merli >Priority: Minor > Fix For: 4.2.2 > > Attachments: BOOKKEEPER-660.diff > > > When a client drops a connection, the server throws an exception. It should > only log at info level and close the socket. > {code} > if (k.isReadable()) { > int rc = sock.read(incomingBuffer); > if (rc < 0) { > throw new IOException("Read error"); > } > if (incomingBuffer.remaining() == 0) { > {code} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Reopened] (BOOKKEEPER-596) Ledgers are gc'ed by mistake in MSLedgerManagerFactory.
[ https://issues.apache.org/jira/browse/BOOKKEEPER-596?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli reopened BOOKKEEPER-596: - It seems like the issue is not completely fixed by this patch, it looks probably worse. I've verified that after a garbage collection cleaning, some ledgers (whose metadata is untouched in ZK) were delete from the bookies. This eventually triggers error when either adding entries to these ledgers (infinite loop in the client.. ) or when reading (entries not found). This was using the HiearchicalLedgerManager (not sure the issue applies to all ledgers managers). I'm trying to isolate a simple way to reproduce the issue, for now I think it's more likely to happen when a big number of ledgers are deleted in a short time and hence collected in the same gc cycle. > Ledgers are gc'ed by mistake in MSLedgerManagerFactory. > --- > > Key: BOOKKEEPER-596 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-596 > Project: Bookkeeper > Issue Type: Bug >Affects Versions: 4.2.0, 4.2.1 >Reporter: Sijie Guo >Assignee: Sijie Guo >Priority: Blocker > Fix For: 4.2.2, 4.3.0 > > Attachments: > 0001-BOOKKEEPER-596-Ledgers-are-gc-ed-by-mistake-in-MSLed.patch, > 0001-BOOKKEEPER-596-Ledgers-are-gc-ed-by-mistake-in-MSLed.patch, > BOOKKEEPER-596.patch, BOOKKEEPER-596.patch, BOOKKEEPER-596.patch > > > details: > https://issues.apache.org/jira/browse/BOOKKEEPER-590?focusedCommentId=13616397&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13616397 -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-596) Ledgers are gc'ed by mistake in MSLedgerManagerFactory.
[ https://issues.apache.org/jira/browse/BOOKKEEPER-596?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13720920#comment-13720920 ] Matteo Merli commented on BOOKKEEPER-596: - Looks like it's more than the set inclusive of the last ledger in the range. I think some range might me missing when doing the scan: I create 10001 ledgers (to span over 2 ranges : 00/ and 00/0001) {code} LedgerRangeIterator iterator = getLedgerManager().getLedgerRanges(); while (iterator.hasNext()) { LedgerRange ledgerRange = iterator.next(); LOG.info("Found range: {}", ledgerRange.getLedgers()); } {code} but the iterator just gets the ledgers in the 00/0001 range > Ledgers are gc'ed by mistake in MSLedgerManagerFactory. > --- > > Key: BOOKKEEPER-596 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-596 > Project: Bookkeeper > Issue Type: Bug >Affects Versions: 4.2.0, 4.2.1 >Reporter: Sijie Guo >Assignee: Sijie Guo >Priority: Blocker > Fix For: 4.2.2, 4.3.0 > > Attachments: > 0001-BOOKKEEPER-596-Ledgers-are-gc-ed-by-mistake-in-MSLed.patch, > 0001-BOOKKEEPER-596-Ledgers-are-gc-ed-by-mistake-in-MSLed.patch, > BOOKKEEPER-596.patch, BOOKKEEPER-596.patch, BOOKKEEPER-596.patch > > > details: > https://issues.apache.org/jira/browse/BOOKKEEPER-590?focusedCommentId=13616397&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13616397 -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-596) Ledgers are gc'ed by mistake in MSLedgerManagerFactory.
[ https://issues.apache.org/jira/browse/BOOKKEEPER-596?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13720942#comment-13720942 ] Matteo Merli commented on BOOKKEEPER-596: - One issue is the list of l2 nodes is not sorted, and also the ledger ranges iterator is missing the last ledger range > Ledgers are gc'ed by mistake in MSLedgerManagerFactory. > --- > > Key: BOOKKEEPER-596 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-596 > Project: Bookkeeper > Issue Type: Bug >Affects Versions: 4.2.0, 4.2.1 >Reporter: Sijie Guo >Assignee: Sijie Guo >Priority: Blocker > Fix For: 4.2.2, 4.3.0 > > Attachments: > 0001-BOOKKEEPER-596-Ledgers-are-gc-ed-by-mistake-in-MSLed.patch, > 0001-BOOKKEEPER-596-Ledgers-are-gc-ed-by-mistake-in-MSLed.patch, > BOOKKEEPER-596.patch, BOOKKEEPER-596.patch, BOOKKEEPER-596.patch > > > details: > https://issues.apache.org/jira/browse/BOOKKEEPER-590?focusedCommentId=13616397&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13616397 -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-596) Ledgers are gc'ed by mistake in MSLedgerManagerFactory.
[ https://issues.apache.org/jira/browse/BOOKKEEPER-596?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-596: Attachment: BOOKKEEPER-596.diff Fixing the ledger ranges iterations > Ledgers are gc'ed by mistake in MSLedgerManagerFactory. > --- > > Key: BOOKKEEPER-596 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-596 > Project: Bookkeeper > Issue Type: Bug >Affects Versions: 4.2.0, 4.2.1 >Reporter: Sijie Guo >Assignee: Sijie Guo >Priority: Blocker > Fix For: 4.2.2, 4.3.0 > > Attachments: > 0001-BOOKKEEPER-596-Ledgers-are-gc-ed-by-mistake-in-MSLed.patch, > 0001-BOOKKEEPER-596-Ledgers-are-gc-ed-by-mistake-in-MSLed.patch, > BOOKKEEPER-596.diff, BOOKKEEPER-596.patch, BOOKKEEPER-596.patch, > BOOKKEEPER-596.patch > > > details: > https://issues.apache.org/jira/browse/BOOKKEEPER-590?focusedCommentId=13616397&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13616397 -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-663) HierarchicalLedgerManager iterator is missing some ranges and the last ledger in the range
Matteo Merli created BOOKKEEPER-663: --- Summary: HierarchicalLedgerManager iterator is missing some ranges and the last ledger in the range Key: BOOKKEEPER-663 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-663 Project: Bookkeeper Issue Type: Bug Reporter: Matteo Merli Assignee: Matteo Merli Fix For: 4.2.2, 4.3.0 The HierarchicalLedgerManager is missing some ledger ranges when iterating over 2nd level ranges. Also, within these ranges, the last ledger (*) it's not included in the iteration. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Resolved] (BOOKKEEPER-596) Ledgers are gc'ed by mistake in MSLedgerManagerFactory.
[ https://issues.apache.org/jira/browse/BOOKKEEPER-596?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli resolved BOOKKEEPER-596. - Resolution: Fixed Moving the last patch to BOOKKEEPER-663 > Ledgers are gc'ed by mistake in MSLedgerManagerFactory. > --- > > Key: BOOKKEEPER-596 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-596 > Project: Bookkeeper > Issue Type: Bug >Affects Versions: 4.2.0, 4.2.1 >Reporter: Sijie Guo >Assignee: Sijie Guo >Priority: Blocker > Fix For: 4.2.2, 4.3.0 > > Attachments: > 0001-BOOKKEEPER-596-Ledgers-are-gc-ed-by-mistake-in-MSLed.patch, > 0001-BOOKKEEPER-596-Ledgers-are-gc-ed-by-mistake-in-MSLed.patch, > BOOKKEEPER-596.diff, BOOKKEEPER-596.patch, BOOKKEEPER-596.patch, > BOOKKEEPER-596.patch > > > details: > https://issues.apache.org/jira/browse/BOOKKEEPER-590?focusedCommentId=13616397&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13616397 -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-663) HierarchicalLedgerManager iterator is missing some ranges and the last ledger in the range
[ https://issues.apache.org/jira/browse/BOOKKEEPER-663?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-663: Attachment: BOOKKEEPER-663.diff > HierarchicalLedgerManager iterator is missing some ranges and the last ledger > in the range > -- > > Key: BOOKKEEPER-663 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-663 > Project: Bookkeeper > Issue Type: Bug >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.2, 4.3.0 > > Attachments: BOOKKEEPER-663.diff > > > The HierarchicalLedgerManager is missing some ledger ranges when iterating > over 2nd level ranges. > Also, within these ranges, the last ledger (*) it's not included in the > iteration. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-663) HierarchicalLedgerManager iterator is missing some ranges and the last ledger in the range
[ https://issues.apache.org/jira/browse/BOOKKEEPER-663?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-663: Attachment: (was: BOOKKEEPER-663.diff) > HierarchicalLedgerManager iterator is missing some ranges and the last ledger > in the range > -- > > Key: BOOKKEEPER-663 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-663 > Project: Bookkeeper > Issue Type: Bug >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.2, 4.3.0 > > Attachments: BOOKKEEPER-663.patch > > > The HierarchicalLedgerManager is missing some ledger ranges when iterating > over 2nd level ranges. > Also, within these ranges, the last ledger (*) it's not included in the > iteration. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-663) HierarchicalLedgerManager iterator is missing some ranges and the last ledger in the range
[ https://issues.apache.org/jira/browse/BOOKKEEPER-663?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-663: Attachment: BOOKKEEPER-663.patch > HierarchicalLedgerManager iterator is missing some ranges and the last ledger > in the range > -- > > Key: BOOKKEEPER-663 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-663 > Project: Bookkeeper > Issue Type: Bug >Reporter: Matteo Merli >Assignee: Matteo Merli > Fix For: 4.2.2, 4.3.0 > > Attachments: BOOKKEEPER-663.patch > > > The HierarchicalLedgerManager is missing some ledger ranges when iterating > over 2nd level ranges. > Also, within these ranges, the last ledger (*) it's not included in the > iteration. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-665) BK client should not try to read entries from non-available bookies
Matteo Merli created BOOKKEEPER-665: --- Summary: BK client should not try to read entries from non-available bookies Key: BOOKKEEPER-665 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-665 Project: Bookkeeper Issue Type: Bug Reporter: Matteo Merli Assignee: Matteo Merli Priority: Minor If a bookie is not in the available list, we shouldn't try to read from it but just treat the read from that replica as failed. This could be especially true if the bookie node is partitioned because that could mean we need to wait the connection timeout. Also during the auto-replication of ledgers most of the logs consist of errors that say it was not possible to read from the failed bookie. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (BOOKKEEPER-665) BK client should not try to read entries from non-available bookies
[ https://issues.apache.org/jira/browse/BOOKKEEPER-665?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-665: Attachment: BOOKKEEPER-665.patch Patch to skip non-available bookies in reads. Not sure on how to unit test this behavior. > BK client should not try to read entries from non-available bookies > --- > > Key: BOOKKEEPER-665 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-665 > Project: Bookkeeper > Issue Type: Bug >Reporter: Matteo Merli >Assignee: Matteo Merli >Priority: Minor > Attachments: BOOKKEEPER-665.patch > > > If a bookie is not in the available list, we shouldn't try to read from it > but just treat the read from that replica as failed. > This could be especially true if the bookie node is partitioned because that > could mean we need to wait the connection timeout. Also during the > auto-replication of ledgers most of the logs consist of errors that say it > was not possible to read from the failed bookie. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-665) BK client should not try to read entries from non-available bookies
[ https://issues.apache.org/jira/browse/BOOKKEEPER-665?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13722726#comment-13722726 ] Matteo Merli commented on BOOKKEEPER-665: - If the bookie machine is reachable and the process is down, it fails fast and retries on the next bookie. But if it's partitioned (or process unresponsive) it will need to wait for the speculative read timeout which is by default 2s. But if we already know the bookie is down (not being in zk) we should avoid trying to read from that, or at least leave it as the last one to be tried after all the other replicas have failed the read operation too. > BK client should not try to read entries from non-available bookies > --- > > Key: BOOKKEEPER-665 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-665 > Project: Bookkeeper > Issue Type: Bug >Reporter: Matteo Merli >Assignee: Matteo Merli >Priority: Minor > Attachments: BOOKKEEPER-665.patch > > > If a bookie is not in the available list, we shouldn't try to read from it > but just treat the read from that replica as failed. > This could be especially true if the bookie node is partitioned because that > could mean we need to wait the connection timeout. Also during the > auto-replication of ledgers most of the logs consist of errors that say it > was not possible to read from the failed bookie. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-665) BK client should not try to read entries from non-available bookies
[ https://issues.apache.org/jira/browse/BOOKKEEPER-665?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13729641#comment-13729641 ] Matteo Merli commented on BOOKKEEPER-665: - If zk service totally down, only an application that just uses a single (or a few) ledger to read/write could possibly keep on working. I believe that in most of the cases applications are constantly opening/creating ledgers, in which case being able to operate without ZK will be of little help. If a bookie is partition from the network and it's the 1st one in the ensemble list, then every read to all the ledgers it contains will timeout in 2s, even if we have perfectly fine copies of that. Also, trying to read from dead bookies is especially annoying when doing automatic replication, because it tries to read every fragment from the failed bookie, filling the logs with 1000s of errors. > BK client should not try to read entries from non-available bookies > --- > > Key: BOOKKEEPER-665 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-665 > Project: Bookkeeper > Issue Type: Bug >Reporter: Matteo Merli >Assignee: Matteo Merli >Priority: Minor > Attachments: BOOKKEEPER-665.patch > > > If a bookie is not in the available list, we shouldn't try to read from it > but just treat the read from that replica as failed. > This could be especially true if the bookie node is partitioned because that > could mean we need to wait the connection timeout. Also during the > auto-replication of ledgers most of the logs consist of errors that say it > was not possible to read from the failed bookie. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (BOOKKEEPER-665) BK client should not try to read entries from non-available bookies
[ https://issues.apache.org/jira/browse/BOOKKEEPER-665?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13730866#comment-13730866 ] Matteo Merli commented on BOOKKEEPER-665: - I understand the goal of BOOKKEEPR-537, although to me seems more critical to avoid the potential 2s timeout on read than to survive a complete ZK unavailability. I am saying this, because: # Having read/write on current open ledgers, would just buy a few seconds before a new ledger would need be created triggering an error. # Our application uses ZK anyway to have locks on shared resources, so if ZK is down we need to shutdown anyway to avoid conflicts (I admit this might not be true for everyone) # Observing prod systems, the complete loss of the ZK quorum is usually due to : * Network hardware failures * Planned hardware maintenance (moving racks with zk servers inside, ...) All these scenarios are usually requiring manual intervention and the downtime would be measured in hours rather than just a few seconds. {quote} as I said, I am -1 on skipping solution. you possibly could reorder the read sequence to try read from bookies that available in zookeeper, but you need to take care since the checking is in the critical path that each read would access the check. an experiment result would help convincing this change. {quote} Ok, I first thought of leaving the reads from non-available bookies as the last resource, but the change looked complicated. I'll give it another try and will update here. {quote} I don't quite get this. if this bookie is a failed one, should automatic replication replicate entries from other hosts rather than the failed one? if not, properly a change is need on replication part not on normal part, right? And for logs problems, shall we review the logging part? {quote} Yes, this is probably better to be separated from this jira. Just as a quick overview, automatic replication is currently trying to read a few entries from all the bookies in the ensemble to verify the replication status of a segment. That obviously print many error messages since it never succeeds to connect to the "failed" bookie, which was the reason we initiated the auto-replication in the first place. > BK client should not try to read entries from non-available bookies > --- > > Key: BOOKKEEPER-665 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-665 > Project: Bookkeeper > Issue Type: Bug >Reporter: Matteo Merli >Assignee: Matteo Merli >Priority: Minor > Attachments: BOOKKEEPER-665.patch > > > If a bookie is not in the available list, we shouldn't try to read from it > but just treat the read from that replica as failed. > This could be especially true if the bookie node is partitioned because that > could mean we need to wait the connection timeout. Also during the > auto-replication of ledgers most of the logs consist of errors that say it > was not possible to read from the failed bookie. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-669) Race condition in ledger deletion and eviction from cache
Matteo Merli created BOOKKEEPER-669: --- Summary: Race condition in ledger deletion and eviction from cache Key: BOOKKEEPER-669 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-669 Project: Bookkeeper Issue Type: Bug Components: bookkeeper-server Reporter: Matteo Merli There is a race condition between when a ledger is delete and an eviction from LedgerCache occur. The resulting exception is: {code} 14:06:24.754 [SyncThread] ERROR org.apache.bookkeeper.bookie.Bookie - Exception in SyncThread java.lang.NullPointerException: null at org.apache.bookkeeper.bookie.LedgerCacheImpl.evictFileInfoIfNecessary(LedgerCacheImpl.java:809) ~[bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] at org.apache.bookkeeper.bookie.LedgerCacheImpl.getFileInfo(LedgerCacheImpl.java:267) ~[bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] at org.apache.bookkeeper.bookie.LedgerCacheImpl.flushLedger(LedgerCacheImpl.java:425) ~[bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] at org.apache.bookkeeper.bookie.LedgerCacheImpl.flushLedger(LedgerCacheImpl.java:382) ~[bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] at org.apache.bookkeeper.bookie.InterleavedLedgerStorage.flush(InterleavedLedgerStorage.java:167) ~[bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] at org.apache.bookkeeper.bookie.Bookie$SyncThread.run(Bookie.java:330) ~[bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] 14:06:24.755 [SyncThread] INFO org.apache.bookkeeper.bookie.Bookie - Triggering shutdown of Bookie-3181 with exitCode 5 14:06:24.768 [BookieShutdownTrigger] INFO org.apache.bookkeeper.bookie.Bookie - Shutting down Bookie-3181 with exitCode 5 14:06:24.769 [BookieJournal-3181] WARN org.apache.bookkeeper.bookie.Journal - Journal exits when shutting down {code} The problem is that the openLedger list is a normal LinkedList and sometimes is modified while synchronizing on fileInfoCache, other times on openLedgers, and in other places it is accessed without synchronizing. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-695) Some entry logs are not removed from the bookie storage
Matteo Merli created BOOKKEEPER-695: --- Summary: Some entry logs are not removed from the bookie storage Key: BOOKKEEPER-695 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-695 Project: Bookkeeper Issue Type: Bug Reporter: Matteo Merli Priority: Minor Some entry logs appear to be truncated (possible at a momemnt when the bookie was shut down) and the compaction is never getting rid of them: {code} 00:00:06.448 [GarbageCollectorThread] INFO o.a.b.bookie.GarbageCollectorThread - Extracting entry log meta from entryLogId: 1497 00:00:07.140 [GarbageCollectorThread] WARN o.a.b.bookie.GarbageCollectorThread - Premature exception when processing 1497 recovery will take care of the problem java.io.IOException: Short read for ledger entry from entryLog 1497@718702792(31356!=32840) at org.apache.bookkeeper.bookie.EntryLogger.scanEntryLog(EntryLogger.java:514) ~[bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] at org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLog(GarbageCollectorThread.java:572) [bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] at org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLogs(GarbageCollectorThread.java:549) [bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] at org.apache.bookkeeper.bookie.GarbageCollectorThread.run(GarbageCollectorThread.java:268) [bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] {code} These entry logs are not being removed: {code} $ ll -h *.log -rw-r--r-- 1 yahoo users 686M Jul 17 10:45 5d9.log -rw-r--r-- 1 yahoo users 634M Jul 27 19:31 b2d.log -rw-r--r-- 1 yahoo users 1.5G Jul 28 15:22 b35.log -rw-r--r-- 1 yahoo users 2.0G Aug 4 04:57 dbd.log -rw-r--r-- 1 yahoo users 2.0G Aug 4 04:58 dbe.log . {code} -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (BOOKKEEPER-695) Some entry logs are not removed from the bookie storage
[ https://issues.apache.org/jira/browse/BOOKKEEPER-695?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13797294#comment-13797294 ] Matteo Merli commented on BOOKKEEPER-695: - I'm not sure where it would be best to handle this case. My understanding is that we should ignore the truncated entries and compact/gc the entry log as needed. Perhaps it would be easier to handle it in GarbageCollectorThead.extractMetaFromEntryLog, so that it will only affect the GC scanning. If there are truncated/corrupted entries, most likely they will be replayed by the journal, but in any case we cannot do anything to bring them back to life, so we can just skip them. Something like: {code:java} try { entryLogger.scanEntryLog(entryLogId, scanner); } catch (IOException e) { /// print warning but continue } {code} > Some entry logs are not removed from the bookie storage > --- > > Key: BOOKKEEPER-695 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-695 > Project: Bookkeeper > Issue Type: Bug >Reporter: Matteo Merli >Priority: Minor > > Some entry logs appear to be truncated (possible at a momemnt when the bookie > was shut down) and the compaction is never getting rid of them: > {code} > 00:00:06.448 [GarbageCollectorThread] INFO > o.a.b.bookie.GarbageCollectorThread - Extracting entry log meta from > entryLogId: 1497 > 00:00:07.140 [GarbageCollectorThread] WARN > o.a.b.bookie.GarbageCollectorThread - Premature exception when processing > 1497 recovery will take care of the problem > java.io.IOException: Short read for ledger entry from entryLog > 1497@718702792(31356!=32840) > at > org.apache.bookkeeper.bookie.EntryLogger.scanEntryLog(EntryLogger.java:514) > ~[bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] > at > org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLog(GarbageCollectorThread.java:572) > [bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] > at > org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLogs(GarbageCollectorThread.java:549) > [bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] > at > org.apache.bookkeeper.bookie.GarbageCollectorThread.run(GarbageCollectorThread.java:268) > [bookkeeper-server-4.2.1.15.jar:4.2.2-SNAPSHOT] > {code} > These entry logs are not being removed: > {code} > $ ll -h *.log > -rw-r--r-- 1 yahoo users 686M Jul 17 10:45 5d9.log > -rw-r--r-- 1 yahoo users 634M Jul 27 19:31 b2d.log > -rw-r--r-- 1 yahoo users 1.5G Jul 28 15:22 b35.log > -rw-r--r-- 1 yahoo users 2.0G Aug 4 04:57 dbd.log > -rw-r--r-- 1 yahoo users 2.0G Aug 4 04:58 dbe.log > . > {code} -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (BOOKKEEPER-594) AutoRecovery shutting down on SyncDisconnected
[ https://issues.apache.org/jira/browse/BOOKKEEPER-594?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-594: Attachment: BOOKKEEPER-594.diff Proposed patch to make the ReplicationWorker more resilient to exceptions > AutoRecovery shutting down on SyncDisconnected > -- > > Key: BOOKKEEPER-594 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-594 > Project: Bookkeeper > Issue Type: Bug >Reporter: Ivan Kelly >Assignee: Ivan Kelly >Priority: Critical > Fix For: 4.3.0 > > Attachments: BOOKKEEPER-594.diff > > > Currently the AutoRecovery daemon will shut down on SyncDisconnected. This is > the wrong behaviour. It should wait until it gets a expired signal before > shutting down. If autoRecoveryDaemonEnabled=true, then the autorecovery > deamon is running in the same process as the bookie, the bookie death watcher > will take down the bookie at this point also, but as the bookie hasn't > shutdown, exit code will be 0, which is confusing to any monitoring app. -- This message was sent by Atlassian JIRA (v6.1#6144)