def test_state_ttl_config_proto(self): from pyflink.datastream.state import StateTtlConfig from pyflink.common.time import Time state_ttl_config = StateTtlConfig \ .new_builder(Time.milliseconds(1000)) \ .set_update_type(StateTtlConfig.UpdateType.OnCreateAndWrite) \ .set_state_visibility(StateTtlConfig.StateVisibility.NeverReturnExpired) \ .cleanup_full_snapshot() \ .cleanup_incrementally(10, True) \ .cleanup_in_rocksdb_compact_filter(1000) \ .build() state_ttl_config_proto = state_ttl_config._to_proto() state_ttl_config = StateTtlConfig._from_proto(state_ttl_config_proto) self.assertEqual(state_ttl_config.get_ttl(), Time.milliseconds(1000)) self.assertEqual( state_ttl_config.get_update_type(), StateTtlConfig.UpdateType.OnCreateAndWrite) self.assertEqual( state_ttl_config.get_state_visibility(), StateTtlConfig.StateVisibility.NeverReturnExpired) self.assertEqual( state_ttl_config.get_ttl_time_characteristic(), StateTtlConfig.TtlTimeCharacteristic.ProcessingTime) cleanup_strategies = state_ttl_config.get_cleanup_strategies() self.assertTrue(cleanup_strategies.is_cleanup_in_background()) self.assertTrue(cleanup_strategies.in_full_snapshot()) incremental_cleanup_strategy = cleanup_strategies.get_incremental_cleanup_strategy() self.assertEqual(incremental_cleanup_strategy.get_cleanup_size(), 10) self.assertTrue(incremental_cleanup_strategy.run_cleanup_for_every_record()) rocksdb_compact_filter_cleanup_strategy = \ cleanup_strategies.get_rocksdb_compact_filter_cleanup_strategy() self.assertEqual( rocksdb_compact_filter_cleanup_strategy.get_query_time_after_num_entries(), 1000)
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 test_window_aggregate_accumulator_type(self): data_stream = self.env.from_collection( [('a', 1), ('a', 2), ('b', 3), ('a', 6), ('b', 8), ('b', 9), ('a', 15)], type_info=Types.TUPLE([Types.STRING(), Types.INT()])) # type: DataStream watermark_strategy = WatermarkStrategy.for_monotonous_timestamps() \ .with_timestamp_assigner(SecondColumnTimestampAssigner()) class MyAggregateFunction(AggregateFunction): def create_accumulator(self) -> Tuple[int, str]: return 0, '' def add(self, value: Tuple[str, int], accumulator: Tuple[int, str]) -> Tuple[int, str]: return value[1] + accumulator[0], value[0] def get_result(self, accumulator: Tuple[str, int]): return accumulator[1], accumulator[0] def merge(self, acc_a: Tuple[int, str], acc_b: Tuple[int, str]): return acc_a[0] + acc_b[0], acc_a[1] data_stream.assign_timestamps_and_watermarks(watermark_strategy) \ .key_by(lambda x: x[0], key_type=Types.STRING()) \ .window(EventTimeSessionWindows.with_gap(Time.milliseconds(2))) \ .aggregate(MyAggregateFunction(), accumulator_type=Types.TUPLE([Types.INT(), Types.STRING()]), output_type=Types.TUPLE([Types.STRING(), Types.INT()])) \ .add_sink(self.test_sink) self.env.execute('test_time_window_aggregate_accumulator_type') results = self.test_sink.get_results() expected = ['(a,15)', '(a,3)', '(a,6)', '(b,17)', '(b,3)'] self.assert_equals_sorted(expected, results)
def test_window_all_reduce_process(self): self.env.set_parallelism(1) data_stream = self.env.from_collection( [('a', 1), ('a', 2), ('b', 3), ('a', 6), ('b', 8), ('b', 9), ('a', 15)], type_info=Types.TUPLE([Types.STRING(), Types.INT()])) # type: DataStream watermark_strategy = WatermarkStrategy.for_monotonous_timestamps() \ .with_timestamp_assigner(SecondColumnTimestampAssigner()) class MyProcessFunction(ProcessAllWindowFunction): def process(self, context: 'ProcessAllWindowFunction.Context', elements: Iterable[Tuple[str, int]]) -> Iterable[str]: yield "current window start at {}, reduce result {}".format( context.window().start, next(iter(elements)), ) data_stream.assign_timestamps_and_watermarks(watermark_strategy) \ .window_all(EventTimeSessionWindows.with_gap(Time.milliseconds(2))) \ .reduce(lambda a, b: (a[0], a[1] + b[1]), window_function=MyProcessFunction(), output_type=Types.STRING()) \ .add_sink(self.test_sink) self.env.execute('test_window_all_reduce_process') results = self.test_sink.get_results() expected = [ "current window start at 1, reduce result ('a', 6)", "current window start at 6, reduce result ('a', 23)", "current window start at 15, reduce result ('a', 15)" ] self.assert_equals_sorted(expected, results)
def test_event_time_sliding_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(SlidingEventTimeWindows.of(Time.milliseconds(5), Time.milliseconds(2))) \ .process(CountWindowProcessFunction(), Types.TUPLE([Types.STRING(), Types.INT()])) \ .add_sink(self.test_sink) self.env.execute('test_event_time_sliding_window') results = self.test_sink.get_results() expected = ['(hi,2)', '(hi,4)', '(hi,4)', '(hi,3)', '(hi,2)', '(hi,2)', '(hi,1)', '(hi,1)'] self.assert_equals_sorted(expected, results)
def test_window_aggregate_passthrough(self): data_stream = self.env.from_collection( [('a', 1), ('a', 2), ('b', 3), ('a', 6), ('b', 8), ('b', 9), ('a', 15)], type_info=Types.TUPLE([Types.STRING(), Types.INT()])) # type: DataStream watermark_strategy = WatermarkStrategy.for_monotonous_timestamps() \ .with_timestamp_assigner(SecondColumnTimestampAssigner()) class MyAggregateFunction(AggregateFunction): def create_accumulator(self) -> Tuple[str, Dict[int, int]]: return '', {0: 0, 1: 0} def add( self, value: Tuple[str, int], accumulator: Tuple[str, Dict[int, int]] ) -> Tuple[str, Dict[int, int]]: number_map = accumulator[1] number_map[value[1] % 2] += 1 return value[0], number_map def get_result( self, accumulator: Tuple[str, Dict[int, int]]) -> Tuple[str, int]: number_map = accumulator[1] return accumulator[0], number_map[0] - number_map[1] def merge( self, acc_a: Tuple[str, Dict[int, int]], acc_b: Tuple[str, Dict[int, int]]) -> Tuple[str, Dict[int, int]]: number_map_a = acc_a[1] number_map_b = acc_b[1] new_number_map = { 0: number_map_a[0] + number_map_b[0], 1: number_map_a[1] + number_map_b[1] } return acc_a[0], new_number_map data_stream.assign_timestamps_and_watermarks(watermark_strategy) \ .key_by(lambda x: x[0], key_type=Types.STRING()) \ .window(EventTimeSessionWindows.with_gap(Time.milliseconds(2))) \ .aggregate(MyAggregateFunction(), output_type=Types.TUPLE([Types.STRING(), Types.INT()])) \ .add_sink(self.test_sink) self.env.execute('test_time_window_aggregate_passthrough') results = self.test_sink.get_results() expected = ['(a,-1)', '(a,0)', '(a,1)', '(b,-1)', '(b,0)'] self.assert_equals_sorted(expected, results)
def test_window_aggregate_process(self): data_stream = self.env.from_collection( [('a', 1), ('a', 2), ('b', 3), ('a', 6), ('b', 8), ('b', 9), ('a', 15)], type_info=Types.TUPLE([Types.STRING(), Types.INT()])) # type: DataStream watermark_strategy = WatermarkStrategy.for_monotonous_timestamps() \ .with_timestamp_assigner(SecondColumnTimestampAssigner()) class MyAggregateFunction(AggregateFunction): def create_accumulator(self) -> Tuple[int, str]: return 0, '' def add(self, value: Tuple[str, int], accumulator: Tuple[int, str]) -> Tuple[int, str]: return value[1] + accumulator[0], value[0] def get_result(self, accumulator: Tuple[str, int]): return accumulator[1], accumulator[0] def merge(self, acc_a: Tuple[int, str], acc_b: Tuple[int, str]): return acc_a[0] + acc_b[0], acc_a[1] class MyProcessWindowFunction(ProcessWindowFunction): def process(self, key: str, context: ProcessWindowFunction.Context, elements: Iterable[Tuple[str, int]]) -> Iterable[str]: agg_result = next(iter(elements)) yield "key {} timestamp sum {}".format(agg_result[0], agg_result[1]) def clear(self, context: ProcessWindowFunction.Context) -> None: pass data_stream.assign_timestamps_and_watermarks(watermark_strategy) \ .key_by(lambda x: x[0], key_type=Types.STRING()) \ .window(EventTimeSessionWindows.with_gap(Time.milliseconds(2))) \ .aggregate(MyAggregateFunction(), window_function=MyProcessWindowFunction(), accumulator_type=Types.TUPLE([Types.INT(), Types.STRING()]), output_type=Types.STRING()) \ .add_sink(self.test_sink) self.env.execute('test_time_window_aggregate_accumulator_type') results = self.test_sink.get_results() expected = [ 'key a timestamp sum 15', 'key a timestamp sum 3', 'key a timestamp sum 6', 'key b timestamp sum 17', 'key b timestamp sum 3' ] self.assert_equals_sorted(expected, results)
def _from_proto(proto): update_type = StateTtlConfig.UpdateType._from_proto(proto.update_type) state_visibility = StateTtlConfig.StateVisibility._from_proto(proto.state_visibility) ttl_time_characteristic = \ StateTtlConfig.TtlTimeCharacteristic._from_proto(proto.ttl_time_characteristic) ttl = Time.milliseconds(proto.ttl) cleanup_strategies = StateTtlConfig.CleanupStrategies._from_proto(proto.cleanup_strategies) builder = StateTtlConfig.new_builder(ttl) \ .set_update_type(update_type) \ .set_state_visibility(state_visibility) \ .set_ttl_time_characteristic(ttl_time_characteristic) builder._strategies = cleanup_strategies._strategies builder._is_cleanup_in_background = cleanup_strategies._is_cleanup_in_background return builder.build()
def test_window_reduce_passthrough(self): data_stream = self.env.from_collection([ ('a', 1), ('a', 2), ('b', 3), ('a', 6), ('b', 8), ('b', 9), ('a', 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(2))) \ .reduce(lambda a, b: (b[0], a[1] + b[1]), output_type=Types.TUPLE([Types.STRING(), Types.INT()])) \ .add_sink(self.test_sink) self.env.execute('test_time_window_reduce_passthrough') results = self.test_sink.get_results() expected = ['(a,3)', '(a,6)', '(a,15)', '(b,3)', '(b,17)'] self.assert_equals_sorted(expected, results)
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 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_event_time_tumbling_window_all(self): data_stream = self.env.from_collection( [('hi', 1), ('hello', 2), ('hi', 3), ('hello', 4), ('hello', 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) \ .window_all(TumblingEventTimeWindows.of(Time.milliseconds(5))) \ .process(CountAllWindowProcessFunction(), Types.TUPLE([Types.LONG(), Types.LONG(), Types.INT()])) \ .add_sink(self.test_sink) self.env.execute('test_event_time_tumbling_window_all') results = self.test_sink.get_results() expected = ['(0,5,4)', '(15,20,1)', '(5,10,3)'] self.assert_equals_sorted(expected, results)