def python_data_stream_example(): env = StreamExecutionEnvironment.get_execution_environment() # Set the parallelism to be one to make sure that all data including fired timer and normal data # are processed by the same worker and the collected result would be in order which is good for # assertion. env.set_parallelism(1) env.set_stream_time_characteristic(TimeCharacteristic.EventTime) type_info = Types.ROW_NAMED(['createTime', 'orderId', 'payAmount', 'payPlatform', 'provinceId'], [Types.LONG(), Types.LONG(), Types.DOUBLE(), Types.INT(), Types.INT()]) json_row_schema = JsonRowDeserializationSchema.builder().type_info(type_info).build() kafka_props = {'bootstrap.servers': 'localhost:9092', 'group.id': 'pyflink-e2e-source'} kafka_consumer = FlinkKafkaConsumer("timer-stream-source", json_row_schema, kafka_props) kafka_producer = FlinkKafkaProducer("timer-stream-sink", SimpleStringSchema(), kafka_props) watermark_strategy = WatermarkStrategy.for_bounded_out_of_orderness(Duration.of_seconds(5))\ .with_timestamp_assigner(KafkaRowTimestampAssigner()) kafka_consumer.set_start_from_earliest() ds = env.add_source(kafka_consumer).assign_timestamps_and_watermarks(watermark_strategy) ds.key_by(MyKeySelector(), key_type_info=Types.LONG()) \ .process(MyProcessFunction(), output_type=Types.STRING()) \ .add_sink(kafka_producer) env.execute_async("test data stream timer")
def test_side_output_late_data(self): self.env.set_parallelism(1) config = Configuration(j_configuration=get_j_env_configuration( self.env._j_stream_execution_environment)) config.set_integer('python.fn-execution.bundle.size', 1) jvm = get_gateway().jvm watermark_strategy = WatermarkStrategy( jvm.org.apache.flink.api.common.eventtime.WatermarkStrategy. forGenerator(jvm.org.apache.flink.streaming.api.functions.python. eventtime.PerElementWatermarkGenerator.getSupplier()) ).with_timestamp_assigner(SecondColumnTimestampAssigner()) tag = OutputTag('late-data', type_info=Types.ROW([Types.STRING(), Types.INT()])) ds1 = self.env.from_collection( [('a', 0), ('a', 8), ('a', 4), ('a', 6)], type_info=Types.ROW([Types.STRING(), Types.INT()])) ds2 = ds1.assign_timestamps_and_watermarks(watermark_strategy) \ .key_by(lambda e: e[0]) \ .window(TumblingEventTimeWindows.of(Time.milliseconds(5))) \ .allowed_lateness(0) \ .side_output_late_data(tag) \ .process(CountWindowProcessFunction(), Types.TUPLE([Types.STRING(), Types.LONG(), Types.LONG(), Types.INT()])) main_sink = DataStreamTestSinkFunction() ds2.add_sink(main_sink) side_sink = DataStreamTestSinkFunction() ds2.get_side_output(tag).add_sink(side_sink) self.env.execute('test_side_output_late_data') main_expected = ['(a,0,5,1)', '(a,5,10,2)'] self.assert_equals_sorted(main_expected, main_sink.get_results()) side_expected = ['+I[a, 4]'] self.assert_equals_sorted(side_expected, side_sink.get_results())
def keyed_stream_demo(): env = StreamExecutionEnvironment.get_execution_environment() env.set_parallelism(2) env.set_runtime_mode(RuntimeExecutionMode.BATCH) seq_num_source = NumberSequenceSource(1, 1000) output_path = '/opt/examples/datastream/output/keyed_stream' file_sink = FileSink \ .for_row_format(output_path, Encoder.simple_string_encoder()) \ .with_output_file_config(OutputFileConfig.builder().with_part_prefix('pre').with_part_suffix('suf').build()) \ .build() ds = env.from_source( source=seq_num_source, watermark_strategy=WatermarkStrategy.for_monotonous_timestamps(), source_name='seq_num_source', type_info=Types.LONG()) ds.map(lambda a: Row(a % 4, 1), output_type=Types.ROW([Types.LONG(), Types.LONG()])) \ .key_by(lambda a: a[0]) \ .map(lambda a: (a[0], a[1] + 1)) \ .key_by(lambda a: a[0]) \ .reduce(lambda a, b: (a[0], a[1] + b[1])) \ .key_by(lambda a: a[0]) \ .process(MyKeyedProcessFunction(), Types.LONG()) \ .filter(lambda i: i <= 10) \ .sink_to(file_sink) env.execute('10-keyed_stream')
def test_add_python_file(self): import uuid env = self.env python_file_dir = os.path.join(self.tempdir, "python_file_dir_" + str(uuid.uuid4())) os.mkdir(python_file_dir) python_file_path = os.path.join(python_file_dir, "test_dep1.py") with open(python_file_path, 'w') as f: f.write("def add_two(a):\n return a + 2") def plus_two_map(value): from test_dep1 import add_two return add_two(value) get_j_env_configuration(env._j_stream_execution_environment).\ setString("taskmanager.numberOfTaskSlots", "10") env.add_python_file(python_file_path) ds = env.from_collection([1, 2, 3, 4, 5]) ds = ds.map(plus_two_map, Types.LONG()) \ .slot_sharing_group("data_stream") \ .map(lambda i: i, Types.LONG()) \ .slot_sharing_group("table") python_file_path = os.path.join(python_file_dir, "test_dep2.py") with open(python_file_path, 'w') as f: f.write("def add_three(a):\n return a + 3") def plus_three(value): from test_dep2 import add_three return add_three(value) t_env = StreamTableEnvironment.create( stream_execution_environment=env, environment_settings=EnvironmentSettings.in_streaming_mode()) env.add_python_file(python_file_path) from pyflink.table.udf import udf from pyflink.table.expressions import col add_three = udf(plus_three, result_type=DataTypes.BIGINT()) tab = t_env.from_data_stream(ds, col('a')) \ .select(add_three(col('a'))) t_env.to_append_stream(tab, Types.ROW([Types.LONG()])) \ .map(lambda i: i[0]) \ .add_sink(self.test_sink) env.execute("test add_python_file") result = self.test_sink.get_results(True) expected = ['6', '7', '8', '9', '10'] result.sort() expected.sort() self.assertEqual(expected, result)
def test_from_collection_with_data_types(self): # verify from_collection for the collection with single object. ds = self.env.from_collection(['Hi', 'Hello'], type_info=Types.STRING()) ds.add_sink(self.test_sink) self.env.execute("test from collection with single object") results = self.test_sink.get_results(False) expected = ['Hello', 'Hi'] results.sort() expected.sort() self.assertEqual(expected, results) # verify from_collection for the collection with multiple objects like tuple. ds = self.env.from_collection([(1, None, 1, True, 32767, -2147483648, 1.23, 1.98932, bytearray(b'flink'), 'pyflink', datetime.date(2014, 9, 13), datetime.time(hour=12, minute=0, second=0, microsecond=123000), datetime.datetime(2018, 3, 11, 3, 0, 0, 123000), [1, 2, 3], decimal.Decimal('1000000000000000000.05'), decimal.Decimal('1000000000000000000.0599999999999' '9999899999999999')), (2, None, 2, True, 43878, 9147483648, 9.87, 2.98936, bytearray(b'flink'), 'pyflink', datetime.date(2015, 10, 14), datetime.time(hour=11, minute=2, second=2, microsecond=234500), datetime.datetime(2020, 4, 15, 8, 2, 6, 235000), [2, 4, 6], decimal.Decimal('2000000000000000000.74'), decimal.Decimal('2000000000000000000.061111111111111' '11111111111111'))], type_info=Types.ROW( [Types.LONG(), Types.LONG(), Types.SHORT(), Types.BOOLEAN(), Types.SHORT(), Types.INT(), Types.FLOAT(), Types.DOUBLE(), Types.PICKLED_BYTE_ARRAY(), Types.STRING(), Types.SQL_DATE(), Types.SQL_TIME(), Types.SQL_TIMESTAMP(), Types.BASIC_ARRAY(Types.LONG()), Types.BIG_DEC(), Types.BIG_DEC()])) ds.add_sink(self.test_sink) self.env.execute("test from collection with tuple object") results = self.test_sink.get_results(False) # if user specifies data types of input data, the collected result should be in row format. expected = [ '+I[1, null, 1, true, 32767, -2147483648, 1.23, 1.98932, [102, 108, 105, 110, 107], ' 'pyflink, 2014-09-13, 12:00:00, 2018-03-11 03:00:00.123, [1, 2, 3], ' '1000000000000000000.05, 1000000000000000000.05999999999999999899999999999]', '+I[2, null, 2, true, -21658, 557549056, 9.87, 2.98936, [102, 108, 105, 110, 107], ' 'pyflink, 2015-10-14, 11:02:02, 2020-04-15 08:02:06.235, [2, 4, 6], ' '2000000000000000000.74, 2000000000000000000.06111111111111111111111111111]'] results.sort() expected.sort() self.assertEqual(expected, results)
def _create_orc_basic_row_and_data() -> Tuple[RowType, RowTypeInfo, List[Row]]: row_type = DataTypes.ROW([ DataTypes.FIELD('char', DataTypes.CHAR(10)), DataTypes.FIELD('varchar', DataTypes.VARCHAR(10)), DataTypes.FIELD('bytes', DataTypes.BYTES()), DataTypes.FIELD('boolean', DataTypes.BOOLEAN()), DataTypes.FIELD('decimal', DataTypes.DECIMAL(2, 0)), DataTypes.FIELD('int', DataTypes.INT()), DataTypes.FIELD('bigint', DataTypes.BIGINT()), DataTypes.FIELD('double', DataTypes.DOUBLE()), DataTypes.FIELD('date', DataTypes.DATE().bridged_to('java.sql.Date')), DataTypes.FIELD('timestamp', DataTypes.TIMESTAMP(3).bridged_to('java.sql.Timestamp')), ]) row_type_info = Types.ROW_NAMED( ['char', 'varchar', 'bytes', 'boolean', 'decimal', 'int', 'bigint', 'double', 'date', 'timestamp'], [Types.STRING(), Types.STRING(), Types.PRIMITIVE_ARRAY(Types.BYTE()), Types.BOOLEAN(), Types.BIG_DEC(), Types.INT(), Types.LONG(), Types.DOUBLE(), Types.SQL_DATE(), Types.SQL_TIMESTAMP()] ) data = [Row( char='char', varchar='varchar', bytes=b'varbinary', boolean=True, decimal=Decimal(1.5), int=2147483647, bigint=-9223372036854775808, double=2e-308, date=date(1970, 1, 1), timestamp=datetime(1970, 1, 2, 3, 4, 5, 600000), )] return row_type, row_type_info, data
def event_timer_timer_demo(): env = StreamExecutionEnvironment.get_execution_environment() ds = env.from_collection( collection=[ (1000, 'Alice', 110.1), (4000, 'Bob', 30.2), (3000, 'Alice', 20.0), (2000, 'Bob', 53.1), (5000, 'Alice', 13.1), (3000, 'Bob', 3.1), (7000, 'Bob', 16.1), (10000, 'Alice', 20.1) ], type_info=Types.TUPLE([Types.LONG(), Types.STRING(), Types.FLOAT()])) ds = ds.assign_timestamps_and_watermarks( WatermarkStrategy.for_bounded_out_of_orderness(Duration.of_seconds(2)) .with_timestamp_assigner(MyTimestampAssigner())) # apply the process function onto a keyed stream ds.key_by(lambda value: value[1]) \ .process(Sum()) \ .print() # submit for execution env.execute()
def max_travellers_per_destination(): env = StreamExecutionEnvironment.get_execution_environment() t_env = StreamTableEnvironment.create(stream_execution_environment=env) t_env.execute_sql( create_table_ddl( "WATERMARK FOR dropOffTime AS dropOffTime - INTERVAL '30' SECONDS") ) taxi_ride = t_env.from_path('TaxiRide') no_of_travelers_per_dest = taxi_ride \ .select(taxi_ride.passengerCount, taxi_ride.dropOffTime, taxi_ride.destLocationZone) \ .window(Tumble().over('1.hour').on(taxi_ride.dropOffTime).alias('w')) \ .group_by(taxi_ride.destLocationZone, col('w')) \ .select(taxi_ride.destLocationZone, \ col('w').start.alias('start'), \ col('w').end.alias('end'), \ taxi_ride.passengerCount.count.alias('cnt')) t_env.to_append_stream( no_of_travelers_per_dest, Types.ROW_NAMED(['destLocationZone', 'start', 'end', 'cnt'], [ Types.STRING(), Types.SQL_TIMESTAMP(), Types.SQL_TIMESTAMP(), Types.LONG() ])).print() env.execute('Max-Travellers-Per-Destination')
def popular_taxi_vendor(): env = StreamExecutionEnvironment.get_execution_environment() env.set_parallelism(1) t_env = StreamTableEnvironment.create(stream_execution_environment=env) t_env.execute_sql( create_table_ddl( "WATERMARK FOR pickupTime AS pickupTime - INTERVAL '30' SECONDS")) taxi_ride = t_env.from_path('TaxiRide') popular_rides = taxi_ride.select(taxi_ride.vendorId, taxi_ride.pickupTime) \ .window(Slide.over('15.minutes').every('5.minutes').on(taxi_ride.pickupTime).alias('w')) \ .group_by(taxi_ride.vendorId, col('w')) \ .select(taxi_ride.vendorId, \ col('w').start.alias('start'), \ col('w').end.alias('end'), \ taxi_ride.vendorId.count.alias('cnt')) t_env.to_append_stream( popular_rides, Types.ROW_NAMED(['vendorId', 'start', 'end', 'cnt'], [ Types.INT(), Types.SQL_TIMESTAMP(), Types.SQL_TIMESTAMP(), Types.LONG() ])).print() env.execute('Popular-Taxi-Vendor')
def test_session_window_late_merge(self): data_stream = self.env.from_collection([ ('hi', 0), ('hi', 8), ('hi', 4)], type_info=Types.TUPLE([Types.STRING(), Types.INT()])) # type: DataStream watermark_strategy = WatermarkStrategy.for_monotonous_timestamps() \ .with_timestamp_assigner(SecondColumnTimestampAssigner()) data_stream.assign_timestamps_and_watermarks(watermark_strategy) \ .key_by(lambda x: x[0], key_type=Types.STRING()) \ .window(EventTimeSessionWindows.with_gap(Time.milliseconds(5))) \ .process(CountWindowProcessFunction(), Types.TUPLE([Types.STRING(), Types.LONG(), Types.LONG(), Types.INT()])) \ .add_sink(self.test_sink) self.env.execute('test_session_window_late_merge') results = self.test_sink.get_results() expected = ['(hi,0,13,3)'] self.assert_equals_sorted(expected, results)
def python_data_stream_example(): env = StreamExecutionEnvironment.get_execution_environment() # Set the parallelism to be one to make sure that all data including fired timer and normal data # are processed by the same worker and the collected result would be in order which is good for # assertion. env.set_parallelism(1) env.set_stream_time_characteristic(TimeCharacteristic.EventTime) t_env = StreamTableEnvironment.create(stream_execution_environment=env) create_kafka_source_ddl = """ CREATE TABLE payment_msg( createTime VARCHAR, rt as TO_TIMESTAMP(createTime), orderId BIGINT, payAmount DOUBLE, payPlatform INT, provinceId INT, WATERMARK FOR rt as rt - INTERVAL '2' SECOND ) WITH ( 'connector.type' = 'kafka', 'connector.version' = 'universal', 'connector.topic' = 'timer-stream-source', 'connector.properties.bootstrap.servers' = 'localhost:9092', 'connector.properties.group.id' = 'test_3', 'connector.startup-mode' = 'earliest-offset', 'format.type' = 'json' ) """ t_env.execute_sql(create_kafka_source_ddl) t = t_env.from_path("payment_msg").select("createTime, orderId, payAmount, payPlatform," " provinceId") source_type_info = Types.ROW([ Types.STRING(), Types.LONG(), Types.DOUBLE(), Types.INT(), Types.INT()]) ds = t_env.to_append_stream(table=t, type_info=source_type_info) producer_props = {'bootstrap.servers': 'localhost:9092', 'group.id': 'pyflink-e2e-source'} kafka_producer = FlinkKafkaProducer("timer-stream-sink", SimpleStringSchema(), producer_props) ds.key_by(MyKeySelector(), key_type_info=Types.LONG()) \ .process(MyProcessFunction(), output_type=Types.STRING()) \ .add_sink(kafka_producer) env.execute_async("test data stream timer")
def test_event_time_tumbling_window(self): data_stream = self.env.from_collection([ ('hi', 1), ('hi', 2), ('hi', 3), ('hi', 4), ('hi', 5), ('hi', 8), ('hi', 9), ('hi', 15)], type_info=Types.TUPLE([Types.STRING(), Types.INT()])) # type: DataStream watermark_strategy = WatermarkStrategy.for_monotonous_timestamps() \ .with_timestamp_assigner(SecondColumnTimestampAssigner()) data_stream.assign_timestamps_and_watermarks(watermark_strategy) \ .key_by(lambda x: x[0], key_type=Types.STRING()) \ .window(TumblingEventTimeWindows.of(Time.milliseconds(5))) \ .process(CountWindowProcessFunction(), Types.TUPLE([Types.STRING(), Types.LONG(), Types.LONG(), Types.INT()])) \ .add_sink(self.test_sink) self.env.execute('test_event_time_tumbling_window') results = self.test_sink.get_results() expected = ['(hi,0,5,4)', '(hi,5,10,3)', '(hi,15,20,1)'] self.assert_equals_sorted(expected, results)
def test_event_time_dynamic_gap_session_window(self): self.env.set_parallelism(1) data_stream = self.env.from_collection([ ('hi', 1), ('hi', 2), ('hi', 3), ('hi', 4), ('hi', 9), ('hi', 9), ('hi', 15)], type_info=Types.TUPLE([Types.STRING(), Types.INT()])) # type: DataStream watermark_strategy = WatermarkStrategy.for_monotonous_timestamps() \ .with_timestamp_assigner(SecondColumnTimestampAssigner()) data_stream.assign_timestamps_and_watermarks(watermark_strategy) \ .key_by(lambda x: x[0], key_type=Types.STRING()) \ .window(EventTimeSessionWindows.with_dynamic_gap(MySessionWindowTimeGapExtractor())) \ .process(CountWindowProcessFunction(), Types.TUPLE([Types.STRING(), Types.LONG(), Types.LONG(), Types.INT()])) \ .add_sink(self.test_sink) self.env.execute('test_event_time_dynamic_gap_session_window') results = self.test_sink.get_results() expected = ['(hi,1,8,4)', '(hi,9,30,3)'] self.assert_equals_sorted(expected, results)
def test_event_time_session_window_with_purging_trigger(self): data_stream = self.env.from_collection([ ('hi', 1), ('hi', 2), ('hi', 3), ('hi', 4), ('hi', 8), ('hi', 9), ('hi', 15)], type_info=Types.TUPLE([Types.STRING(), Types.INT()])) # type: DataStream watermark_strategy = WatermarkStrategy.for_monotonous_timestamps() \ .with_timestamp_assigner(SecondColumnTimestampAssigner()) data_stream.assign_timestamps_and_watermarks(watermark_strategy) \ .key_by(lambda x: x[0], key_type=Types.STRING()) \ .window(EventTimeSessionWindows.with_gap(Time.milliseconds(3))) \ .trigger(PurgingTrigger.of(EventTimeTrigger.create())) \ .process(CountWindowProcessFunction(), Types.TUPLE([Types.STRING(), Types.LONG(), Types.LONG(), Types.INT()])) \ .add_sink(self.test_sink) self.env.execute('test_event_time_session_window_with_purging_trigger') results = self.test_sink.get_results() expected = ['(hi,1,7,4)', '(hi,8,12,2)', '(hi,15,18,1)'] self.assert_equals_sorted(expected, results)
def test_from_and_to_data_stream_event_time(self): from pyflink.table import Schema ds = self.env.from_collection([(1, 42, "a"), (2, 5, "a"), (3, 1000, "c"), (100, 1000, "c")], Types.ROW_NAMED( ["a", "b", "c"], [Types.LONG(), Types.INT(), Types.STRING()])) ds = ds.assign_timestamps_and_watermarks( WatermarkStrategy.for_monotonous_timestamps() .with_timestamp_assigner(MyTimestampAssigner())) table = self.t_env.from_data_stream(ds, Schema.new_builder() .column_by_metadata("rowtime", "TIMESTAMP_LTZ(3)") .watermark("rowtime", "SOURCE_WATERMARK()") .build()) self.assertEqual("""( `a` BIGINT, `b` INT, `c` STRING, `rowtime` TIMESTAMP_LTZ(3) *ROWTIME* METADATA, WATERMARK FOR `rowtime`: TIMESTAMP_LTZ(3) AS SOURCE_WATERMARK() )""", table._j_table.getResolvedSchema().toString()) self.t_env.create_temporary_view("t", ds, Schema.new_builder() .column_by_metadata("rowtime", "TIMESTAMP_LTZ(3)") .watermark("rowtime", "SOURCE_WATERMARK()") .build()) result = self.t_env.execute_sql("SELECT " "c, SUM(b) " "FROM t " "GROUP BY c, TUMBLE(rowtime, INTERVAL '0.005' SECOND)") with result.collect() as result: collected_result = [str(item) for item in result] expected_result = [item for item in map(str, [Row('a', 47), Row('c', 1000), Row('c', 1000)])] expected_result.sort() collected_result.sort() self.assertEqual(expected_result, collected_result) ds = self.t_env.to_data_stream(table) ds.key_by(lambda k: k.c, key_type=Types.STRING()) \ .window(MyTumblingEventTimeWindow()) \ .apply(SumWindowFunction(), Types.TUPLE([Types.STRING(), Types.INT()])) \ .add_sink(self.test_sink) self.env.execute() expected_results = ['(a,47)', '(c,1000)', '(c,1000)'] actual_results = self.test_sink.get_results(False) expected_results.sort() actual_results.sort() self.assertEqual(expected_results, actual_results)
def test_from_and_to_changelog_stream_event_time(self): from pyflink.table import Schema self.env.set_parallelism(1) ds = self.env.from_collection( [(1, 42, "a"), (2, 5, "a"), (3, 1000, "c"), (100, 1000, "c")], Types.ROW([Types.LONG(), Types.INT(), Types.STRING()])) ds = ds.assign_timestamps_and_watermarks( WatermarkStrategy.for_monotonous_timestamps( ).with_timestamp_assigner(MyTimestampAssigner())) changelog_stream = ds.map(lambda t: Row(t.f1, t.f2), Types.ROW([Types.INT(), Types.STRING()])) # derive physical columns and add a rowtime table = self.t_env.from_changelog_stream( changelog_stream, Schema.new_builder().column_by_metadata( "rowtime", DataTypes.TIMESTAMP_LTZ(3)).column_by_expression( "computed", str(col("f1").upper_case)).watermark( "rowtime", str(source_watermark())).build()) self.t_env.create_temporary_view("t", table) # access and reorder columns reordered = self.t_env.sql_query("SELECT computed, rowtime, f0 FROM t") # write out the rowtime column with fully declared schema result = self.t_env.to_changelog_stream( reordered, Schema.new_builder().column( "f1", DataTypes.STRING()).column_by_metadata( "rowtime", DataTypes.TIMESTAMP_LTZ(3)).column_by_expression( "ignored", str(col("f1").upper_case)).column( "f0", DataTypes.INT()).build()) # test event time window and field access result.key_by(lambda k: k.f1) \ .window(MyTumblingEventTimeWindow()) \ .apply(SumWindowFunction(), Types.TUPLE([Types.STRING(), Types.INT()])) \ .add_sink(self.test_sink) self.env.execute() expected_results = ['(A,47)', '(C,1000)', '(C,1000)'] actual_results = self.test_sink.get_results(False) expected_results.sort() actual_results.sort() self.assertEqual(expected_results, actual_results)
def test_to_retract_stream(self): self.env.set_parallelism(1) t_env = StreamTableEnvironment.create( self.env, environment_settings=EnvironmentSettings.in_streaming_mode()) table = t_env.from_elements([(1, "Hi", "Hello"), (1, "Hi", "Hello")], ["a", "b", "c"]) new_table = table.group_by("c").select("a.sum, c as b") ds = t_env.to_retract_stream(table=new_table, type_info=Types.ROW([Types.LONG(), Types.STRING()])) test_sink = DataStreamTestSinkFunction() ds.map(lambda x: x).add_sink(test_sink) self.env.execute("test_to_retract_stream") result = test_sink.get_results(True) expected = ["(True, Row(f0=1, f1='Hello'))", "(False, Row(f0=1, f1='Hello'))", "(True, Row(f0=2, f1='Hello'))"] self.assertEqual(result, expected)
def test_to_append_stream(self): self.env.set_parallelism(1) t_env = StreamTableEnvironment.create( self.env, environment_settings=EnvironmentSettings.in_streaming_mode()) table = t_env.from_elements([(1, "Hi", "Hello"), (2, "Hello", "Hi")], ["a", "b", "c"]) new_table = table.select("a + 1, b + 'flink', c") ds = t_env.to_append_stream(table=new_table, type_info=Types.ROW([Types.LONG(), Types.STRING(), Types.STRING()])) test_sink = DataStreamTestSinkFunction() ds.add_sink(test_sink) self.env.execute("test_to_append_stream") result = test_sink.get_results(False) expected = ['+I[2, Hiflink, Hello]', '+I[3, Helloflink, Hi]'] self.assertEqual(result, expected)
def _create_parquet_basic_row_and_data() -> Tuple[RowType, RowTypeInfo, List[Row]]: row_type = DataTypes.ROW([ DataTypes.FIELD('char', DataTypes.CHAR(10)), DataTypes.FIELD('varchar', DataTypes.VARCHAR(10)), DataTypes.FIELD('binary', DataTypes.BINARY(10)), DataTypes.FIELD('varbinary', DataTypes.VARBINARY(10)), DataTypes.FIELD('boolean', DataTypes.BOOLEAN()), DataTypes.FIELD('decimal', DataTypes.DECIMAL(2, 0)), DataTypes.FIELD('int', DataTypes.INT()), DataTypes.FIELD('bigint', DataTypes.BIGINT()), DataTypes.FIELD('double', DataTypes.DOUBLE()), DataTypes.FIELD('date', DataTypes.DATE().bridged_to('java.sql.Date')), DataTypes.FIELD('time', DataTypes.TIME().bridged_to('java.sql.Time')), DataTypes.FIELD('timestamp', DataTypes.TIMESTAMP(3).bridged_to('java.sql.Timestamp')), DataTypes.FIELD('timestamp_ltz', DataTypes.TIMESTAMP_LTZ(3)), ]) row_type_info = Types.ROW_NAMED( ['char', 'varchar', 'binary', 'varbinary', 'boolean', 'decimal', 'int', 'bigint', 'double', 'date', 'time', 'timestamp', 'timestamp_ltz'], [Types.STRING(), Types.STRING(), Types.PRIMITIVE_ARRAY(Types.BYTE()), Types.PRIMITIVE_ARRAY(Types.BYTE()), Types.BOOLEAN(), Types.BIG_DEC(), Types.INT(), Types.LONG(), Types.DOUBLE(), Types.SQL_DATE(), Types.SQL_TIME(), Types.SQL_TIMESTAMP(), Types.INSTANT()] ) datetime_ltz = datetime.datetime(1970, 2, 3, 4, 5, 6, 700000, tzinfo=pytz.timezone('UTC')) timestamp_ltz = Instant.of_epoch_milli( ( calendar.timegm(datetime_ltz.utctimetuple()) + calendar.timegm(time.localtime(0)) ) * 1000 + datetime_ltz.microsecond // 1000 ) data = [Row( char='char', varchar='varchar', binary=b'binary', varbinary=b'varbinary', boolean=True, decimal=Decimal(1.5), int=2147483647, bigint=-9223372036854775808, double=2e-308, date=datetime.date(1970, 1, 1), time=datetime.time(1, 1, 1), timestamp=datetime.datetime(1970, 1, 2, 3, 4, 5, 600000), timestamp_ltz=timestamp_ltz )] return row_type, row_type_info, data
def popular_destination_query(): env = StreamExecutionEnvironment.get_execution_environment() t_env = StreamTableEnvironment.create(stream_execution_environment=env) t_env.execute_sql( create_table_ddl( "WATERMARK FOR pickupTime AS pickupTime - INTERVAL '30' SECONDS")) query = f"""SELECT destLocationId, wstart, wend, cnt FROM (SELECT destLocationId, HOP_START(pickupTime, INTERVAL '5' MINUTE, INTERVAL '15' MINUTE) AS wstart, HOP_END(pickupTime, INTERVAL '5' MINUTE, INTERVAL '15' MINUTE) AS wend, COUNT(destLocationId) AS cnt FROM (SELECT pickupTime, destLocationId FROM TaxiRide) GROUP BY destLocationId, HOP(pickupTime, INTERVAL '5' MINUTE, INTERVAL '15' MINUTE) ) WHERE cnt > {args.threshold} """ results = t_env.sql_query(query) t_env.to_append_stream( results, Types.ROW_NAMED(['destLocationId', 'wstart', 'wend', 'cnt'], [ Types.INT(), Types.SQL_TIMESTAMP(), Types.SQL_TIMESTAMP(), Types.LONG() ])).print() env.execute('Popular-Destination')
def open(self, ctx: Context[Any, CountWindow]): count_descriptor = ValueStateDescriptor('slide-count-assigner', Types.LONG()) self._count = ctx.get_partitioned_state(count_descriptor)
def test_from_java_type(self): basic_int_type_info = Types.INT() self.assertEqual(basic_int_type_info, _from_java_type(basic_int_type_info.get_java_type_info())) basic_short_type_info = Types.SHORT() self.assertEqual(basic_short_type_info, _from_java_type(basic_short_type_info.get_java_type_info())) basic_long_type_info = Types.LONG() self.assertEqual(basic_long_type_info, _from_java_type(basic_long_type_info.get_java_type_info())) basic_float_type_info = Types.FLOAT() self.assertEqual(basic_float_type_info, _from_java_type(basic_float_type_info.get_java_type_info())) basic_double_type_info = Types.DOUBLE() self.assertEqual(basic_double_type_info, _from_java_type(basic_double_type_info.get_java_type_info())) basic_char_type_info = Types.CHAR() self.assertEqual(basic_char_type_info, _from_java_type(basic_char_type_info.get_java_type_info())) basic_byte_type_info = Types.BYTE() self.assertEqual(basic_byte_type_info, _from_java_type(basic_byte_type_info.get_java_type_info())) basic_big_int_type_info = Types.BIG_INT() self.assertEqual(basic_big_int_type_info, _from_java_type(basic_big_int_type_info.get_java_type_info())) basic_big_dec_type_info = Types.BIG_DEC() self.assertEqual(basic_big_dec_type_info, _from_java_type(basic_big_dec_type_info.get_java_type_info())) basic_sql_date_type_info = Types.SQL_DATE() self.assertEqual(basic_sql_date_type_info, _from_java_type(basic_sql_date_type_info.get_java_type_info())) basic_sql_time_type_info = Types.SQL_TIME() self.assertEqual(basic_sql_time_type_info, _from_java_type(basic_sql_time_type_info.get_java_type_info())) basic_sql_timestamp_type_info = Types.SQL_TIMESTAMP() self.assertEqual(basic_sql_timestamp_type_info, _from_java_type(basic_sql_timestamp_type_info.get_java_type_info())) row_type_info = Types.ROW([Types.INT(), Types.STRING()]) self.assertEqual(row_type_info, _from_java_type(row_type_info.get_java_type_info())) tuple_type_info = Types.TUPLE([Types.CHAR(), Types.INT()]) self.assertEqual(tuple_type_info, _from_java_type(tuple_type_info.get_java_type_info())) primitive_int_array_type_info = Types.PRIMITIVE_ARRAY(Types.INT()) self.assertEqual(primitive_int_array_type_info, _from_java_type(primitive_int_array_type_info.get_java_type_info())) object_array_type_info = Types.OBJECT_ARRAY(Types.SQL_DATE()) self.assertEqual(object_array_type_info, _from_java_type(object_array_type_info.get_java_type_info())) pickled_byte_array_type_info = Types.PICKLED_BYTE_ARRAY() self.assertEqual(pickled_byte_array_type_info, _from_java_type(pickled_byte_array_type_info.get_java_type_info())) sql_date_type_info = Types.SQL_DATE() self.assertEqual(sql_date_type_info, _from_java_type(sql_date_type_info.get_java_type_info())) map_type_info = Types.MAP(Types.INT(), Types.STRING()) self.assertEqual(map_type_info, _from_java_type(map_type_info.get_java_type_info())) list_type_info = Types.LIST(Types.INT()) self.assertEqual(list_type_info, _from_java_type(list_type_info.get_java_type_info()))
def to_java_typeinfo(type_info: TypeInformation): if isinstance(type_info, BasicTypeInfo): basic_type = type_info._basic_type if basic_type == BasicType.STRING: j_typeinfo = JTypes.STRING elif basic_type == BasicType.BYTE: j_typeinfo = JTypes.LONG elif basic_type == BasicType.BOOLEAN: j_typeinfo = JTypes.BOOLEAN elif basic_type == BasicType.SHORT: j_typeinfo = JTypes.LONG elif basic_type == BasicType.INT: j_typeinfo = JTypes.LONG elif basic_type == BasicType.LONG: j_typeinfo = JTypes.LONG elif basic_type == BasicType.FLOAT: j_typeinfo = JTypes.DOUBLE elif basic_type == BasicType.DOUBLE: j_typeinfo = JTypes.DOUBLE elif basic_type == BasicType.CHAR: j_typeinfo = JTypes.STRING elif basic_type == BasicType.BIG_INT: j_typeinfo = JTypes.BIG_INT elif basic_type == BasicType.BIG_DEC: j_typeinfo = JTypes.BIG_DEC elif basic_type == BasicType.INSTANT: j_typeinfo = JTypes.INSTANT else: raise TypeError("Invalid BasicType %s." % basic_type) elif isinstance(type_info, PrimitiveArrayTypeInfo): element_type = type_info._element_type if element_type == Types.BOOLEAN(): j_typeinfo = JPrimitiveArrayTypeInfo.BOOLEAN_PRIMITIVE_ARRAY_TYPE_INFO elif element_type == Types.BYTE(): j_typeinfo = JPrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO elif element_type == Types.SHORT(): j_typeinfo = JPrimitiveArrayTypeInfo.SHORT_PRIMITIVE_ARRAY_TYPE_INFO elif element_type == Types.INT(): j_typeinfo = JPrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO elif element_type == Types.LONG(): j_typeinfo = JPrimitiveArrayTypeInfo.LONG_PRIMITIVE_ARRAY_TYPE_INFO elif element_type == Types.FLOAT(): j_typeinfo = JPrimitiveArrayTypeInfo.FLOAT_PRIMITIVE_ARRAY_TYPE_INFO elif element_type == Types.DOUBLE(): j_typeinfo = JPrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO elif element_type == Types.CHAR(): j_typeinfo = JPrimitiveArrayTypeInfo.CHAR_PRIMITIVE_ARRAY_TYPE_INFO else: raise TypeError("Invalid element type for a primitive array.") elif isinstance(type_info, BasicArrayTypeInfo): element_type = type_info._element_type if element_type == Types.BOOLEAN(): j_typeinfo = JBasicArrayTypeInfo.BOOLEAN_ARRAY_TYPE_INFO elif element_type == Types.BYTE(): j_typeinfo = JBasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO elif element_type == Types.SHORT(): j_typeinfo = JBasicArrayTypeInfo.SHORT_ARRAY_TYPE_INFO elif element_type == Types.INT(): j_typeinfo = JBasicArrayTypeInfo.INT_ARRAY_TYPE_INFO elif element_type == Types.LONG(): j_typeinfo = JBasicArrayTypeInfo.LONG_ARRAY_TYPE_INFO elif element_type == Types.FLOAT(): j_typeinfo = JBasicArrayTypeInfo.FLOAT_ARRAY_TYPE_INFO elif element_type == Types.DOUBLE(): j_typeinfo = JBasicArrayTypeInfo.DOUBLE_ARRAY_TYPE_INFO elif element_type == Types.CHAR(): j_typeinfo = JBasicArrayTypeInfo.CHAR_ARRAY_TYPE_INFO elif element_type == Types.STRING(): j_typeinfo = JBasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO else: raise TypeError("Invalid element type for a basic array.") elif isinstance(type_info, ObjectArrayTypeInfo): element_type = type_info._element_type j_typeinfo = JTypes.OBJECT_ARRAY(to_java_typeinfo(element_type)) elif isinstance(type_info, MapTypeInfo): j_key_typeinfo = to_java_typeinfo(type_info._key_type_info) j_value_typeinfo = to_java_typeinfo(type_info._value_type_info) j_typeinfo = JMapTypeInfo(j_key_typeinfo, j_value_typeinfo) else: j_typeinfo = JPickledByteArrayTypeInfo.PICKLED_BYTE_ARRAY_TYPE_INFO return j_typeinfo
def open(self, runtime_context: RuntimeContext): state_desc = MapStateDescriptor('map', Types.LONG(), Types.LONG()) self.state = runtime_context.get_map_state(state_desc)
def open(self, runtime_context: RuntimeContext): state_desc = ValueStateDescriptor('cnt', Types.LONG()) self.cnt_state = runtime_context.get_state(state_desc)
def __init__(self, count_elements: int): self._count_elements = count_elements self._count_state_desc = ValueStateDescriptor( "trigger-count-%s" % count_elements, Types.LONG()) self._ctx = None # type: TriggerContext