Skip to content

Conversation

@jvrao
Copy link
Contributor

@jvrao jvrao commented Aug 3, 2018

Writer owns the metadata of the current ensemble ensemble only.
Previous ensemble segments of the same ledger can freely modified
by the replication worker.

In the current code, write ledger handle, which allows reads also
are blind sided by the non-disruptive ensemble changes performed
by the replication worker. This could potentially direct readers
to wrong destination, leading to unsuccessful reads.

Fix this problem by placing a watcher on the zk node just like
readOnlyLedgerHandle. When new metadata is received take the
older (non-current) ensemble segment information and version
number from the new metadata.

Signed-off-by: Venkateswararao Jujjuri (JV) vjujjuri@salesforce.com

Descriptions of the changes in this PR:

Motivation

(Explain: why you're making that change, what is the problem you're trying to solve)

Changes

(Describe: what changes you have made)

Master Issue: #


In order to uphold a high standard for quality for code contributions, Apache BookKeeper runs various precommit
checks for pull requests. A pull request can only be merged when it passes precommit checks. However running all
the precommit checks can take a long time, some trivial changes don't need to run all the precommit checks. You
can check following list to skip the tests that don't need to run for your pull request. Leave them unchecked if
you are not sure, committers will help you:

  • [skip bookkeeper-server bookie tests]: skip testing org.apache.bookkeeper.bookie in bookkeeper-server module.
  • [skip bookkeeper-server client tests]: skip testing org.apache.bookkeeper.client in bookkeeper-server module.
  • [skip bookkeeper-server replication tests]: skip testing org.apache.bookkeeper.replication in bookkeeper-server module.
  • [skip bookkeeper-server tls tests]: skip testing org.apache.bookkeeper.tls in bookkeeper-server module.
  • [skip bookkeeper-server remaining tests]: skip testing all other tests in bookkeeper-server module.
  • [skip integration tests]: skip docker based integration tests. if you make java code changes, you shouldn't skip integration tests.
  • [skip build java8]: skip build on java8. ONLY skip this when ONLY changing files under documentation under site.
  • [skip build java9]: skip build on java9. ONLY skip this when ONLY changing files under documentation under site.


Be sure to do all of the following to help us incorporate your contribution
quickly and easily:

If this PR is a BookKeeper Proposal (BP):

  • Make sure the PR title is formatted like:
    <BP-#>: Description of bookkeeper proposal
    e.g. BP-1: 64 bits ledger is support
  • Attach the master issue link in the description of this PR.
  • Attach the google doc link if the BP is written in Google Doc.

Otherwise:

  • Make sure the PR title is formatted like:
    <Issue #>: Description of pull request
    e.g. Issue 123: Description ...
  • Make sure tests pass via mvn clean apache-rat:check install spotbugs:check.
  • Replace <Issue #> in the title with the actual Issue number.

Writer owns the metadata of the current ensemble ensemble only.
Previous ensemble segments of the same ledger can freely modified
by the replication worker.

In the current code, write ledger handle, which allows reads also
are blind sided by the non-disruptive ensemble changes performed
by the replication worker. This could potentially direct readers
to wrong destination, leading to unsuccessful reads.

Fix this problem by placing a watcher on the zk node just like
readOnlyLedgerHandle. When new metadata is received take the
older (non-current) ensemble segment information and version
number from the new metadata.

Signed-off-by: Venkateswararao Jujjuri (JV) <vjujjuri@salesforce.com>
Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

Very interesting case.
Nice fix.
+1

@ivankelly shall we merge this change before you proceed with your refactor on immutable metadata ?

@eolivelli
Copy link
Contributor

@jvrao there are checkstyle issues, we must fix them before merging to master

