Skip to content

[PIP-63] Readonly-Topic-Ownership-Support #11960

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 1 commit into from

Conversation

Jason918
Copy link
Contributor

@Jason918 Jason918 commented Sep 8, 2021

Implementation of PIP-63

User manual

Here is the manual for admin and consumer to use this feature. This part is actually missing in PIP-63.

PIP-63 P1

For example, given a cluster called "my-cluster" with 2 brokers, 192.168.0.1 and 192.168.0.2.
It has namespace called "my-tenant/writer-namespace".And consumers are consuming with service url "pulsar://192.168.0.1:6550,192.168.0.2:6650".

Here is the steps to set up readonly namespace on a new broker 192.168.0.3.

  1. set up namespace isolation policy "writer-ns-policy" to avoid assigning new broker 192.168.0.3 to writer-namespace.
bin/pulsar-admin ns-isolation-policy set \
--auto-failover-policy-type min_available \
--auto-failover-policy-params min_limit=1,usage_threshold=80 \
--namespaces my-tenant/writer-namespace \
--primary 192.168.0.1,192.168.0.2 my-cluster writer-ns-policy
  1. Create a namespaces "my-tenant/reader-namespace" and set up writerNamespace=my-tenant/writer-namespace
pulsar-admin namespaces create my-tenant/reader-namespace
# This is a new admin command to be added.
pulsar-admin namespaces set-writer-namespace my-tenant/writer-namespace my-tenant/reader-namespace

  1. set up namespace isolation policy to assign 192.168.0.3 to reader-namespace.
bin/pulsar-admin ns-isolation-policy set \
--auto-failover-policy-type min_available \
--auto-failover-policy-params min_limit=1,usage_threshold=80 \
--namespaces my-tenant/reader-namespace \
--primary 192.168.0.3 my-cluster reader-ns-policy
  1. Update serviceUrl of some of the consumer clients to "pulsar://192.168.0.3:6550", and these consumers will consume messages from the readonly-namespace in 192.168.0.3Further more, the serviceUrl can be updated online with org.apache.pulsar.client.api.ServiceUrlProvider.
    Most importantly, we will not expose the readonly-namespace to consumer users, and no other configuration is needed to achieve this. Once admin sets up the readonly namespace, they can update serviceUrl on their own with some external service governance mechanism. So it's transparent to consumers. Here is image showing the new cluster state.

PIP-63 P2

Motivation

See PIP-63

Modifications

As described in PIP-63. This pr comes with the key changes.

A) Readonly namespace lookup

The following figure show the lookup procedure with some pre-settings.

  • Broker-1 is the current leader and all brokers uses the default centralized load manager, so that all bundle assignment will go through it.
  • Two ns isolation policies are set, for the convenience of explanation, all bundles of writer-namespace will be assigned to broker-2 and all bundles of reader-namespace will be assigned to broker-3.
  • The consumer client is consuming topic "persistent://my-tenant/writer-namespace/topic" with serviceUrl of "pulsar://192.168.0.3:6650"

image

A new string field redirectTopic needs to be added to CommandLookupTopicResponse.

In step 1, broker-3 have the info that the writerNamespace of "my-tenant/reader-namespace" is "my-tenant/writer-namespace" and broker-3 itself is in the isolation-policy of "my-tenant/reader-namespace". So when it receive lookup request of "my-tenant/writer-namespace", it will redirect the lookup to the leader and change the namespace of the lookup topic to "my-tenant/reader-namespace". So in the following steps(3-6), "persistent://my-tenant/reader-namespace/topic" is used to finish the lookup request.

B) How readonly topic owner read data

Related logic is mostly handled in org.apache.bookkeeper.mledger.impl.RemoteManagedLedgerImpl which extends ManagedLedgerImpl for readonly topics, and changed some key behaviors, including

  • Forbid all entry modification ops, like 'addEntry' and 'truncate'.
  • Read all ledger info from zk, and update LAC by call org.apache.bookkeeper.client.api.ReadHandle#readLastAddConfirmedAsync of the last ledger in the ledger list. See method RemoteManagedLedgerImpl#asyncUpdateLastConfirmedEntry() for details.
  • Using a org.apache.pulsar.client.api.CursorClient for managing cursors in writer topic. And it's RemoteManagedCursorImpl which extends ManagedCursorImpl used for cursor management in readonly topic.
  • For consumers, each time RemoteManagedCursorImpl#checkForNewEntries will trigger asyncUpdateLastConfirmedEntry to update LAC from bookies. It's not quite efficient for now, but I look forward to optimize this to a long polling way from broker (or bookie) in the following PRs, since this PR is already quite big for reviewers.

C) How does readonly topic owner keep metadata in-sync

Add a method watchManagedLedgerInfo in org.apache.bookkeeper.mledger.impl.MetaStore.
Every RemoteManagedLedgerImpl will watch its ManagedLedgerInfo to see if there is a new ledger rolled.

D) How does readonly topic owner handle acknowledgment

As described in PIP-63, A set of “cursor” related RPCs between writable topic owner and readonly topic owners is introduced for the acknowledgment. Here is the details.

  • CursorData GET_CURSOR(topic, subscription). Get cursor persisted state data, including persisted markdelete info of the giving persistent subscription from writable topic owner.
  • CursorData CREATE_CURSOR(topic, subscription, CursorData). Creating a new cursor for the giving subscription in writable topic owner, a cursor state data after creating is return. The cursor data in parameter is used for indicating initial consuming position and properties.
  • DELETE_CURSOR(topic, subscription). Delete a cursor when unsubscribe is called.
  • UPDATE_CURSOR(topic, subscription, CursorData). Update persisted cursor data for acknowledgment.
    User docs can be find in "pulsar-client-api/src/main/java/org/apache/pulsar/client/api/CursorClient.java"

A CursorClient is added for readonly topic owners to use these apis. A little different from PIP-63 in Topic Loading and Ack.

  • Topic Loading: when the readonly topic owner is loading the topic, CREATE_CURSOR is used to get the persisted data from writable topic owner.
  • Subscribe : calls CREATE_CURSOR to forward the subscribe request to writable topic owner if this topic is not already initialized in this broker. Upon successfully subscribe, readonly topic owner caches the corresponding cursor using returned cursor data.
  • Unsubscribe: remove cursor from cursor cache, and forward the unsubscribe request to writable topic owner with DELETE_CURSOR.
  • Ack: Process the ack same as writable topic owner, but instead of persist data to its managed ledger, readonly topic owner use UPDATE_CURSOR to persist the data through writable topic owner.

Verifying this change

  • Make sure that the change passes the CI checks.

This change added tests and can be verified as follows:

  • RemoteManagedCursorImplTest.java
  • RemoteManagedLedgerImplTest.java
  • CursorClientTest.java
  • ReadonlyTopicOwnerTest.java

Does this pull request potentially affect one of the following parts:

If yes was chosen, please highlight the changes

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API: (no)
  • The schema: (no)
  • The default values of configurations: (no)
  • The wire protocol: (no)
  • The rest endpoints: (no)
  • The admin cli options: (no)
  • Anything that affects deployment: (no)

Documentation

Check the box below and label this PR (if you have committer privilege).

  • doc

    (If this PR contains doc changes)

@BewareMyPower
Copy link
Contributor

You can apply following diff to fix C++ client build failure.

diff --git a/pulsar-client-cpp/lib/Commands.cc b/pulsar-client-cpp/lib/Commands.cc
index 87c149c5b87..760549aaa31 100644
--- a/pulsar-client-cpp/lib/Commands.cc
+++ b/pulsar-client-cpp/lib/Commands.cc
@@ -642,6 +642,18 @@ std::string Commands::messageType(BaseCommand_Type type) {
         case BaseCommand::END_TXN_ON_SUBSCRIPTION_RESPONSE:
             return "END_TXN_ON_SUBSCRIPTION_RESPONSE";
             break;
+        case BaseCommand::GET_CURSOR:
+            return "GET_CURSOR";
+        case BaseCommand::GET_CURSOR_RESPONSE:
+            return "GET_CURSOR_RESPONSE";
+        case BaseCommand::CREATE_CURSOR:
+            return "CREATE_CURSOR";
+        case BaseCommand::CREATE_CURSOR_RESPONSE:
+            return "CREATE_CURSOR_RESPONSE";
+        case BaseCommand::DELETE_CURSOR:
+            return "DELETE_CURSOR";
+        case BaseCommand::UPDATE_CURSOR:
+            return "UPDATE_CURSOR";
     };
     BOOST_THROW_EXCEPTION(std::logic_error("Invalid BaseCommand enumeration value"));
 }

@Jason918
Copy link
Contributor Author

Jason918 commented Sep 8, 2021

/pulsarbot run-failure-checks

@Jason918
Copy link
Contributor Author

Jason918 commented Sep 8, 2021

You can apply following diff to fix C++ client build failure.

