xintongsong commented on code in PR #48:
URL: https://github.com/apache/flink-agents/pull/48#discussion_r2188060756
##########
python/flink_agents/api/execution_enviroment.py:
##########
@@ -54,7 +76,21 @@ def from_list(self, input: List[Dict[str, Any]]) ->
'AgentsExecutionEnvironment'
"""
@abstractmethod
- def apply(self, workflow: Workflow) -> 'AgentsExecutionEnvironment':
+ def from_datastream(
Review Comment:
The current api would not work in the following use case:
```
env1 = env.from_datastream(ds1).apply(workflow1)
env2 = env.from_datastream(ds2).apply(workflow2)
output1 = env1.to_datastream()
output2 = env2.to_datastream()
```
I think we would need to return some sort of `AgentBuilder` for
`from_datastream/list()`, and make `apply()` and `to_datastream/list()` methods
of the builder.
##########
python/flink_agents/runtime/remote_execution_environment.py:
##########
@@ -0,0 +1,165 @@
+################################################################################
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#################################################################################
+import uuid
+from typing import Any, Dict, Generator, List
+
+import cloudpickle
+from pyflink.common import Row
+from pyflink.common.typeinfo import PickledBytesTypeInfo, RowTypeInfo
+from pyflink.datastream import (
+ DataStream,
+ KeyedProcessFunction,
+ KeyedStream,
+ KeySelector,
+ StreamExecutionEnvironment,
+)
+from pyflink.util.java_utils import invoke_method
+
+from flink_agents.api.event import InputEvent
+from flink_agents.api.execution_enviroment import AgentsExecutionEnvironment
+from flink_agents.api.workflow import Workflow
+from flink_agents.plan.workflow_plan import WorkflowPlan
+
+
+class MapKeyedProcessFunctionAdapter(KeyedProcessFunction):
+ """Util class for converting element in KeyedStream to Row."""
+
+ def process_element(
+ self, value: Any, ctx: "KeyedProcessFunction.Context"
+ ) -> Generator:
+ """Convert element to Row contains key."""
+ ctx.get_current_key()
+ yield Row(ctx.get_current_key(), InputEvent(input=value))
+
+
+class RemoteExecutionEnvironment(AgentsExecutionEnvironment):
+ """Implementation of AgentsExecutionEnvironment for execution with
DataStream."""
+
+ __env: StreamExecutionEnvironment
+ __input: DataStream
+ __workflow_plan: WorkflowPlan
+ __output: DataStream
+
+ def from_datastream(
+ self, input: DataStream, key_selector: KeySelector = None
+ ) -> "AgentsExecutionEnvironment":
+ """Set input for agents.
+
+ Parameters
+ ----------
+ input : DataStream
+ Receive a DataStream as input.
+ key_selector : KeySelector
+ Extract key from each input record.
+ """
+ if isinstance(input, KeyedStream):
+ self.__input = input.process(
+ MapKeyedProcessFunctionAdapter(),
+ output_type=RowTypeInfo(
+ [PickledBytesTypeInfo(), PickledBytesTypeInfo()]
+ ),
+ )
+ else:
+ self.__input = input.map(
+ lambda x: Row(
+ key_selector.get_key(x)
+ if key_selector is not None
+ else uuid.uuid4(),
+ InputEvent(input=x),
+ ),
+ output_type=RowTypeInfo(
+ [PickledBytesTypeInfo(), PickledBytesTypeInfo()]
+ ),
+ )
+ return self
+
+ def apply(self, workflow: Workflow) -> "AgentsExecutionEnvironment":
+ """Set workflow of execution environment.
+
+ Parameters
+ ----------
+ workflow : Workflow
+ The workflow user defined to run in execution environment.
+ """
+ self.__workflow_plan = WorkflowPlan.from_workflow(workflow)
+ return self
+
+ def to_datastream(self) -> DataStream:
+ """Get outputs of workflow execution. Used for remote execution.
+
+ Returns:
+ -------
+ DataStream
+ Outputs of workflow execution.
+ """
+ j_data_stream_output = invoke_method(
+ None,
+ "org.apache.flink.agents.runtime.FlinkAgent",
+ "connectToWorkflow",
+ [
+ self.__input._j_data_stream,
+ self.__workflow_plan.model_dump_json(serialize_as_any=True),
+ ],
+ [
+ "org.apache.flink.streaming.api.datastream.DataStream",
+ "java.lang.String",
+ ],
+ )
+ output_stream = DataStream(j_data_stream_output)
+ self.__output = output_stream.map(lambda x:
cloudpickle.loads(x).output)
+ return self.__output
+
+ def execute(self) -> None:
+ """Execute agents.
+
+ This method is not supported for remote execution environments.
+ """
+ msg = "RemoteExecutionEnvironment does not support execute locally."
+ raise NotImplementedError(msg)
+
+ def from_list(self, input: List[Dict[str, Any]]) ->
"AgentsExecutionEnvironment":
+ """Set input list of workflow execution.
+
+ This method is not supported for remote execution environments.
+ """
+ msg = "RemoteExecutionEnvironment does not support from_list."
+ raise NotImplementedError(msg)
+
+ def to_list(self) -> List[Dict[str, Any]]:
+ """Get output list of workflow execution.
+
+ This method is not supported for remote execution environments.
+ """
+ msg = "RemoteExecutionEnvironment does not support to_list."
+ raise NotImplementedError(msg)
+
+
+def get_execution_environment(**kwargs: Dict[str, Any]) ->
AgentsExecutionEnvironment:
Review Comment:
Might be better to use a name different from it's supper class. Maybe
`create_instance()`?
##########
python/flink_agents/runtime/remote_execution_environment.py:
##########
@@ -0,0 +1,165 @@
+################################################################################
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#################################################################################
+import uuid
+from typing import Any, Dict, Generator, List
+
+import cloudpickle
+from pyflink.common import Row
+from pyflink.common.typeinfo import PickledBytesTypeInfo, RowTypeInfo
+from pyflink.datastream import (
+ DataStream,
+ KeyedProcessFunction,
+ KeyedStream,
+ KeySelector,
+ StreamExecutionEnvironment,
+)
+from pyflink.util.java_utils import invoke_method
+
+from flink_agents.api.event import InputEvent
+from flink_agents.api.execution_enviroment import AgentsExecutionEnvironment
+from flink_agents.api.workflow import Workflow
+from flink_agents.plan.workflow_plan import WorkflowPlan
+
+
+class MapKeyedProcessFunctionAdapter(KeyedProcessFunction):
+ """Util class for converting element in KeyedStream to Row."""
+
+ def process_element(
+ self, value: Any, ctx: "KeyedProcessFunction.Context"
+ ) -> Generator:
+ """Convert element to Row contains key."""
+ ctx.get_current_key()
+ yield Row(ctx.get_current_key(), InputEvent(input=value))
+
+
+class RemoteExecutionEnvironment(AgentsExecutionEnvironment):
+ """Implementation of AgentsExecutionEnvironment for execution with
DataStream."""
+
+ __env: StreamExecutionEnvironment
+ __input: DataStream
+ __workflow_plan: WorkflowPlan
+ __output: DataStream
+
+ def from_datastream(
+ self, input: DataStream, key_selector: KeySelector = None
+ ) -> "AgentsExecutionEnvironment":
+ """Set input for agents.
+
+ Parameters
+ ----------
+ input : DataStream
+ Receive a DataStream as input.
+ key_selector : KeySelector
+ Extract key from each input record.
+ """
+ if isinstance(input, KeyedStream):
+ self.__input = input.process(
+ MapKeyedProcessFunctionAdapter(),
+ output_type=RowTypeInfo(
+ [PickledBytesTypeInfo(), PickledBytesTypeInfo()]
+ ),
+ )
+ else:
+ self.__input = input.map(
+ lambda x: Row(
+ key_selector.get_key(x)
+ if key_selector is not None
+ else uuid.uuid4(),
+ InputEvent(input=x),
Review Comment:
I think it's the operator's responsibility to wrap the input into an
InputEvent.
##########
python/flink_agents/api/execution_enviroment.py:
##########
@@ -19,14 +19,31 @@
from abc import ABC, abstractmethod
from typing import Any, Dict, List
+from pyflink.datastream import DataStream, KeySelector,
StreamExecutionEnvironment
+
from flink_agents.api.workflow import Workflow
class AgentsExecutionEnvironment(ABC):
"""Base class for workflow execution environment."""
- @staticmethod
- def get_execution_environment(**kwargs: Dict[str, Any]) ->
'AgentsExecutionEnvironment':
+ __env: StreamExecutionEnvironment = None
+
+ @classmethod
+ def from_env(
+ cls, env: StreamExecutionEnvironment
+ ) -> type["AgentsExecutionEnvironment"]:
+ """Set StreamExecutionEnvironment of AgentsExecutionEnvironment.
+
+ Currently, this property is only used for distinguishing execution
environment.
+ """
+ cls.__env = env
+ return cls
Review Comment:
Having a method that modifies the class property doesn't make sense. Why not
adding `env` as an argument of `get_execution_environment`?
##########
python/flink_agents/api/execution_enviroment.py:
##########
@@ -75,6 +111,16 @@ def to_list(self) -> List[Dict[str, Any]]:
Outputs of workflow execution.
"""
+ @abstractmethod
+ def to_datastream(self) -> DataStream:
Review Comment:
The commit message mentioned both datastream and table, but it seems only
datastream is supported.
##########
python/flink_agents/runtime/remote_execution_environment.py:
##########
@@ -0,0 +1,165 @@
+################################################################################
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#################################################################################
+import uuid
+from typing import Any, Dict, Generator, List
+
+import cloudpickle
+from pyflink.common import Row
+from pyflink.common.typeinfo import PickledBytesTypeInfo, RowTypeInfo
+from pyflink.datastream import (
+ DataStream,
+ KeyedProcessFunction,
+ KeyedStream,
+ KeySelector,
+ StreamExecutionEnvironment,
+)
+from pyflink.util.java_utils import invoke_method
+
+from flink_agents.api.event import InputEvent
+from flink_agents.api.execution_enviroment import AgentsExecutionEnvironment
+from flink_agents.api.workflow import Workflow
+from flink_agents.plan.workflow_plan import WorkflowPlan
+
+
+class MapKeyedProcessFunctionAdapter(KeyedProcessFunction):
+ """Util class for converting element in KeyedStream to Row."""
+
+ def process_element(
+ self, value: Any, ctx: "KeyedProcessFunction.Context"
+ ) -> Generator:
+ """Convert element to Row contains key."""
+ ctx.get_current_key()
+ yield Row(ctx.get_current_key(), InputEvent(input=value))
+
+
+class RemoteExecutionEnvironment(AgentsExecutionEnvironment):
+ """Implementation of AgentsExecutionEnvironment for execution with
DataStream."""
+
+ __env: StreamExecutionEnvironment
+ __input: DataStream
+ __workflow_plan: WorkflowPlan
+ __output: DataStream
+
+ def from_datastream(
+ self, input: DataStream, key_selector: KeySelector = None
+ ) -> "AgentsExecutionEnvironment":
+ """Set input for agents.
+
+ Parameters
+ ----------
+ input : DataStream
+ Receive a DataStream as input.
+ key_selector : KeySelector
+ Extract key from each input record.
+ """
+ if isinstance(input, KeyedStream):
+ self.__input = input.process(
+ MapKeyedProcessFunctionAdapter(),
+ output_type=RowTypeInfo(
+ [PickledBytesTypeInfo(), PickledBytesTypeInfo()]
+ ),
+ )
+ else:
+ self.__input = input.map(
+ lambda x: Row(
+ key_selector.get_key(x)
+ if key_selector is not None
+ else uuid.uuid4(),
Review Comment:
Not sure if auto generating a uuid is what the user wants. Maybe we can
start with always requiring a key selector, and users can explicitly set the
key_selector to generate a uuid?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]