-
Notifications
You must be signed in to change notification settings - Fork 3.5k
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
Support writing general records to Pulsar sink #9590
Changes from 3 commits
f8b2d1d
cd8e6fc
acd51c7
27613ee
0c90e99
6bb2137
1c4a03b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,7 +24,6 @@ | |
import lombok.Builder; | ||
import lombok.Data; | ||
import lombok.extern.slf4j.Slf4j; | ||
import lombok.val; | ||
import org.apache.commons.lang3.StringUtils; | ||
import org.apache.pulsar.client.api.BatcherBuilder; | ||
import org.apache.pulsar.client.api.CompressionType; | ||
|
@@ -39,12 +38,15 @@ | |
import org.apache.pulsar.client.api.PulsarClientException; | ||
import org.apache.pulsar.client.api.Schema; | ||
import org.apache.pulsar.client.api.TypedMessageBuilder; | ||
import org.apache.pulsar.client.api.schema.GenericRecord; | ||
import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; | ||
import org.apache.pulsar.client.impl.schema.KeyValueSchema; | ||
import org.apache.pulsar.common.functions.ConsumerConfig; | ||
import org.apache.pulsar.common.functions.CryptoConfig; | ||
import org.apache.pulsar.common.functions.FunctionConfig; | ||
import org.apache.pulsar.common.functions.ProducerConfig; | ||
import org.apache.pulsar.common.schema.KeyValueEncodingType; | ||
import org.apache.pulsar.common.schema.SchemaType; | ||
import org.apache.pulsar.functions.api.Record; | ||
import org.apache.pulsar.functions.instance.FunctionResultRouter; | ||
import org.apache.pulsar.functions.instance.SinkRecord; | ||
|
@@ -93,10 +95,10 @@ private interface PulsarSinkProcessor<T> { | |
void close() throws Exception; | ||
} | ||
|
||
private abstract class PulsarSinkProcessorBase implements PulsarSinkProcessor<T> { | ||
abstract class PulsarSinkProcessorBase implements PulsarSinkProcessor<T> { | ||
protected Map<String, Producer<T>> publishProducers = new ConcurrentHashMap<>(); | ||
protected Schema schema; | ||
protected Crypto crypto; | ||
protected Crypto crypto; | ||
|
||
protected PulsarSinkProcessorBase(Schema schema, Crypto crypto) { | ||
this.schema = schema; | ||
|
@@ -153,11 +155,16 @@ protected Producer<T> getProducer(String destinationTopic, Schema schema) { | |
protected Producer<T> getProducer(String producerId, String producerName, String topicName, Schema schema) { | ||
return publishProducers.computeIfAbsent(producerId, s -> { | ||
try { | ||
return createProducer( | ||
log.info("Initializing producer {} on topic {} with schema {}", | ||
producerName, topicName, schema); | ||
Producer<T> producer = createProducer( | ||
client, | ||
topicName, | ||
producerName, | ||
schema != null ? schema : this.schema); | ||
log.info("Initialized producer {} on topic {} with schema {}: {} -> {}", | ||
producerName, topicName, schema, producerId, producer); | ||
return producer; | ||
} catch (PulsarClientException e) { | ||
log.error("Failed to create Producer while doing user publish", e); | ||
throw new RuntimeException(e); | ||
|
@@ -209,13 +216,21 @@ public Function<Throwable, Void> getPublishErrorHandler(SinkRecord<T> record, bo | |
class PulsarSinkAtMostOnceProcessor extends PulsarSinkProcessorBase { | ||
public PulsarSinkAtMostOnceProcessor(Schema schema, Crypto crypto) { | ||
super(schema, crypto); | ||
// initialize default topic | ||
try { | ||
publishProducers.put(pulsarSinkConfig.getTopic(), | ||
if (!(schema instanceof AutoConsumeSchema)) { | ||
// initialize default topic | ||
try { | ||
publishProducers.put(pulsarSinkConfig.getTopic(), | ||
createProducer(client, pulsarSinkConfig.getTopic(), null, schema)); | ||
} catch (PulsarClientException e) { | ||
log.error("Failed to create Producer while doing user publish", e); | ||
throw new RuntimeException(e); } | ||
} catch (PulsarClientException e) { | ||
log.error("Failed to create Producer while doing user publish", e); | ||
throw new RuntimeException(e); | ||
} | ||
} else { | ||
if (log.isDebugEnabled()) { | ||
log.debug("The Pulsar producer is not initialized until the first record is" | ||
+ " published for `AUTO_CONSUME` schema."); | ||
} | ||
} | ||
} | ||
|
||
@Override | ||
|
@@ -400,7 +415,16 @@ Schema<T> initializeSchema() throws ClassNotFoundException { | |
ConsumerConfig consumerConfig = new ConsumerConfig(); | ||
consumerConfig.setSchemaProperties(pulsarSinkConfig.getSchemaProperties()); | ||
if (!StringUtils.isEmpty(pulsarSinkConfig.getSchemaType())) { | ||
consumerConfig.setSchemaType(pulsarSinkConfig.getSchemaType()); | ||
if (GenericRecord.class.isAssignableFrom(typeArg)) { | ||
consumerConfig.setSchemaType(SchemaType.AUTO_CONSUME.toString()); | ||
SchemaType configuredSchemaType = SchemaType.valueOf(pulsarSinkConfig.getSchemaType()); | ||
if (SchemaType.AUTO_CONSUME != configuredSchemaType) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Need to add There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No. The schema type is already overwritten in line 419. This is just to log an info message to indicate that the schema type has been overwritten to |
||
log.info("The configured schema type {} is not able to write GenericRecords." | ||
+ " So overwrite the schema type to be {}", configuredSchemaType, SchemaType.AUTO_CONSUME); | ||
} | ||
} else { | ||
consumerConfig.setSchemaType(pulsarSinkConfig.getSchemaType()); | ||
} | ||
return (Schema<T>) topicSchema.getSchema(pulsarSinkConfig.getTopic(), typeArg, | ||
consumerConfig, false); | ||
} else { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@sijie initially you pointed out that working here in PulsarSink is not the right way, but we should only work on TopicSchema
#9481 (comment)
In fact I believe that in my PR #9481 I took the right way, driven by your suggestions.
I believe that this change is not enough in order to support by needs.
BTW if the integration test I added to #9481 works with this patch then we can converge to a good solution.
My goal is to get that usecase work, in the best way for the project for the mid/long term
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@eolivelli Yes and no on my original comment.
My original comment is to make sure we returned the write schema information via TopicSchema. Because we are using
AUTO_CONSUME
in the PulsarSink to indicateGenericRecord
are published to the Pulsar topic.AUTO_CONSUME
can be used by both source and sink. In order to not impact sources, I didn't add the logic inTopicSchema
. Instead, I add it in PulsarSink to make it more explicit, which results in one line of similar change as your initial change. But it doesn't your original and current implementation is in the right direction.The main problem of your previous and current implementation on #9481 is you are trying to hijack the existing AVRO implementation to introduce the support of lazy schema initialization. The lazy schema initialization is already implemented as part of multi-schema write support. So you don't need to add such a hack.