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_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 _open(self): # start BeamFnLoopbackWorkerPoolServicer when executed in MiniCluster j_configuration = get_j_env_configuration( self._j_stream_execution_environment) def startup_loopback_server(): from pyflink.common import Configuration from pyflink.fn_execution.beam.beam_worker_pool_service import \ BeamFnLoopbackWorkerPoolServicer config = Configuration(j_configuration=j_configuration) config.set_string("python.loopback-server.address", BeamFnLoopbackWorkerPoolServicer().start()) python_worker_execution_mode = os.environ.get( '_python_worker_execution_mode') if python_worker_execution_mode is None: if is_local_deployment(j_configuration): startup_loopback_server() elif python_worker_execution_mode == 'loopback': if is_local_deployment(j_configuration): startup_loopback_server() else: raise ValueError( "Loopback mode is enabled, however the job wasn't configured to " "run in local deployment mode") elif python_worker_execution_mode != 'process': raise ValueError( "It only supports to execute the Python worker in 'loopback' mode and 'process' " "mode, unknown mode '%s' is configured" % python_worker_execution_mode)
def test_set_runtime_mode(self): self.env.set_runtime_mode(RuntimeExecutionMode.BATCH) config = get_j_env_configuration(self.env._j_stream_execution_environment) runtime_mode = config.getValue( get_gateway().jvm.org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE) self.assertEqual(runtime_mode, "BATCH")
def _generate_stream_graph(self, clear_transformations: bool = False, job_name: str = None) \ -> JavaObject: gateway = get_gateway() JPythonConfigUtil = gateway.jvm.org.apache.flink.python.util.PythonConfigUtil # start BeamFnLoopbackWorkerPoolServicer when executed in MiniCluster j_configuration = get_j_env_configuration(self._j_stream_execution_environment) def startup_loopback_server(): jvm = gateway.jvm env_config = JPythonConfigUtil.getEnvironmentConfig( self._j_stream_execution_environment) parallelism = self.get_parallelism() if parallelism > 1 and env_config.containsKey(jvm.PythonOptions.PYTHON_ARCHIVES.key()): import logging logging.warning("Loopback mode is disabled as python archives are used and the " "parallelism of the job is greater than 1. The Python user-defined " "functions will be executed in an independent Python process.") else: from pyflink.fn_execution.beam.beam_worker_pool_service import \ BeamFnLoopbackWorkerPoolServicer j_env = jvm.System.getenv() get_field_value(j_env, "m").put( 'PYFLINK_LOOPBACK_SERVER_ADDRESS', BeamFnLoopbackWorkerPoolServicer().start()) python_worker_execution_mode = None if hasattr(self, "_python_worker_execution_mode"): python_worker_execution_mode = getattr(self, "_python_worker_execution_mode") if python_worker_execution_mode is None: if is_local_deployment(j_configuration): startup_loopback_server() elif python_worker_execution_mode == 'loopback': if is_local_deployment(j_configuration): startup_loopback_server() else: raise ValueError("Loopback mode is enabled, however the job wasn't configured to " "run in local deployment mode") elif python_worker_execution_mode != 'process': raise ValueError( "It only supports to execute the Python worker in 'loopback' mode and 'process' " "mode, unknown mode '%s' is configured" % python_worker_execution_mode) JPythonConfigUtil.configPythonOperator(self._j_stream_execution_environment) gateway.jvm.org.apache.flink.python.chain.PythonOperatorChainingOptimizer.apply( self._j_stream_execution_environment) JPythonConfigUtil.setPartitionCustomOperatorNumPartitions( get_field_value(self._j_stream_execution_environment, "transformations")) j_stream_graph = self._j_stream_execution_environment.getStreamGraph(clear_transformations) if job_name is not None: j_stream_graph.setJobName(job_name) return j_stream_graph
def test_get_execution_environment_with_config(self): configuration = Configuration() configuration.set_integer('parallelism.default', 12) configuration.set_string('pipeline.name', 'haha') env = StreamExecutionEnvironment.get_execution_environment(configuration) execution_config = env.get_config() self.assertEqual(execution_config.get_parallelism(), 12) config = Configuration( j_configuration=get_j_env_configuration(env._j_stream_execution_environment)) self.assertEqual(config.get_string('pipeline.name', ''), 'haha')
def setUp(self): self.env = StreamExecutionEnvironment.get_execution_environment() self._load_dependency_jars() config = Configuration( j_configuration=get_j_env_configuration(self.env._j_stream_execution_environment)) config.set_boolean("execution.checkpointing.checkpoints-after-tasks-finish.enabled", True) self.env.set_parallelism(4) self.env.enable_checkpointing(100) self.env.set_restart_strategy(RestartStrategies.no_restart()) self.t_env = StreamTableEnvironment.create(self.env) self.temp_dir = tempfile.mkdtemp()
def _generate_stream_graph(self, clear_transformations: bool = False, job_name: str = None) \ -> JavaObject: gateway = get_gateway() JPythonConfigUtil = gateway.jvm.org.apache.flink.python.util.PythonConfigUtil # start BeamFnLoopbackWorkerPoolServicer when executed in MiniCluster j_configuration = get_j_env_configuration( self._j_stream_execution_environment) def startup_loopback_server(): from pyflink.fn_execution.beam.beam_worker_pool_service import \ BeamFnLoopbackWorkerPoolServicer jvm = gateway.jvm j_env = jvm.System.getenv() get_field_value(j_env, "m").put( 'PYFLINK_LOOPBACK_SERVER_ADDRESS', BeamFnLoopbackWorkerPoolServicer().start()) python_worker_execution_mode = None if hasattr(self, "_python_worker_execution_mode"): python_worker_execution_mode = getattr( self, "_python_worker_execution_mode") if python_worker_execution_mode is None: if is_local_deployment(j_configuration): startup_loopback_server() elif python_worker_execution_mode == 'loopback': if is_local_deployment(j_configuration): startup_loopback_server() else: raise ValueError( "Loopback mode is enabled, however the job wasn't configured to " "run in local deployment mode") elif python_worker_execution_mode != 'process': raise ValueError( "It only supports to execute the Python worker in 'loopback' mode and 'process' " "mode, unknown mode '%s' is configured" % python_worker_execution_mode) JPythonConfigUtil.configPythonOperator( self._j_stream_execution_environment) gateway.jvm.org.apache.flink.python.chain.PythonOperatorChainingOptimizer.apply( self._j_stream_execution_environment) JPythonConfigUtil.setPartitionCustomOperatorNumPartitions( get_field_value(self._j_stream_execution_environment, "transformations")) j_stream_graph = self._j_stream_execution_environment.getStreamGraph( clear_transformations) if job_name is not None: j_stream_graph.setJobName(job_name) return j_stream_graph
def setUp(self) -> None: from pyflink.datastream import StreamExecutionEnvironment super(DataStreamConversionTestCases, self).setUp() self.env = StreamExecutionEnvironment.get_execution_environment() self.t_env = StreamTableEnvironment.create(self.env) self.env.set_parallelism(2) config = get_j_env_configuration(self.env._j_stream_execution_environment) config.setString("akka.ask.timeout", "20 s") self.t_env.get_config().get_configuration().set_string( "python.fn-execution.bundle.size", "1") self.test_sink = DataStreamTestSinkFunction()
def _generate_stream_graph(self, clear_transformations: bool = False, job_name: str = None) \ -> JavaObject: gateway = get_gateway() JPythonConfigUtil = gateway.jvm.org.apache.flink.python.util.PythonConfigUtil # start BeamFnLoopbackWorkerPoolServicer when executed in MiniCluster j_configuration = get_j_env_configuration(self._j_stream_execution_environment) if not self._remote_mode and is_local_deployment(j_configuration): from pyflink.common import Configuration from pyflink.fn_execution.beam.beam_worker_pool_service import \ BeamFnLoopbackWorkerPoolServicer jvm = gateway.jvm env_config = JPythonConfigUtil.getEnvironmentConfig( self._j_stream_execution_environment) parallelism = self.get_parallelism() if parallelism > 1 and env_config.containsKey(jvm.PythonOptions.PYTHON_ARCHIVES.key()): import logging logging.warning("Lookback mode is disabled as python archives are used and the " "parallelism of the job is greater than 1. The Python user-defined " "functions will be executed in an independent Python process.") else: config = Configuration(j_configuration=j_configuration) config.set_string( "loopback.server.address", BeamFnLoopbackWorkerPoolServicer().start()) JPythonConfigUtil.configPythonOperator(self._j_stream_execution_environment) gateway.jvm.org.apache.flink.python.chain.PythonOperatorChainingOptimizer.apply( self._j_stream_execution_environment) JPythonConfigUtil.setPartitionCustomOperatorNumPartitions( get_field_value(self._j_stream_execution_environment, "transformations")) j_stream_graph = self._j_stream_execution_environment.getStreamGraph(clear_transformations) if job_name is not None: j_stream_graph.setJobName(job_name) return j_stream_graph
def test_set_sys_executable_for_local_mode(self): jvm = get_gateway().jvm actual_executable = get_j_env_configuration(self.t_env._get_j_env()) \ .getString(jvm.PythonOptions.PYTHON_EXECUTABLE.key(), None) self.assertEqual(sys.executable, actual_executable)
def setUp(self) -> None: super(EmbeddedWindowTests, self).setUp() config = get_j_env_configuration( self.env._j_stream_execution_environment) config.setString("python.execution-mode", "thread")