def _check_if_row_data_type(ds) -> bool: j_type_info = ds._j_data_stream.getType() if not is_instance_of( j_type_info, 'org.apache.flink.table.runtime.typeutils.InternalTypeInfo' ): return False return is_instance_of( j_type_info.toLogicalType(), 'org.apache.flink.table.types.logical.RowType' )
def test_es_sink_dynamic(self): ds = self.env.from_collection([{ 'name': 'ada', 'id': '1' }, { 'name': 'luna', 'id': '2' }], type_info=Types.MAP( Types.STRING(), Types.STRING())) es_dynamic_index_sink = Elasticsearch7SinkBuilder() \ .set_emitter(ElasticsearchEmitter.dynamic_index('name', 'id')) \ .set_hosts(['localhost:9200']) \ .build() j_emitter = get_field_value(es_dynamic_index_sink.get_java_function(), 'emitter') self.assertTrue( is_instance_of( j_emitter, 'org.apache.flink.connector.elasticsearch.sink.SimpleElasticsearchEmitter' )) ds.sink_to(es_dynamic_index_sink).name('es dynamic index sink')
def _check_specified_offsets_initializer( self, source: KafkaSource, offsets: Dict[KafkaTopicPartition, int], reset_strategy: KafkaOffsetResetStrategy, is_start: bool = True): if is_start: field_name = 'startingOffsetsInitializer' else: field_name = 'stoppingOffsetsInitializer' offsets_initializer = get_field_value(source.get_java_function(), field_name) self.assertEqual( offsets_initializer.getClass().getCanonicalName(), 'org.apache.flink.connector.kafka.source.enumerator.initializer' '.SpecifiedOffsetsInitializer') initial_offsets = get_field_value(offsets_initializer, 'initialOffsets') self.assertTrue( is_instance_of(initial_offsets, get_gateway().jvm.java.util.Map)) self.assertEqual(initial_offsets.size(), len(offsets)) for j_topic_partition in initial_offsets: topic_partition = KafkaTopicPartition( j_topic_partition.topic(), j_topic_partition.partition()) self.assertIsNotNone(offsets.get(topic_partition)) self.assertEqual(initial_offsets[j_topic_partition], offsets[topic_partition]) offset_reset_strategy = get_field_value(offsets_initializer, 'offsetResetStrategy') self.assertTrue( offset_reset_strategy.equals( reset_strategy._to_j_offset_reset_strategy()))
def test_no_watermarks(self): jvm = get_gateway().jvm j_watermark_strategy = WatermarkStrategy.no_watermarks( )._j_watermark_strategy self.assertTrue( is_instance_of( j_watermark_strategy.createWatermarkGenerator(None), jvm.org. apache.flink.api.common.eventtime.NoWatermarksGenerator))
def test_for_monotonous_timestamps(self): jvm = get_gateway().jvm j_watermark_strategy = WatermarkStrategy.for_monotonous_timestamps( )._j_watermark_strategy self.assertTrue( is_instance_of( j_watermark_strategy.createWatermarkGenerator(None), jvm.org.apache.flink.api.common.eventtime. AscendingTimestampsWatermarks))
def test_with_idleness(self): jvm = get_gateway().jvm j_watermark_strategy = WatermarkStrategy.no_watermarks().with_idleness( Duration.of_seconds(5))._j_watermark_strategy self.assertTrue( is_instance_of( j_watermark_strategy, jvm.org.apache.flink.api.common. eventtime.WatermarkStrategyWithIdleness)) self.assertEqual( get_field_value(j_watermark_strategy, "idlenessTimeout").toMillis(), 5000)
def test_for_bounded_out_of_orderness(self): jvm = get_gateway().jvm j_watermark_strategy = WatermarkStrategy.for_bounded_out_of_orderness( Duration.of_seconds(3))._j_watermark_strategy j_watermark_generator = j_watermark_strategy.createWatermarkGenerator( None) self.assertTrue( is_instance_of( j_watermark_generator, jvm.org.apache.flink.api.common. eventtime.BoundedOutOfOrdernessWatermarks)) self.assertEqual( get_field_value(j_watermark_generator, "outOfOrdernessMillis"), 3000)
def test_with_watermark_alignment(self): jvm = get_gateway().jvm j_watermark_strategy = WatermarkStrategy.no_watermarks( ).with_watermark_alignment( "alignment-group-1", Duration.of_seconds(20), Duration.of_seconds(10))._j_watermark_strategy self.assertTrue( is_instance_of( j_watermark_strategy, jvm.org.apache.flink.api.common. eventtime.WatermarksWithWatermarkAlignment)) alignment_parameters = j_watermark_strategy.getAlignmentParameters() self.assertEqual(alignment_parameters.getWatermarkGroup(), "alignment-group-1") self.assertEqual(alignment_parameters.getMaxAllowedWatermarkDrift(), 20000) self.assertEqual(alignment_parameters.getUpdateInterval(), 10000)
def test_set_topic_pattern(self): source = KafkaSource.builder() \ .set_bootstrap_servers('localhost:9092') \ .set_topic_pattern('test_topic*') \ .set_value_only_deserializer(SimpleStringSchema()) \ .build() kafka_subscriber = get_field_value(source.get_java_function(), 'subscriber') self.assertEqual( kafka_subscriber.getClass().getCanonicalName(), 'org.apache.flink.connector.kafka.source.enumerator.subscriber.TopicPatternSubscriber' ) topic_pattern = get_field_value(kafka_subscriber, 'topicPattern') self.assertTrue( is_instance_of(topic_pattern, get_gateway().jvm.java.util.regex.Pattern)) self.assertEqual(topic_pattern.toString(), 'test_topic*')
def test_set_topics(self): source = KafkaSource.builder() \ .set_bootstrap_servers('localhost:9092') \ .set_topics('test_topic1', 'test_topic2') \ .set_value_only_deserializer(SimpleStringSchema()) \ .build() kafka_subscriber = get_field_value(source.get_java_function(), 'subscriber') self.assertEqual( kafka_subscriber.getClass().getCanonicalName(), 'org.apache.flink.connector.kafka.source.enumerator.subscriber.TopicListSubscriber' ) topics = get_field_value(kafka_subscriber, 'topics') self.assertTrue( is_instance_of(topics, get_gateway().jvm.java.util.List)) self.assertEqual(topics.size(), 2) self.assertEqual(topics[0], 'test_topic1') self.assertEqual(topics[1], 'test_topic2')
def test_set_partitions(self): topic_partition_1 = KafkaTopicPartition('test_topic', 1) topic_partition_2 = KafkaTopicPartition('test_topic', 2) source = KafkaSource.builder() \ .set_bootstrap_servers('localhost:9092') \ .set_partitions({topic_partition_1, topic_partition_2}) \ .set_value_only_deserializer(SimpleStringSchema()) \ .build() kafka_subscriber = get_field_value(source.get_java_function(), 'subscriber') self.assertEqual( kafka_subscriber.getClass().getCanonicalName(), 'org.apache.flink.connector.kafka.source.enumerator.subscriber.PartitionSetSubscriber' ) partitions = get_field_value(kafka_subscriber, 'subscribedPartitions') self.assertTrue( is_instance_of(partitions, get_gateway().jvm.java.util.Set)) self.assertTrue( topic_partition_1._to_j_topic_partition() in partitions) self.assertTrue( topic_partition_2._to_j_topic_partition() in partitions)
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)
def test_es_sink(self): ds = self.env.from_collection([{ 'name': 'ada', 'id': '1' }, { 'name': 'luna', 'id': '2' }], type_info=Types.MAP( Types.STRING(), Types.STRING())) es_sink = Elasticsearch7SinkBuilder() \ .set_emitter(ElasticsearchEmitter.static_index('foo', 'id')) \ .set_hosts(['localhost:9200']) \ .set_delivery_guarantee(DeliveryGuarantee.AT_LEAST_ONCE) \ .set_bulk_flush_max_actions(1) \ .set_bulk_flush_max_size_mb(2) \ .set_bulk_flush_interval(1000) \ .set_bulk_flush_backoff_strategy(FlushBackoffType.CONSTANT, 3, 3000) \ .set_connection_username('foo') \ .set_connection_password('bar') \ .set_connection_path_prefix('foo-bar') \ .set_connection_request_timeout(30000) \ .set_connection_timeout(31000) \ .set_socket_timeout(32000) \ .build() j_emitter = get_field_value(es_sink.get_java_function(), 'emitter') self.assertTrue( is_instance_of( j_emitter, 'org.apache.flink.connector.elasticsearch.sink.SimpleElasticsearchEmitter' )) self.assertEqual( get_field_value(es_sink.get_java_function(), 'hosts')[0].toString(), 'http://localhost:9200') self.assertEqual( get_field_value(es_sink.get_java_function(), 'deliveryGuarantee').toString(), 'at-least-once') j_build_bulk_processor_config = get_field_value( es_sink.get_java_function(), 'buildBulkProcessorConfig') self.assertEqual( j_build_bulk_processor_config.getBulkFlushMaxActions(), 1) self.assertEqual(j_build_bulk_processor_config.getBulkFlushMaxMb(), 2) self.assertEqual(j_build_bulk_processor_config.getBulkFlushInterval(), 1000) self.assertEqual( j_build_bulk_processor_config.getFlushBackoffType().toString(), 'CONSTANT') self.assertEqual( j_build_bulk_processor_config.getBulkFlushBackoffRetries(), 3) self.assertEqual( j_build_bulk_processor_config.getBulkFlushBackOffDelay(), 3000) j_network_client_config = get_field_value(es_sink.get_java_function(), 'networkClientConfig') self.assertEqual(j_network_client_config.getUsername(), 'foo') self.assertEqual(j_network_client_config.getPassword(), 'bar') self.assertEqual(j_network_client_config.getConnectionRequestTimeout(), 30000) self.assertEqual(j_network_client_config.getConnectionTimeout(), 31000) self.assertEqual(j_network_client_config.getSocketTimeout(), 32000) self.assertEqual(j_network_client_config.getConnectionPathPrefix(), 'foo-bar') ds.sink_to(es_sink).name('es sink')