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-22902] Port KafkaSink to FLIP-143 #16676

Merged
merged 3 commits into from
Aug 6, 2021
Merged

Conversation

fapaul
Copy link

@fapaul fapaul commented Aug 2, 2021

What is the purpose of the change

This commit introduces a new KafkaSink which is based on FLIP-143.

Brief change log

Besides adding the new KafkaSink the PR has the following additional commits.

  • ca1c2e6 to extract a test utility for finding the latest completed checkpoint of a job
  • b640ffc introduces a utility to extract needed information from the Sink.InitContext to a SerializationSchema

Verifying this change

The changes are covered by multiple unit tests and also integration tests against a real Kafka cluster.

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

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

Documentation

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

We also plan to add the documentation which is tracked as part of https://issues.apache.org/jira/browse/FLINK-23664

@flinkbot
Copy link
Collaborator

flinkbot commented Aug 2, 2021

Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
to review your pull request. We will use this comment to track the progress of the review.

Automated Checks

Last check on commit 483a1a6 (Sat Aug 28 13:11:08 UTC 2021)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!

Mention the bot in a comment to re-run the automated checks.

Review Progress

  • ❓ 1. The [description] looks good.
  • ❓ 2. There is [consensus] that the contribution should go into to Flink.
  • ❓ 3. Needs [attention] from.
  • ❓ 4. The change fits into the overall [architecture].
  • ❓ 5. Overall code [quality] is good.

Please see the Pull Request Review Guide for a full explanation of the review process.


The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commands
The @flinkbot bot supports the following commands:

  • @flinkbot approve description to approve one or more aspects (aspects: description, consensus, architecture and quality)
  • @flinkbot approve all to approve all aspects
  • @flinkbot approve-until architecture to approve everything until architecture
  • @flinkbot attention @username1 [@username2 ..] to require somebody's attention
  • @flinkbot disapprove architecture to remove an approval you gave earlier

@flinkbot
Copy link
Collaborator

flinkbot commented Aug 2, 2021

CI report:

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

@fapaul fapaul force-pushed the FLINK-23124 branch 2 times, most recently from 22e7bb8 to 0f0ab58 Compare August 2, 2021 15:14
@AHeise AHeise self-assigned this Aug 3, 2021
Copy link
Contributor

@AHeise AHeise left a comment

Choose a reason for hiding this comment

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

Thank you very much for your contribution. The general structure looks good and I'm leaving a first impression of the production code.

@fapaul fapaul force-pushed the FLINK-23124 branch 7 times, most recently from b13e3be to 3b5f480 Compare August 4, 2021 14:20
@fapaul
Copy link
Author

fapaul commented Aug 4, 2021

@AHeise thanks for your review. I have addressed all your comments, please have another look.

@fapaul
Copy link
Author

fapaul commented Aug 5, 2021

@flinkbot run azure

Copy link
Contributor

@AHeise AHeise left a comment

Choose a reason for hiding this comment

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

A more detailed round. I have not deeply looked into the IT but the structure looks good and the covered cases should be sufficient.

class KafkaWriterState {
private final int subtaskId;
private final long transactionalIdOffset;
private final String transactionalIdPrefix;
Copy link
Contributor

Choose a reason for hiding this comment

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

By storing it here, do you effectively allow users to change the prefix even when resuming from checkpoint?

Copy link
Author

Choose a reason for hiding this comment

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

I have to store it here to abort transactions from previous runs. If the job is stopped and started with a new prefix the new one is used for all newly created states.

Copy link
Contributor

Choose a reason for hiding this comment

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

Okay nice. Do we want to expose it to the user that they may change the prefix or should we communicate that the prefix should remain stable? I'm assuming quite a few edge cases would not work well if a prefix is changed (think of lingering transactions opened before downscaling without recent checkpoint). So I would probably communicate that the prefix is supposed to be stable for now.

Copy link
Author

Choose a reason for hiding this comment

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

Yeah downscaling before checkpoint case is definitely a problem, I can update the doc string to hint that the prefix should remain stable.

* Exposes information about how man records have been emitted overall and at the beginning of a
* checkpoint.
*/
private static final class InfiniteIntegerSource
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you just use a env.fromSequence(0, Long.MAX_VALUE) with a chained map that implements this functionality? You are making our future lives harder :/

Copy link
Author

Choose a reason for hiding this comment

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

I do not see how I can easily replace it because I am relying on the fact that the Source finishes after the first checkpointCompleted event.

Copy link
Contributor

Choose a reason for hiding this comment

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

Ah correct. Let's keep it this way then.

@fapaul fapaul force-pushed the FLINK-23124 branch 2 times, most recently from e8c6ec0 to 5f63691 Compare August 6, 2021 09:50
@fapaul fapaul marked this pull request as ready for review August 6, 2021 10:11
Copy link
Contributor

@AHeise AHeise left a comment

Choose a reason for hiding this comment

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

A few more nits.

/**
* Adapter between {@link Sink.InitContext} and {@link SerializationSchema.InitializationContext}.
*/
public class InitContextInitializationContextAdapter
Copy link
Contributor

Choose a reason for hiding this comment

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

I see that this is implemented similarly to RuntimeContextSerializationInitializationContextAdapter but has the same flaws.
In general, when implementing a small class it should not rely on a big class to be injected. That makes reuse and testing much harder. In this case, it would be much better to just pass the UserCodeClassLoader and the MetricGroup directly and have no dependency to the InitContext.
Now the metric group should only be added when it's actually needed, so here I would rather use a Supplier<MetricGroup> (it doesn't make sense that the InitContext is passed to this Adapter and then passed back to the mapMetricGroup; the InitContext is by definition available on call-site)

