HuangXingBo commented on a change in pull request #15149: URL: https://github.com/apache/flink/pull/15149#discussion_r597380205
########## File path: flink-python/src/main/java/org/apache/flink/streaming/api/utils/output/TimerOperandType.java ########## @@ -0,0 +1,33 @@ +/* + * 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. + */ + +package org.apache.flink.streaming.api.utils.output; + +/** The Flag for indicating the timer operation type. */ +public enum TimerOperandType { Review comment: Maybe we need to add a `valueOf` method to index to the corresponding `TimerOperandType` according to the `value`(byte param). ########## File path: flink-python/src/main/java/org/apache/flink/streaming/api/utils/output/OutputWithTimerRowHandler.java ########## @@ -0,0 +1,94 @@ +/* + * 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. + */ + +package org.apache.flink.streaming.api.utils.output; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.streaming.api.TimerService; +import org.apache.flink.streaming.api.operators.KeyContext; +import org.apache.flink.streaming.api.operators.TimestampedCollector; +import org.apache.flink.types.Row; + +/** This handler can accepts the runner output which contains timer registration event. */ +public class OutputWithTimerRowHandler { Review comment: change to `OutputWithTimerRowHandler<OUT>` ? ########## File path: flink-python/src/main/java/org/apache/flink/streaming/api/utils/input/TwoInputWithTimerRowFactory.java ########## @@ -0,0 +1,96 @@ +/* + * 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. + */ + +package org.apache.flink.streaming.api.utils.input; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.TimeDomain; +import org.apache.flink.types.Row; + +import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.KeyedProcessFunctionInputFlag.EVENT_TIME_TIMER; Review comment: I think we can redesign this inputFlag. We can design a structure of two layers flag which includes normal_data flag and timer flag. The first layer flag indicates whether it is normal data or timer data. The second layer flag indicates which type of timer it is. ########## File path: flink-python/src/main/java/org/apache/flink/streaming/api/utils/output/TimerOperandType.java ########## @@ -0,0 +1,33 @@ +/* + * 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. + */ + +package org.apache.flink.streaming.api.utils.output; + +/** The Flag for indicating the timer operation type. */ +public enum TimerOperandType { + REGISTER_EVENT_TIMER((byte) 0), + REGISTER_PROC_TIMER((byte) 1), + DELETE_EVENT_TIMER((byte) 2), + DELETE_PROC_TIMER((byte) 3); + + public final byte value; Review comment: ```suggestion private final byte value; ``` ########## File path: flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java ########## @@ -0,0 +1,222 @@ +/* + * 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. + */ + +package org.apache.flink.streaming.api.operators.python; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.memory.ManagedMemoryUseCase; +import org.apache.flink.python.PythonFunctionRunner; +import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.runtime.state.VoidNamespaceSerializer; +import org.apache.flink.streaming.api.SimpleTimerService; +import org.apache.flink.streaming.api.TimeDomain; +import org.apache.flink.streaming.api.TimerService; +import org.apache.flink.streaming.api.functions.python.DataStreamPythonFunctionInfo; +import org.apache.flink.streaming.api.operators.InternalTimer; +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.streaming.api.operators.TimestampedCollector; +import org.apache.flink.streaming.api.operators.Triggerable; +import org.apache.flink.streaming.api.runners.python.beam.BeamDataStreamPythonFunctionRunner; +import org.apache.flink.streaming.api.utils.PythonOperatorUtils; +import org.apache.flink.streaming.api.utils.PythonTypeUtils; +import org.apache.flink.streaming.api.utils.input.TwoInputWithTimerRowFactory; +import org.apache.flink.streaming.api.utils.output.OutputWithTimerRowHandler; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.types.Row; + +import java.util.Collections; + +/** KeyedCoProcessOperator. */ +public class PythonKeyedCoProcessOperator<OUT> + extends TwoInputPythonFunctionOperator<Row, Row, Row, OUT> + implements ResultTypeQueryable<OUT>, Triggerable<Row, VoidNamespace> { + + private static final String KEYED_CO_PROCESS_FUNCTION_URN = + "flink:transform:keyed_process_function:v1"; + + private static final String FLAT_MAP_CODER_URN = "flink:coder:flat_map:v1"; + + /** The TypeInformation of current key. */ + private final TypeInformation<Row> keyTypeInfo; + + /** Serializer for current key. */ + private final TypeSerializer keyTypeSerializer; + + private final TypeInformation<OUT> outputTypeInfo; + + /** TimerService for current operator to register or fire timer. */ + private transient TimerService timerService; + + private transient TwoInputWithTimerRowFactory runnerInputFactory; + private transient OutputWithTimerRowHandler runnerOutputHandler; + + public PythonKeyedCoProcessOperator( + Configuration config, + TypeInformation<Row> inputTypeInfo1, + TypeInformation<Row> inputTypeInfo2, + TypeInformation<OUT> outputTypeInfo, + DataStreamPythonFunctionInfo pythonFunctionInfo) { + super( + config, + pythonFunctionInfo, + FLAT_MAP_CODER_URN, + TwoInputWithTimerRowFactory.getRunnerInputTypeInfo( + inputTypeInfo1, inputTypeInfo2, constructKeyTypeInfo(inputTypeInfo1)), + OutputWithTimerRowHandler.getRunnerOutputTypeInfo( + outputTypeInfo, constructKeyTypeInfo(inputTypeInfo1))); + this.keyTypeInfo = constructKeyTypeInfo(inputTypeInfo1); + this.keyTypeSerializer = + PythonTypeUtils.TypeInfoToSerializerConverter.typeInfoSerializerConverter( + keyTypeInfo); + this.outputTypeInfo = outputTypeInfo; + } + + @Override + public PythonFunctionRunner createPythonFunctionRunner() throws Exception { + return new BeamDataStreamPythonFunctionRunner( + getRuntimeContext().getTaskName(), + createPythonEnvironmentManager(), + getRunnerInputTypeInfo(), + getRunnerOutputTypeInfo(), + KEYED_CO_PROCESS_FUNCTION_URN, + PythonOperatorUtils.getUserDefinedDataStreamStatefulFunctionProto( + getPythonFunctionInfo(), + getRuntimeContext(), + Collections.EMPTY_MAP, + keyTypeInfo), + getCoderUrn(), + getJobOptions(), + getFlinkMetricContainer(), + getKeyedStateBackend(), + keyTypeSerializer, + getContainingTask().getEnvironment().getMemoryManager(), + getOperatorConfig() + .getManagedMemoryFractionOperatorUseCaseOfSlot( + ManagedMemoryUseCase.PYTHON, + getContainingTask() + .getEnvironment() + .getTaskManagerInfo() + .getConfiguration(), + getContainingTask() + .getEnvironment() + .getUserCodeClassLoader() + .asClassLoader())); + } + + @Override + public void open() throws Exception { + InternalTimerService<VoidNamespace> internalTimerService = + getInternalTimerService("user-timers", VoidNamespaceSerializer.INSTANCE, this); + timerService = new SimpleTimerService(internalTimerService); + this.runnerInputFactory = new TwoInputWithTimerRowFactory(); + this.runnerOutputHandler = + new OutputWithTimerRowHandler( + getKeyedStateBackend(), timerService, new TimestampedCollector<>(output)); + super.open(); + } + + @Override + public void processElement1(StreamRecord<Row> element) throws Exception { + processElement(true, element); + } + + @Override + public void processElement2(StreamRecord<Row> element) throws Exception { + processElement(false, element); + } + + private void processElement(boolean isLeft, StreamRecord<Row> element) throws Exception { Review comment: I think we need to adjust the position of `processElement` and put it in front of the `processTimer` method -- 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. For queries about this service, please contact Infrastructure at: [email protected]
