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

Add ability to use Kafka's sinks as pulsar sinks #9927

Merged
merged 11 commits into from
Jun 2, 2021

Conversation

dlg99
Copy link
Contributor

@dlg99 dlg99 commented Mar 16, 2021

Motivation

Provide a way to use Kafka-Connect Sink as a Pulsar Sink, in cases like:

  • company has custom kafka sink and want to try the pulsar out
  • no corresponding pulsar sink exists
    etc.

Modifications

Added KafkaConnectSink. Kafka Schema is autodetected from key/value itself for primitive values with option to unwrap KeyValue. GenericRecord support is TBD, depends on the changes Enrico is working on.
Added profile to pulsar-io/kafka-connect-adaptor to build nar with kafka-connect connector included

Verifying this change

  • Make sure that the change passes the CI checks.

This change added tests and can be verified as follows:

Added unit tests.
Tested locally as

Ran pulsar standalone bin/pulsar standalone
add dependency

       <dependency>
          <groupId>org.apache.kafka</groupId>
          <artifactId>connect-file</artifactId>
          <version>${kafka-client.version}</version>
        </dependency>

into pulsar-io/kafka-connect-adaptor-nar/pom.xml
Built nar as `mvn -f pulsar-io/kafka-connect-adaptor-nar/pom.xml clean package -DskipTests to include kafka's connect-file sink into the nar.
Ran test nar as

bin/pulsar-admin sinks localrun -a ./pulsar-io/kafka-connect-adaptor-nar/target/pulsar-io-kafka-connect-adaptor-nar-2.8.0-SNAPSHOT.nar --name kwrap --namespace public/default/ktest --parallelism 1 -i my-topic --sink-config-file ~/sink.yaml

with

$ cat ~/sink.yaml
processingGuarantees: "EFFECTIVELY_ONCE"
configs:
  "topic": "my-topic"
  "offsetStorageTopic": "kafka-connect-sink-offset"
  "pulsarServiceUrl": "pulsar://localhost:6650/" 
  "kafkaConnectorSinkClass": "org.apache.kafka.connect.file.FileStreamSinkConnector"
  "kafkaConnectorConfigProperties":
    "file": "/tmp/sink_test.out"

set topic schema as

bin/pulsar-admin schemas upload --filename ~/schema.json my-topic

$ cat ~/schema.json
{
    "type": "STRING",
    "schema": "",
    "properties": {
        "key1": "value1"
    }
}

message produced as

bin/pulsar-client produce my-topic --messages "hello-pulsar"

and got

$ cat /tmp/sink_test.out
hello-pulsar

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

  • Does this pull request introduce a new feature? (yes)
  • If yes, how is the feature documented? (not documented yet)
  • If a feature is not documented yet in this PR, please create a followup issue for adding the documentation
    TBD following review

@dlg99
Copy link
Contributor Author

dlg99 commented Mar 16, 2021

this replaces #9825 - reworked according to @sijie 's feedback
@sijie and @eolivelli please take a look.

@yuvalgut
Copy link

cool! that will be really useful.
for supporting avro we can check if the Object is type of org.apache.pulsar.client.impl.schema.generic.GenericAvroRecord and then get hold of the actual avro record with '.getAvroRecord()'.
io.confluent.connect.avro.AvroData will let us to convert the avro into kafka SchemaAndValue:
avroData.toConnectData(avroRecord.getSchema(), avroRecord) and finally generate kafka SinkRecord.

just throwing some ideas...

@dlg99
Copy link
Contributor Author

dlg99 commented Mar 16, 2021

@yuvalgut thank you!
I postponed dealing with GenericRecords for now to make use of the changes @eolivelli is working on, i.e. #9844

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.

Awesome work.

I am not much convinced about the final packaging, because if you want to run an existing Kafka Sink you have to build this module and add that Sink as dependency (this is the same work we are doing with Debezium source?).
I wonder if we can (as a follow up work) add the a ability to pass an additional file (the jar for the Sink implementation) as configuration and dynamically load it.
Probably the new package management facility (still under development in current master branch) will help?

@dlg99
Copy link
Contributor Author

dlg99 commented Mar 16, 2021

I am not much convinced about the final packaging, because if you want to run an existing Kafka Sink you have to build this module and add that Sink as dependency (this is the same work we are doing with Debezium source?).

Final packaging is tricky.
What I have is an option to get started quickly.
I cannot include all possible kafka connect modules - most of the people will need only small subset of them (that's even if this won't create dependency conflicts), others will need some custom connector not available publicly.
Passing a jar assumes the fat jar -> either shading (size of nar increases) or dependency conflicts.
Merging two nars is an option https://jslabonte.wordpress.com/2012/02/01/how-to-merge-nar-files/
Either way, another profile is not for production use (I can't imagine file sink being very useful) but to provide an example.

@sijie sijie added this to the 2.8.0 milestone Mar 17, 2021
final int partition = 0;
final Object key;
final Object value;
if (unwrapKeyValueIfAvailable && sourceRecord.getValue() instanceof KeyValue) {
Copy link
Member

Choose a reason for hiding this comment

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

You might also need to another branch to check if Record is a KVRecord.

Copy link
Contributor

Choose a reason for hiding this comment

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

@sijie KVRecord is not handled by Sinks, it is used only for Sources.
see #10113

}

@Override
public void offset(TopicPartition topicPartition, long l) {
Copy link
Member

Choose a reason for hiding this comment

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

same comment above.

sourceRecord.fail();
return;
}
pendingFlush.whenComplete((ignore, ex) -> {
Copy link
Member

Choose a reason for hiding this comment

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

I also don't think you need to register the callback here. Instead, I think you need to track the pulsar source records in the context in order to make sure the "offsets" are correctly recorded. once you call kafka sink to flush, you need to call sink context to flush. If you successfully flush, you can call sourceRecord to ack, otherwise you fail the source records.

@dlg99
Copy link
Contributor Author

dlg99 commented Mar 24, 2021

updated to reflect CR comments (except some that affect pulsar-io/context and can be done separately/later)
rebased on top of #9947

@dlg99 dlg99 requested a review from sijie March 24, 2021 20:54
sijie pushed a commit that referenced this pull request Mar 26, 2021
### Motivation

Looking at #9927 (comment) there is no way to reliably get partition number.
There is Optional<String> getPartitionId() which returns:
- "{topic}-{partition}" in some Record implementation
- String(partition) in other implementation (parsing back integer is a waste)
- empty Optional (ok) in others

### Modifications

Added `default Optional<Integer> getPartitionIndex()` to the `Record` interface.
Return partition number where appropriate.
@dlg99 dlg99 force-pushed the kafka-connect-adaptor-sink branch 2 times, most recently from b5ca187 to 0b44bbd Compare April 1, 2021 20:07
@dlg99 dlg99 force-pushed the kafka-connect-adaptor-sink branch 3 times, most recently from 4beb265 to 7f08a23 Compare April 19, 2021 17:13
@dlg99
Copy link
Contributor Author

dlg99 commented Apr 19, 2021

rebased on latest master

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.

I did my final pass of review.
Overall is good, I left some final comments

I hope we can merge this patch soon, this way we can continue to build up more features over this bridge to Kafka Connect.

@dlg99 dlg99 force-pushed the kafka-connect-adaptor-sink branch from 7f08a23 to 60aa808 Compare April 21, 2021 00:49
@dlg99 dlg99 requested a review from eolivelli April 21, 2021 15:26
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.

Looks great to me

@sijie can you please review again ?

I believe we can merge this patch and let @dlg99 follow up with other patches.

@eolivelli
Copy link
Contributor

@sijie can you please review ? this patch has been hanging for many days.

@merlimat @rdhabalia @codelipenghui can you help with merging this patch ?

@dlg99 dlg99 force-pushed the kafka-connect-adaptor-sink branch from 94d18f6 to f7232b4 Compare May 4, 2021 23:41
sijie pushed a commit that referenced this pull request May 8, 2021
…10446)

Fixes #10445

### Motivation

SinkContext should expose Subscription type to the Sink
More context: #9927 (comment)

Needed for #9927

### Modifications

Added `getSubscriptionType()` to the `SinkContext` interface and `ContextImpl`
@dlg99 dlg99 force-pushed the kafka-connect-adaptor-sink branch from f7232b4 to c080b7f Compare May 10, 2021 20:57
eolivelli pushed a commit to eolivelli/pulsar that referenced this pull request May 11, 2021
…pache#10446)

Fixes apache#10445

### Motivation

SinkContext should expose Subscription type to the Sink
More context: apache#9927 (comment)

Needed for apache#9927

### Modifications

Added `getSubscriptionType()` to the `SinkContext` interface and `ContextImpl`
@dlg99 dlg99 force-pushed the kafka-connect-adaptor-sink branch from c080b7f to c1e7858 Compare May 18, 2021 20:53
@dlg99 dlg99 requested a review from sijie May 18, 2021 21:12
@dlg99
Copy link
Contributor Author

dlg99 commented May 18, 2021

@sijie please take another look

@codelipenghui codelipenghui modified the milestones: 2.8.0, 2.9.0 May 26, 2021
}

private void flushIfNeeded(boolean force) {
if (force || currentBatchSize.get() >= maxBatchSize) {
Copy link
Member

Choose a reason for hiding this comment

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

@dlg99 This will cause submitting multiple flush tasks, no? Because currentBatchSize will always be larger than maxBatchSize before the flush action is taken.

Copy link
Member

Choose a reason for hiding this comment

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

This potentially will enqueue a lot of tasks in the flush queue which will burn CPU.

required = true,
defaultValue = "",
help = "Pulsar service URL to use for the offset store.")
private String pulsarServiceUrl;
Copy link
Member

Choose a reason for hiding this comment

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

This field will potentially be removed after PulsarClient is exposed to the context. So you might want to document it.

@lhotari lhotari merged commit a2604c8 into apache:master Jun 2, 2021
yangl pushed a commit to yangl/pulsar that referenced this pull request Jun 23, 2021
* Add ability to use Kafka's sinks as Pulsar sinks

* Get topic/partition from the record

* Making use of GenericObject

* Added SubscriptionType check and tests

* support for topic/partition pause/resume/seek

* added comment; only one flush in progress is allowed
@dlg99 dlg99 deleted the kafka-connect-adaptor-sink branch October 14, 2021 23:29
bharanic-dev pushed a commit to bharanic-dev/pulsar that referenced this pull request Mar 18, 2022
* Add ability to use Kafka's sinks as Pulsar sinks

* Get topic/partition from the record

* Making use of GenericObject

* Added SubscriptionType check and tests

* support for topic/partition pause/resume/seek

* added comment; only one flush in progress is allowed
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.

6 participants