So signature should be

public InitContextInitializationContextAdapter(
            Supplier<MetricGroup> metricGroupSupplier, UserCodeClassLoader userCodeClassLoader)

Finally we should cache the result from the Supplier most easily by using Supplier#memoize of Guava. I'd probably wrap the ctor parameter before assigning it to the field.

Copy link
Contributor

Choose a reason for hiding this comment

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

I wonder if this should be in flink-connector-base instead.

Copy link
Author

Choose a reason for hiding this comment

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

I wonder if this should be in flink-connector-base instead.

It can also live in flink-connector-base I just refrained from putting it there because there is no sink-specific code yet and all these adapters are currently also in core.

@@ -84,4 +92,32 @@ public static void waitUntilJobInitializationFinished(
() -> clusterClient.requestJobResult(id).get(),
userCodeClassloader);
}

public static File getMostRecentCompletedCheckpoint(File checkpointDir) throws IOException {
return Files.find(checkpointDir.toPath(), 2, TestUtils::isCompletedCheckpoint)
Copy link
Contributor

Choose a reason for hiding this comment

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

FYI there is a bug here https://issues.apache.org/jira/browse/FLINK-23647. But we would fix it with that ticket.

Copy link
Author

Choose a reason for hiding this comment

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

Which ticket do you mean by that ticket?

Comment on lines +287 to +289
private void testRecoveryWithAssertion(
DeliveryGuarantee guarantee, java.util.function.Consumer<List<Long>> recordsAssertion)
throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

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

FYI, there is another pattern that can be used to implement such a wrapping setup/cleanup code.

   AutoCloseableResult testRecovery(DeliveryGuarantee guarantee) {
       // execute common code
      result = // fetch result
      AutoCloseable after =  () -> { // after code };
      return wrap(result, after);
   }

You can then use the return value in a try-and-resource and add all your assertions in the block. It has huge benefits over your pattern when you have checked exceptions and often is easier on the eye when the auto-formatter went over it.

Copy link
Author

Choose a reason for hiding this comment

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

Thanks for the suggestion, I get the idea. I will try to facilitate it next time or do you want to have the tests refactored?

*
* @param transactionalIdPrefix prefix for the id
* @param subtaskId describing the subtask which is opening the transaction
* @param offset an always incrementing number usually capturing the number of checkpoints taken
Copy link
Contributor

Choose a reason for hiding this comment

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

This could be confused with Kafka offset. Maybe use seq number?

Copy link
Author

@fapaul fapaul Aug 6, 2021

Choose a reason for hiding this comment

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

I renamed it to checkpointOffset and I hope the docstring for the parameter explains enough to make it apparent it has nothing to do with the partition offset

Comment on lines 43 to 44
public static String buildTransactionalId(
String transactionalIdPrefix, int subtaskId, long offset) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Here we could also have an instantiable TransactionalIdFactory with constant transactionalIdPrefix and subtaskId. You could then have a pre-computed subtask prefix consisting of

prefix = sb.append(transactionalIdPrefix)
                .append(TRANSACTIONAL_ID_DELIMITER)
                .append(subtaskId)
                .append(TRANSACTIONAL_ID_DELIMITER)
               ```
               
               Then this method would just `return prefix + offset`;

Copy link
Author

Choose a reason for hiding this comment

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

Hmm, it means we need to instantiate the factory basically for every transaction. What would be the benefit?

Fabian Paul added 2 commits August 6, 2021 14:19
This commit introduces a new KafkaSink which is based on
FLIP-143.
Copy link
Contributor

@AHeise AHeise left a comment

Choose a reason for hiding this comment

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

LGTM. Thank you very much!

@AHeise AHeise merged commit 8719481 into apache:master Aug 6, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants