def set_config(self, key: str, value) -> 'PulsarSinkBuilder': """ Set an arbitrary property for the PulsarSink and Pulsar Producer. The valid keys can be found in PulsarSinkOptions and PulsarOptions. Make sure the option could be set only once or with same value. """ j_config_option = ConfigOptions.key(key).string_type().no_default_value()._j_config_option self._j_pulsar_sink_builder.setConfig(j_config_option, value) return self
def test_source_deprecated_method(self): test_option = ConfigOptions.key('pulsar.source.enableAutoAcknowledgeMessage') \ .boolean_type().no_default_value() pulsar_source = PulsarSource.builder() \ .set_service_url('pulsar://localhost:6650') \ .set_admin_url('http://localhost:8080') \ .set_topics('ada') \ .set_deserialization_schema( PulsarDeserializationSchema.flink_type_info(Types.STRING(), None)) \ .set_subscription_name('ff') \ .set_config(test_option, True) \ .set_config_with_dict({'pulsar.source.autoCommitCursorInterval': '1000'}) \ .build() configuration = get_field_value(pulsar_source.get_java_function(), "sourceConfiguration") self.assertEqual( configuration.getBoolean(test_option._j_config_option), True) self.assertEqual( configuration.getLong( ConfigOptions.key('pulsar.source.autoCommitCursorInterval'). long_type().no_default_value()._j_config_option), 1000)
def test_pulsar_source(self): TEST_OPTION_NAME = 'pulsar.source.enableAutoAcknowledgeMessage' pulsar_source = PulsarSource.builder() \ .set_service_url('pulsar://localhost:6650') \ .set_admin_url('http://localhost:8080') \ .set_topics('ada') \ .set_start_cursor(StartCursor.earliest()) \ .set_unbounded_stop_cursor(StopCursor.never()) \ .set_bounded_stop_cursor(StopCursor.at_publish_time(22)) \ .set_subscription_name('ff') \ .set_subscription_type(SubscriptionType.Exclusive) \ .set_deserialization_schema( PulsarDeserializationSchema.flink_type_info(Types.STRING())) \ .set_deserialization_schema( PulsarDeserializationSchema.flink_schema(SimpleStringSchema())) \ .set_config(TEST_OPTION_NAME, True) \ .set_properties({'pulsar.source.autoCommitCursorInterval': '1000'}) \ .build() ds = self.env.from_source(source=pulsar_source, watermark_strategy=WatermarkStrategy.for_monotonous_timestamps(), source_name="pulsar source") ds.print() plan = eval(self.env.get_execution_plan()) self.assertEqual('Source: pulsar source', plan['nodes'][0]['type']) configuration = get_field_value(pulsar_source.get_java_function(), "sourceConfiguration") self.assertEqual( configuration.getString( ConfigOptions.key('pulsar.client.serviceUrl') .string_type() .no_default_value()._j_config_option), 'pulsar://localhost:6650') self.assertEqual( configuration.getString( ConfigOptions.key('pulsar.admin.adminUrl') .string_type() .no_default_value()._j_config_option), 'http://localhost:8080') self.assertEqual( configuration.getString( ConfigOptions.key('pulsar.consumer.subscriptionName') .string_type() .no_default_value()._j_config_option), 'ff') self.assertEqual( configuration.getString( ConfigOptions.key('pulsar.consumer.subscriptionType') .string_type() .no_default_value()._j_config_option), SubscriptionType.Exclusive.name) test_option = ConfigOptions.key(TEST_OPTION_NAME).boolean_type().no_default_value() self.assertEqual( configuration.getBoolean( test_option._j_config_option), True) self.assertEqual( configuration.getLong( ConfigOptions.key('pulsar.source.autoCommitCursorInterval') .long_type() .no_default_value()._j_config_option), 1000)
def set_config(self, key: Union[str, ConfigOption], value) -> 'PulsarSourceBuilder': """ Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be found in PulsarSourceOptions and PulsarOptions. Make sure the option could be set only once or with same value. """ if isinstance(key, ConfigOption): warnings.warn("set_config(key: ConfigOption, value) is deprecated. " "Use set_config(key: str, value) instead.", DeprecationWarning, stacklevel=2) j_config_option = key._j_config_option else: j_config_option = \ ConfigOptions.key(key).string_type().no_default_value()._j_config_option self._j_pulsar_source_builder.setConfig(j_config_option, value) return self
def test_pulsar_sink(self): ds = self.env.from_collection([('ab', 1), ('bdc', 2), ('cfgs', 3), ('deeefg', 4)], type_info=Types.ROW( [Types.STRING(), Types.INT()])) TEST_OPTION_NAME = 'pulsar.producer.chunkingEnabled' pulsar_sink = PulsarSink.builder() \ .set_service_url('pulsar://localhost:6650') \ .set_admin_url('http://localhost:8080') \ .set_producer_name('fo') \ .set_topics('ada') \ .set_serialization_schema( PulsarSerializationSchema.flink_schema(SimpleStringSchema())) \ .set_delivery_guarantee(DeliveryGuarantee.AT_LEAST_ONCE) \ .set_topic_routing_mode(TopicRoutingMode.ROUND_ROBIN) \ .delay_sending_message(MessageDelayer.fixed(Duration.of_seconds(12))) \ .set_config(TEST_OPTION_NAME, True) \ .set_properties({'pulsar.producer.batchingMaxMessages': '100'}) \ .build() ds.sink_to(pulsar_sink).name('pulsar sink') plan = eval(self.env.get_execution_plan()) self.assertEqual('pulsar sink: Writer', plan['nodes'][1]['type']) configuration = get_field_value(pulsar_sink.get_java_function(), "sinkConfiguration") self.assertEqual( configuration.getString( ConfigOptions.key('pulsar.client.serviceUrl').string_type(). no_default_value()._j_config_option), 'pulsar://localhost:6650') self.assertEqual( configuration.getString( ConfigOptions.key('pulsar.admin.adminUrl').string_type(). no_default_value()._j_config_option), 'http://localhost:8080') self.assertEqual( configuration.getString( ConfigOptions.key('pulsar.producer.producerName').string_type( ).no_default_value()._j_config_option), 'fo - %s') j_pulsar_serialization_schema = get_field_value( pulsar_sink.get_java_function(), 'serializationSchema') j_serialization_schema = get_field_value(j_pulsar_serialization_schema, 'serializationSchema') self.assertTrue( is_instance_of( j_serialization_schema, 'org.apache.flink.api.common.serialization.SimpleStringSchema') ) self.assertEqual( configuration.getString( ConfigOptions.key('pulsar.sink.deliveryGuarantee').string_type( ).no_default_value()._j_config_option), 'at-least-once') j_topic_router = get_field_value(pulsar_sink.get_java_function(), "topicRouter") self.assertTrue( is_instance_of( j_topic_router, 'org.apache.flink.connector.pulsar.sink.writer.router.RoundRobinTopicRouter' )) j_message_delayer = get_field_value(pulsar_sink.get_java_function(), 'messageDelayer') delay_duration = get_field_value(j_message_delayer, 'delayDuration') self.assertEqual(delay_duration, 12000) test_option = ConfigOptions.key( TEST_OPTION_NAME).boolean_type().no_default_value() self.assertEqual( configuration.getBoolean(test_option._j_config_option), True) self.assertEqual( configuration.getLong( ConfigOptions.key('pulsar.producer.batchingMaxMessages'). long_type().no_default_value()._j_config_option), 100)