diff --git a/pulsar-client-cpp/lib/Commands.cc b/pulsar-client-cpp/lib/Commands.cc
index 87c149c5b87..760549aaa31 100644
--- a/pulsar-client-cpp/lib/Commands.cc
+++ b/pulsar-client-cpp/lib/Commands.cc
@@ -642,6 +642,18 @@ std::string Commands::messageType(BaseCommand_Type type) {
         case BaseCommand::END_TXN_ON_SUBSCRIPTION_RESPONSE:
             return "END_TXN_ON_SUBSCRIPTION_RESPONSE";
             break;
+        case BaseCommand::GET_CURSOR:
+            return "GET_CURSOR";
+        case BaseCommand::GET_CURSOR_RESPONSE:
+            return "GET_CURSOR_RESPONSE";
+        case BaseCommand::CREATE_CURSOR:
+            return "CREATE_CURSOR";
+        case BaseCommand::CREATE_CURSOR_RESPONSE:
+            return "CREATE_CURSOR_RESPONSE";
+        case BaseCommand::DELETE_CURSOR:
+            return "DELETE_CURSOR";
+        case BaseCommand::UPDATE_CURSOR:
+            return "UPDATE_CURSOR";
     };
     BOOST_THROW_EXCEPTION(std::logic_error("Invalid BaseCommand enumeration value"));
 }

Thanks for your patch. :)

@Jason918
Copy link
Contributor Author

Jason918 commented Sep 8, 2021

/pulsarbot run-failure-checks

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.

This work is very interesting.
I see that many parts are not described in PIP-63 document.
Also, regarding the configuration I would add explicitly a flag that means that a namespace is a "readonly view" over another namespace.

@Jason918
Copy link
Contributor Author

Jason918 commented Sep 8, 2021

/pulsarbot run-failure-checks

1 similar comment
@Jason918
Copy link
Contributor Author

Jason918 commented Sep 8, 2021

/pulsarbot run-failure-checks

@Anonymitaet
Copy link
Member

@Jason918 Thanks for your contribution. For this PR, do we need to update docs?

(The PR template contains info about doc, which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)

@Jason918
Copy link
Contributor Author

Jason918 commented Sep 10, 2021

@Jason918 Thanks for your contribution. For this PR, do we need to update docs?

(The PR template contains info about doc, which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)

@Anonymitaet
Sorry for that, the PR template says that if you have committer privilege, maybe I didn't get that right. I thought it's for committers ( with write access to the project ) to decide about the docs.

About user doc, This PR is not ready for end users to use this feature in a very convenient way. For example, some parameters is added in namespace policies, but it is not covered in pulsar-admin or REST admin interface.
I would like to add these admin tools and user docs in the following PRs.

@Anonymitaet Anonymitaet added the doc Your PR contains doc changes, no matter whether the changes are in markdown or code files. label Sep 10, 2021
@Anonymitaet
Copy link
Member

@Jason918 thanks for your info!
I see you've already added some how-to instructions, so I label this PR with doc.
Please do not forget to add docs later to allow users to know your great code changes. And you can ping me to review the docs, thanks.

@Jason918
Copy link
Contributor Author

/pulsarbot run-failure-checks

2 similar comments
@Jason918
Copy link
Contributor Author

/pulsarbot run-failure-checks

@Jason918
Copy link
Contributor Author

/pulsarbot run-failure-checks

@Jason918
Copy link
Contributor Author

@codelipenghui @linlinnn Hi, can you please take a look at this PR?

@@ -1850,6 +1831,10 @@ public void operationFailed(ManagedLedgerException exception) {
});
}

protected void persistPosition(MarkDeleteEntry mdEntry, VoidCallback callback) {
Copy link
Member

Choose a reason for hiding this comment

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

A minor question, why we need this wrapper?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

RemoteManagedCursorImpl overrided this method and use CursorClient to persist this 'mdEntry' to topic write owner.

@Jason918
Copy link
Contributor Author

Jason918 commented Oct 6, 2021

/pulsarbot run-failure-checks

1 similar comment
@Jason918
Copy link
Contributor Author

Jason918 commented Oct 6, 2021

/pulsarbot run-failure-checks

@hangc0276 hangc0276 self-requested a review October 26, 2021 14:03
@Jason918
Copy link
Contributor Author

Jason918 commented Feb 7, 2022

This PR is too large, and very hard to review throughly.
Plan to do some rework and split this apart with PRs.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
doc Your PR contains doc changes, no matter whether the changes are in markdown or code files.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants