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

[FLINK-29830][Connector/Pulsar] Create the topic with schema before consuming messages in PulsarSinkITCase #21252

Merged
merged 1 commit into from Nov 11, 2022

Conversation

syhily
Copy link
Contributor

@syhily syhily commented Nov 7, 2022

What is the purpose of the change

The tests in PulsarSinkITCase may failed with some error logs like below. This is caused by we may consume messages on a blank topic with no schema. Pulsar topic's schema is define by the first message sent to it. Or you can create schema on topic manually.

Failed to subscribe for topic [persistent://public/default/bCQuHnEp] in topics consumer, subscribe error: org.apache.pulsar.client.api.PulsarClientException$IncompatibleSchemaException: {"errorMsg":"Topic does not have schema to check","reqId":2052942634743575747, "remote":"localhost/127.0.0.1:44835", "local":"/127.0.0.1:41334"}

In this test, we will manually create the topic schema before the test which fixes this race condition.

Brief change log

  • Create schema in PulsarSinkITCase.
  • Log all the exceptions in ControlSource for better debugging.

Verifying this change

This change is already covered by existing tests, such as:

  • PulsarSinkITCase

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

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (no)
  • The serializers: (no)
  • The runtime per-record code paths (performance sensitive): (no)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (no)
  • The S3 file system connector: (no)

Documentation

  • Does this pull request introduce a new feature? (no)
  • If yes, how is the feature documented? (not applicable)

@flinkbot
Copy link
Collaborator

flinkbot commented Nov 7, 2022

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run azure re-run the last Azure build

@syhily
Copy link
Contributor Author

syhily commented Nov 7, 2022

The CI failed in e2e tests. But it's not related to Pulsar. So I didn't trigger the CI again.

Copy link
Contributor

@XComp XComp left a comment

Choose a reason for hiding this comment

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

Thanks @syhily for this PR. I have a few comments. Please find them below.

this.desiredCounts = messageCounts;
this.consumedRecords = Collections.synchronizedList(new ArrayList<>(messageCounts));
this.deadline = new AtomicLong(timeout.toMillis() + System.currentTimeMillis());
this.executor = Executors.newSingleThreadExecutor();
ConsumerBuilder<String> consumerBuilder =
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
ConsumerBuilder<String> consumerBuilder =
final ConsumerBuilder<String> consumerBuilder =

nit

Comment on lines +240 to +244
if (exception != null) {
LOG.error("Error in consuming messages from Pulsar.");
LOG.error("", exception);
return true;
}
Copy link
Contributor

Choose a reason for hiding this comment

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

I feel like there is a non-optimal separation of concerns here: we shouldn't log in a method that checks the state but rather have the logging be done in the calling method.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yep. I think you are right.

.subscriptionType(Exclusive)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest);
this.consumer = sneakyClient(consumerBuilder::subscribe);
this.throwableException = new AtomicReference<>();
Copy link
Contributor

Choose a reason for hiding this comment

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

What about working with CompletableFutures here? It feels more natural to use rather than utilizing an AtomicReference here.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think it's ok to use CompletableFuture with better readability.

Comment on lines +213 to +214
.subscriptionMode(Durable)
.subscriptionType(Exclusive)
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
.subscriptionMode(Durable)
.subscriptionType(Exclusive)
.subscriptionMode(SubscriptionMode.Durable)
.subscriptionType(SubscriptionType.Exclusive)

nit: The enum usage here is not obvious because the enum identifiers are not uppercased. Could we replace the static import for these with a qualified access to improve readability?

private final int desiredCounts;
// This is a thread-safe list.
private final List<String> consumedRecords;
private final AtomicLong deadline;
private final ExecutorService executor;
private final Consumer<String> consumer;
Copy link
Contributor

Choose a reason for hiding this comment

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

The Consumer is only accessed in the constructor. I don't see a necessity to create a dedicated field for that one. Or am I missing something in this aspect? 🤔

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This Consumer is created and used in ControlSource with an SharedReference. Because we may use it both in a Flink testing instance and the testing code for consuming the messages from Pulsar and judge if we can stop this application.

It's quite ok to remove it. But I prefer to keep it with a close method. I'll add it in next commit.

Copy link
Member

@tisonkun tisonkun left a comment

Choose a reason for hiding this comment

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

test_ci_connect_2 passed. So I believe the necessary part of tests related to this patch pass.

Merging...

Style concern, if it's not enforced by the spotless plugin, varies from person to person. I believe who writes the most part of the code has the right to write code in his/her style.

Patching onto style is welcome, but it's not a blocker to the current patch.

Merge since the functionality part somehow blocks #21249.

@tisonkun tisonkun merged commit f8b3b33 into apache:master Nov 11, 2022
@syhily syhily deleted the pull/FLINK-29830 branch November 11, 2022 07:32
zentol pushed a commit to zentol/flink that referenced this pull request Nov 11, 2022
dchristle pushed a commit to dchristle/flink that referenced this pull request Nov 18, 2022
@XComp
Copy link
Contributor

XComp commented Nov 21, 2022

Thanks for taking over @tisonkun. I was offline for a week anyway. So, good that you unblocked PR #21249 by merging this PR.

Style concern, if it's not enforced by the spotless plugin, varies from person to person. I believe who writes the most part of the code has the right to write code in his/her style.

I agree with your view on that. Most of my comments were of cosmetic nature and not really blocking the PR. I created a follow-up issue FLINK-30109 to discuss the sneaky utility methods, though.

But please keep in mind to also maintain the corresponding Jira issue properly. This Jira issue is still open. No backports are created. @syhily may you create the backport PRs to finalize FLINK-29830?

XComp pushed a commit to XComp/flink that referenced this pull request Nov 28, 2022
XComp pushed a commit to XComp/flink that referenced this pull request Nov 28, 2022
XComp pushed a commit that referenced this pull request Dec 2, 2022
XComp pushed a commit that referenced this pull request Dec 2, 2022
sergeitsar pushed a commit to fentik/flink that referenced this pull request Feb 8, 2023
sergeitsar pushed a commit to fentik/flink that referenced this pull request Feb 8, 2023
akkinenivijay pushed a commit to krisnaru/flink that referenced this pull request Feb 11, 2023
cbornet pushed a commit to riptano/flink that referenced this pull request May 27, 2023
cbornet added a commit to riptano/flink that referenced this pull request May 27, 2023
…onsuming messages in PulsarSinkITCase. (apache#21252) (#4)

Co-authored-by: Yufan Sheng <yufan@streamnative.io>
cbornet added a commit to riptano/flink that referenced this pull request May 27, 2023
…onsuming messages in PulsarSinkITCase. (apache#21252) (#4)

Co-authored-by: Yufan Sheng <yufan@streamnative.io>
cbornet added a commit to cbornet/flink-connector-pulsar that referenced this pull request Jun 14, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
4 participants