|
44 | 44 | 'KafkaOffsetResetStrategy',
|
45 | 45 | 'KafkaRecordSerializationSchema',
|
46 | 46 | 'KafkaRecordSerializationSchemaBuilder',
|
47 |
| - 'KafkaTopicSelector' |
| 47 | + 'KafkaTopicSelector', |
| 48 | + 'KafkaRecordDeserializationSchema', |
| 49 | + 'SimpleStringValueKafkaRecordDeserializationSchema' |
48 | 50 | ]
|
49 | 51 |
|
50 | 52 |
|
@@ -353,6 +355,38 @@ def ignore_failures_after_transaction_timeout(self) -> 'FlinkKafkaProducer':
|
353 | 355 |
|
354 | 356 | # ---- KafkaSource ----
|
355 | 357 |
|
| 358 | +class KafkaRecordDeserializationSchema: |
| 359 | + """ |
| 360 | + Base class for KafkaRecordDeserializationSchema. The kafka record deserialization schema |
| 361 | + describes how to turn the byte messages delivered by Apache Kafka into data types (Java/ |
| 362 | + Scala objects) that are processed by Flink. |
| 363 | +
|
| 364 | + In addition, the KafkaRecordDeserializationSchema describes the produced type which lets |
| 365 | + Flink create internal serializers and structures to handle the type. |
| 366 | + """ |
| 367 | + def __init__(self, j_kafka_record_deserialization_schema=None): |
| 368 | + self.j_kafka_record_deserialization_schema = j_kafka_record_deserialization_schema |
| 369 | + |
| 370 | + |
| 371 | +class SimpleStringValueKafkaRecordDeserializationSchema(KafkaRecordDeserializationSchema): |
| 372 | + """ |
| 373 | + Very simple deserialization schema for strings values. By default, the deserializer uses |
| 374 | + 'UTF-8' for byte to string conversion. |
| 375 | + """ |
| 376 | + |
| 377 | + def __init__(self, charset: str = 'UTF-8'): |
| 378 | + gate_way = get_gateway() |
| 379 | + j_char_set = gate_way.jvm.java.nio.charset.Charset.forName(charset) |
| 380 | + j_simple_string_serialization_schema = gate_way.jvm \ |
| 381 | + .org.apache.flink.api.common.serialization.SimpleStringSchema(j_char_set) |
| 382 | + j_kafka_record_deserialization_schema = gate_way.jvm \ |
| 383 | + .org.apache.flink.connector.kafka.source.reader.deserializer \ |
| 384 | + .KafkaRecordDeserializationSchema.valueOnly(j_simple_string_serialization_schema) |
| 385 | + KafkaRecordDeserializationSchema.__init__( |
| 386 | + self, j_kafka_record_deserialization_schema=j_kafka_record_deserialization_schema) |
| 387 | + |
| 388 | + |
| 389 | +# ---- KafkaSource ---- |
356 | 390 |
|
357 | 391 | class KafkaSource(Source):
|
358 | 392 | """
|
@@ -611,6 +645,22 @@ def set_value_only_deserializer(self, deserialization_schema: DeserializationSch
|
611 | 645 | self._j_builder.setValueOnlyDeserializer(deserialization_schema._j_deserialization_schema)
|
612 | 646 | return self
|
613 | 647 |
|
| 648 | + def set_deserializer( |
| 649 | + self, |
| 650 | + kafka_record_deserialization_schema: KafkaRecordDeserializationSchema |
| 651 | + ) -> 'KafkaSourceBuilder': |
| 652 | + """ |
| 653 | + Sets the :class:`~pyflink.datastream.connectors.kafka.KafkaRecordDeserializationSchema` |
| 654 | + for deserializing Kafka ConsumerRecords. |
| 655 | +
|
| 656 | + :param kafka_record_deserialization_schema: the :class:`KafkaRecordDeserializationSchema` |
| 657 | + to use for deserialization. |
| 658 | + :return: this KafkaSourceBuilder. |
| 659 | + """ |
| 660 | + self._j_builder.setDeserializer( |
| 661 | + kafka_record_deserialization_schema.j_kafka_record_deserialization_schema) |
| 662 | + return self |
| 663 | + |
614 | 664 | def set_client_id_prefix(self, prefix: str) -> 'KafkaSourceBuilder':
|
615 | 665 | """
|
616 | 666 | Sets the client id prefix of this KafkaSource.
|
|
0 commit comments