Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

[fix][broker] Fix schema does not replicate successfully #17049

Merged

Conversation

codelipenghui
Copy link
Contributor

@codelipenghui codelipenghui commented Aug 10, 2022

Motivation

#11441 supports replicate schema to remote clusters.
But there is a mistake that the returned schema state is incorrect.

SchemaState getSchemaState() {
if (getSchemaInfo() == null) {
return SchemaState.Ready;
}
return schemaState;
}

Because the replicator used MessageImpl will not have the schema.
And this will cause the producer to skip the schema upload.

if (op.msg != null && op.msg.getSchemaState() == None) {
tryRegisterSchema(cnx, op.msg, op.callback, this.connectionHandler.getEpoch());
return;

We should remove

if (getSchemaInfo() == null) {
return SchemaState.Ready;
}

To return the correct schema state.

And then we should also provide the correct schema hash.
If the message is used by the replicator, the schema hash should
be based on the replicator schema. Otherwise, it should use based
on the schema of the message.

Modification

  • Fixed the incorrect returned schema state
  • Provide the method for getting schema hash for MessageImpl

Verification

Update the test only to create a producer to one cluster.
Because if create a producer for other clusters, the producer will
upload the schema. This is the reason why the test can get
passed before.

  • doc-not-needed

### Motivation

apache#11441 supports replicate schema to remote clusters.
But there is a mistake that the returned schema state is incorrect.

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java#L765-L770

Because the replicator used MessageImpl will not have the schema.
And this will cause the producer to skip the schema upload.

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2147-L2149

We should remove

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java#L766-L768

To return the correct schema state.

And then we should also provide the correct schema hash.
If the message is used by the replicator, the schema hash should
be based on the replicator schema. Otherwise, it should use based
on the schema of the message.

### Modification

- Fixed the incorrect returned schema state
- Provide the method for getting schema hash for MessageImpl

### Verification

Update the test only to create producer to one cluster.
Because if create a producer for other clusters, the producer will
upload the schema. This is the reason that why the test can get
pass before.
@codelipenghui codelipenghui self-assigned this Aug 10, 2022
@codelipenghui codelipenghui added this to the 2.12.0 milestone Aug 10, 2022
@codelipenghui codelipenghui added type/bug The PR fixed a bug or issue reported a bug area/broker labels Aug 10, 2022
@codelipenghui codelipenghui marked this pull request as ready for review August 10, 2022 16:46
Copy link
Member

@mattisonchao mattisonchao left a comment

Choose a reason for hiding this comment

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

LGTM

Copy link
Contributor

@gaoran10 gaoran10 left a comment

Choose a reason for hiding this comment

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

LGTM

@HQebupt
Copy link
Contributor

HQebupt commented Aug 13, 2022

/pulsarbot rerun-failure-checks

@codelipenghui codelipenghui force-pushed the penghui/fix-replicate-schema branch from 23a7e95 to a995883 Compare August 16, 2022 16:21
Comment on lines -1415 to -1421
Position deletedPos = cursor.getMarkDeletedPosition();
Position readPos = cursor.getReadPosition();

Awaitility.await().timeout(30, TimeUnit.SECONDS).until(
() -> cursor.getMarkDeletedPosition().getEntryId() != (cursor.getReadPosition().getEntryId() - 1));

assertNotEquals((readPos.getEntryId() - 1), deletedPos.getEntryId());
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed here because the producer might be closed with pending messages. After the cursor change to the close state, the producer will also be closed after a read entries failure. So that all the pending messages will fail with the Producer Already Closed exception, then the cursor will rewind

This will make the check failed.

@codelipenghui
Copy link
Contributor Author

There should be another issue related to this part

Here might break the message replication order. I'm working on a fix for now.
We should cherry-pick this one along with the ordering issue fix PR.

@codelipenghui
Copy link
Contributor Author

/pulsarbot run-failure-checks

@codelipenghui codelipenghui merged commit 7689133 into apache:master Aug 18, 2022
@codelipenghui codelipenghui deleted the penghui/fix-replicate-schema branch August 18, 2022 03:22
Technoboy- pushed a commit that referenced this pull request Aug 18, 2022
### Motivation

#11441 supports replicate schema to remote clusters.
But there is a mistake that the returned schema state is incorrect.

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java#L765-L770

Because the replicator used MessageImpl will not have the schema.
And this will cause the producer to skip the schema upload.

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2147-L2149

We should remove

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java#L766-L768

To return the correct schema state.

And then we should also provide the correct schema hash.
If the message is used by the replicator, the schema hash should
be based on the replicator schema. Otherwise, it should use based
on the schema of the message.

### Modification

- Fixed the incorrect returned schema state
- Provide the method for getting schema hash for MessageImpl
codelipenghui added a commit that referenced this pull request Aug 19, 2022
But there is a mistake that the returned schema state is incorrect.

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java#L765-L770

Because the replicator used MessageImpl will not have the schema.
And this will cause the producer to skip the schema upload.

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2147-L2149

We should remove

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java#L766-L768

To return the correct schema state.

And then we should also provide the correct schema hash.
If the message is used by the replicator, the schema hash should
be based on the replicator schema. Otherwise, it should use based
on the schema of the message.

- Fixed the incorrect returned schema state
- Provide the method for getting schema hash for MessageImpl

(cherry picked from commit 7689133)
@codelipenghui codelipenghui modified the milestones: 2.12.0, 2.11.0 Aug 19, 2022
nicoloboschi pushed a commit to datastax/pulsar that referenced this pull request Aug 22, 2022
But there is a mistake that the returned schema state is incorrect.

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java#L765-L770

Because the replicator used MessageImpl will not have the schema.
And this will cause the producer to skip the schema upload.

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2147-L2149

We should remove

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java#L766-L768

To return the correct schema state.

And then we should also provide the correct schema hash.
If the message is used by the replicator, the schema hash should
be based on the replicator schema. Otherwise, it should use based
on the schema of the message.

- Fixed the incorrect returned schema state
- Provide the method for getting schema hash for MessageImpl

(cherry picked from commit 7689133)
(cherry picked from commit c1d305b)
codelipenghui added a commit that referenced this pull request Sep 8, 2022
But there is a mistake that the returned schema state is incorrect.

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java#L765-L770

Because the replicator used MessageImpl will not have the schema.
And this will cause the producer to skip the schema upload.

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L2147-L2149

We should remove

https://github.com/apache/pulsar/blob/e826d849ceef9d6aef28569ad57950bba90dfff1/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java#L766-L768

To return the correct schema state.

And then we should also provide the correct schema hash.
If the message is used by the replicator, the schema hash should
be based on the replicator schema. Otherwise, it should use based
on the schema of the message.

- Fixed the incorrect returned schema state
- Provide the method for getting schema hash for MessageImpl

(cherry picked from commit 7689133)
@codelipenghui codelipenghui added cherry-picked/branch-2.9 Archived: 2.9 is end of life and removed cherry-picked/branch-2.11 labels Sep 8, 2022
@@ -134,7 +134,7 @@ function test_group_other() {
-Dexclude='**/ManagedLedgerTest.java,
**/OffloadersCacheTest.java
**/PrimitiveSchemaTest.java,
BlobStoreManagedLedgerOffloaderTest.java'
BlobStoreManagedLedgerOffloaderTest.java' -DtestReuseFork=false
Copy link
Member

Choose a reason for hiding this comment

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

@codelipenghui do you remember why this was needed? I added #17943 to remove it.

Copy link
Contributor

Choose a reason for hiding this comment

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

The motivation is here #17152

Although I think it's safe to add it back

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker cherry-picked/branch-2.9 Archived: 2.9 is end of life cherry-picked/branch-2.10 doc-not-needed Your PR changes do not impact docs release/2.9.4 release/2.10.2 type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging this pull request may close these issues.

9 participants