dianfu commented on code in PR #27198: URL: https://github.com/apache/flink/pull/27198#discussion_r2496779186
########## flink-python/pyflink/examples/datastream/asyncio/remote_model_inference.py: ########## @@ -0,0 +1,150 @@ +################################################################################ +# 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 argparse +import asyncio +import functools +import json +import logging +import random +import sys +from typing import List + +from pyflink.common import Encoder, Types, Time, Row +from pyflink.datastream import StreamExecutionEnvironment, AsyncDataStream, AsyncFunction, \ + RuntimeContext, AsyncRetryStrategy, async_retry_predicates, CheckpointingMode +from pyflink.datastream.connectors.file_system import (FileSink, OutputFileConfig, RollingPolicy) +from pyflink.table import StreamTableEnvironment, TableDescriptor, Schema, DataTypes + + +class AsyncLLMRequest(AsyncFunction[Row, str]): + + def __init__(self): + self.retried_keys = {} + + def open(self, runtime_context: RuntimeContext): + # create model inference client here + pass + + def close(self): + # close the model inference client here + pass + + async def async_invoke(self, value: Row) -> List[str]: + # issue the asynchronous request + await asyncio.sleep(random.randint(1, 2)) + + if value.user_id not in self.retried_keys and random.randint(1, 10) % 3 == 0: + self.retried_keys[value.user_id] = True + # remote model inference request may time out + raise TimeoutError + else: + if value.user_id in self.retried_keys: + del self.retried_keys[value.user_id] + # remote model inference request completes + # note that the result should be a collection even there is only one result + analysis_result = "positive" + result = { + "user_id": value.user_id, + "comments": value.comments, + "analysis_result": analysis_result + } + return [json.dumps(result)] + + def timeout(self, value: Row) -> List[str]: + # return a default value in case timeout + result = { + "user_id": value.user_id, + "comments": value.comments, + "analysis_result": None + } + return [json.dumps(result)] Review Comment: @bgeng777 Good catch! This is not expected. I think we should initialize self._delayed_retry_timer as None in the constructor. -- 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]