2018-08-03T04:37:29.104 [INFO] Starting audit...
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java:62:8: Unused import: org.apache.bookkeeper.client.ReadOnlyLedgerHandle.MetadataUpdater. [UnusedImports]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java:34:8: Unused import: org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener. [UnusedImports]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java:58:8: Unused import: org.apache.bookkeeper.common.util.OrderedScheduler. [UnusedImports]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java:250: Line is longer than 120 characters (found 139). [LineLength]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java:251: Line is longer than 120 characters (found 132). [LineLength]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java:274: Line is longer than 120 characters (found 142). [LineLength]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java:284: Line is longer than 120 characters (found 130). [LineLength]

@eolivelli
Copy link
Contributor

@jvrao there are also compilation errors, maybe it is just enough to rebase to current master

2018-08-03\T\04:35:51.103 [ERROR] Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:3.7.0:testCompile (default-testCompile) on project bookkeeper-server: Compilation failure: Compilation failure:
2018-08-03\T\04:35:51.103 [ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_remaining_tests/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java:[230,70] incompatible types: capture#1 of ? extends java.util.List<org.apache.bookkeeper.net.BookieSocketAddress> cannot be converted to java.util.ArrayList<org.apache.bookkeeper.net.BookieSocketAddress>
2018-08-03\T\04:35:51.103 [ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_remaining_tests/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java:[243,43] incompatible types: java.util.List<org.apache.bookkeeper.net.BookieSocketAddress> cannot be converted to java.util.ArrayList<org.apache.bookkeeper.net.BookieSocketAddress>
2018-08-03\T\04:35:51.103 [ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_remaining_tests/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java:[248,97] incompatible types: capture#2 of ? extends java.util.List<org.apache.bookkeeper.net.BookieSocketAddress> cannot be converted to java.util.ArrayList<org.apache.bookkeeper.net.BookieSocketAddress>
2018-08-03\T\04:35:51.103 [ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_remaining_tests/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java:[272,70] incompatible types: capture#3 of ? extends java.util.List<org.apache.bookkeeper.net.BookieSocketAddress> cannot be converted to java.util.ArrayList<org.apache.bookkeeper.net.BookieSocketAddress>

Copy link
Contributor

@ivankelly ivankelly left a comment

Choose a reason for hiding this comment

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

I disagree that the write owns the current ensemble. The consistent metadata store owns it, the writer can only make suggestions to change it. The writer should never use the metadata until that exact copy has been stored on the metadata store, which is why having mutable metadata is so dangerous. Anyhow, a philosophical point, and unrelated to whether this change is good.

One thing to note is that this will trigger more load on zookeeper. For example, with Pulsar, you may have 100,000s topics each with a ledger open, that 100,000s new watches. So I would add a parameter to ClientConfiguration to make this optional, and off by default (to not add new load to unsuspecting users). @merlimat @sijie Opinions on this?

}
if (Version.Occurred.BEFORE == occurred) { // the metadata is updated
try {
bk.getMainWorkerPool().executeOrdered(ledgerId, new MetadataMerger(newMetadata));
Copy link
Contributor

Choose a reason for hiding this comment

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

There's no need to merge. The metadata read from zookeeper must have the same last ensemble as the metadata currently being used, or else we're violating a whole load of properties. So in theory, you should be able to assign newMetadata to metadata. In practice it can be tricky with all the mutation that occurs while handling bookie failure. So leave the merge for now, but I will remove it once the ledger immutable metadata changes are in (I should have remaining patches up today or monday/tuesday next week)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@ivankelly where are we now with the whole set of immutable changes?
There is another problem with this patch, the metadata is being accessed with and without lock in the code and that needs to be corrected too; may be covered as part of immutable changes. Also, we need to stop the writer as we discussed.

Copy link
Contributor

Choose a reason for hiding this comment

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

@jvrao the stack is blocked, waiting on #1589, but otherwise the change is pretty much ready.

}

// Shutdown a bookie in the last ensemble and continue writing
ArrayList<BookieSocketAddress> ensemble1, ensemble2, ensemble1n;
Copy link
Contributor

Choose a reason for hiding this comment

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

ArrayList -> List // I changed this a couple of days ago

@jvrao
Copy link
Contributor Author

jvrao commented Aug 3, 2018

Anyhow, a philosophical point, and unrelated to whether this change is good.

You are absolutely right. I will change my commit message.

@jvrao jvrao requested review from athanatos and sijie August 3, 2018 16:45
@jvrao
Copy link
Contributor Author

jvrao commented Aug 4, 2018

retest please

@sijie
Copy link
Member

sijie commented Aug 4, 2018

One thing to note is that this will trigger more load on zookeeper. For example, with Pulsar, you may have 100,000s topics each with a ledger open, that 100,000s new watches. So I would add a parameter to ClientConfiguration to make this optional, and off by default (to not add new load to unsuspecting users). @merlimat @sijie Opinions on this?

I think it is a good idea to have this behavior controlled by a flag.

@eolivelli
Copy link
Contributor

I agree we should make this optional, at least in a first version.
Maybe in the future we will activate it by default.

These days I am checking the reader part. Currently we add a watch for each reader, this is a waste of resources in some use case and it slows down reads. But this is another story, I will start a separate thread.

There is a stale patch on ZK about 'persistent recursive watches' which will reduce a lot the expense of resources in cases like ours, but it is not finding enough support/consensus to be accepted. Maybe some of you could take a look.

apache/zookeeper#136

@sijie
Copy link
Member

sijie commented Aug 6, 2018

Currently we add a watch for each reader, this is a waste of resources in some use case and it slows down reads.

I am not sure it is a waste of resources and why it would slow down reads.

the readers have to be notified with the ensemble changes, otherwise the readers will be stuck at tailing.

There is a stale patch on ZK about 'persistent recursive watches' which will reduce a lot the expense of resources in cases like ours,

I am not sure a recursive watches will help in this case. because a reader only cares about the ledgers that it is interested in. it doesn't care about all the ledgers. that says if you have thousands of clients, that each reader only cares about one ledger, how "persistent recursive watches" will help this situation.

@eolivelli
Copy link
Contributor

@sijie this is a very different use case from your experience with huge systems.

The case in which that watch is not very useful is this:

  • You have in the cluster only 1 or 2 bookies.
  • You are using BK for storing blobs for cery long time.

So you are never hitting ensemble changes, no automatic re-replication (only manual and in case of bad errors).
Ledgers metadata are very cold, they necwr change, so that watch will not be useful.
If you are randomly opening many ledgers and the closing them, you will create a lot of useless watches.

You can make many optimizations, like keeping open ledger handles in a cache, but those watches will have a cost and can be saved.

For the case in which there is an ensemble change the reader ledger handle can be reopened so that metadata are reread.

I will start another email thread, this is a bit off topic here

@sijie
Copy link
Member

sijie commented Aug 6, 2018

@sijie this is a very different use case from your experience with huge systems.

@eolivelli sorry. please open a thread for that. but my point is the existing behavior on readonly has its reason being there.

@eolivelli
Copy link
Contributor

@sijie

the existing behavior on readonly has its reason being there.

100% agree ! Regular BK usage needs that watch.

@eolivelli
Copy link
Contributor

retest this please

1 similar comment
@eolivelli
Copy link
Contributor

retest this please

@eolivelli
Copy link
Contributor

This patch needs a rebase.
It is an important change

@jvrao
Copy link
Contributor Author

jvrao commented Jan 3, 2019

retest this please

1 similar comment
@eolivelli
Copy link
Contributor

retest this please

@sijie
Copy link
Member

sijie commented Feb 21, 2019

@jvrao are you still working on this?

@eolivelli
Copy link
Contributor

@jvrao do you have committed this patch into Salesforce fork ?

cc @dlg99

@athanatos athanatos removed their request for review June 21, 2022 21:32
@StevenLuMT
Copy link
Member

fix old workflow,please see #3455 for detail

@hezhangjian
Copy link
Member

closed by no updates.

@hezhangjian hezhangjian closed this May 2, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants