fredia commented on code in PR #25853: URL: https://github.com/apache/flink/pull/25853#discussion_r1897727311
########## flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/declare/DeclarationChain.java: ########## @@ -0,0 +1,216 @@ +/* + * 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.runtime.asyncprocessing.declare; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.core.state.StateFutureUtils; +import org.apache.flink.util.function.FunctionWithException; +import org.apache.flink.util.function.ThrowingConsumer; + +import java.util.Deque; +import java.util.LinkedList; + +/** + * A declaration chain allows to declare multiple async operations in a single chain. + * + * @param <IN> The type of the input elements. + */ +@Experimental +public class DeclarationChain<IN> implements ThrowingConsumer<IN, Exception> { + + private final DeclarationContext context; + + private final Deque<Transformation<?, ?>> transformations; + + private DeclarationStage<?> currentStage; + + DeclarationChain(DeclarationContext context) { + this.context = context; + this.transformations = new LinkedList<>(); + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public void accept(IN in) throws Exception { + StateFuture future = StateFutureUtils.completedFuture(in); + for (Transformation trans : transformations) { + future = trans.apply(future); + } + } + + public DeclarationStage<IN> firstStage() throws DeclarationException { + if (currentStage != null) { + throw new DeclarationException( + "Diverged declaration. Please make sure you call firstStage() once."); + } + DeclarationStage<IN> declarationStage = new DeclarationStage<>(); + currentStage = declarationStage; + return declarationStage; + } + + Transformation<?, ?> getLastTransformation() { + return transformations.getLast(); + } + + /** + * A DeclarationStage is a single stage in a declaration chain. It allows a further chaining of + * operations. + * + * @param <T> The output of previous transformations. Will be the input type of further chained + * operation. + */ + public class DeclarationStage<T> { + + private boolean afterThen = false; + + private void preCheck() throws DeclarationException { + if (afterThen) { + throw new DeclarationException( + "Double thenCompose called for single declaration block."); + } + if (currentStage != this) { + throw new DeclarationException( + "Diverged declaration. Please make sure you are declaring on the last point."); + } + afterThen = true; + } + + public <U> DeclarationStage<U> thenCompose( + FunctionWithException<T, StateFuture<U>, Exception> action) + throws DeclarationException { + preCheck(); + DeclarationStage<U> next = new DeclarationStage<>(); + ComposeTransformation<T, U> trans = new ComposeTransformation<>(action, next); + transformations.add(trans); + currentStage = next; + getLastTransformation().declare(); + return next; + } + + public DeclarationStage<Void> thenAccept(ThrowingConsumer<T, Exception> action) + throws DeclarationException { + preCheck(); + DeclarationStage<Void> next = new DeclarationStage<>(); + AcceptTransformation<T> trans = new AcceptTransformation<>(action, next); + transformations.add(trans); + currentStage = next; + getLastTransformation().declare(); + return next; + } + + public DeclarationStage<T> withName(String name) throws DeclarationException { + getLastTransformation().withName(name); + return this; + } + + public DeclarationChain<IN> finish() throws DeclarationException { + preCheck(); + getLastTransformation().declare(); + return DeclarationChain.this; + } + } + + @Internal + interface Transformation<FROM, TO> { + StateFuture<TO> apply(StateFuture<FROM> upstream) throws Exception; + + void withName(String name) throws DeclarationException; + + void declare() throws DeclarationException; + } + + private abstract static class AbstractTransformation<FROM, TO> + implements Transformation<FROM, TO> { + + String name = null; + + @Override + public void withName(String newName) throws DeclarationException { + if (name != null) { + throw new DeclarationException("Double naming"); + } + name = newName; + declare(); + } + } + + private class ComposeTransformation<FROM, TO> extends AbstractTransformation<FROM, TO> { + + DeclarationStage<TO> to; Review Comment: Looks useless. ########## flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/operators/AsyncKeyedProcessOperator.java: ########## @@ -0,0 +1,220 @@ +/* + * 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.runtime.asyncprocessing.operators; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.runtime.asyncprocessing.declare.DeclarationContext; +import org.apache.flink.runtime.asyncprocessing.declare.DeclaredVariable; +import org.apache.flink.runtime.asyncprocessing.streaming.api.AsyncKeyedProcessFunction; +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.operators.InternalTimer; +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.Triggerable; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.OutputTag; +import org.apache.flink.util.function.ThrowingConsumer; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** A {@link StreamOperator} for executing {@link AsyncKeyedProcessFunction}. */ +@Internal +public class AsyncKeyedProcessOperator<K, IN, OUT> + extends AbstractAsyncStateUdfStreamOperator<OUT, AsyncKeyedProcessFunction<K, IN, OUT>> + implements OneInputStreamOperator<IN, OUT>, Triggerable<K, VoidNamespace> { + + private static final long serialVersionUID = 1L; + + // Shared timestamp variable for collector, context and onTimerContext. + private transient DeclaredVariable<Long> sharedTimestamp; + + private transient TimestampedCollectorWithDeclaredVariable<OUT> collector; + + private transient ContextImpl context; + + private transient OnTimerContextImpl onTimerContext; + + private transient DeclarationContext declarationContext; + + private transient ThrowingConsumer<IN, Exception> processor; + private transient ThrowingConsumer<Long, Exception> timerProcessor; + + public AsyncKeyedProcessOperator(AsyncKeyedProcessFunction<K, IN, OUT> function) { + super(function); + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public void open() throws Exception { + super.open(); + declarationContext = new DeclarationContext(getDeclarationManager()); + userFunction.open(declarationContext); + sharedTimestamp = + declarationContext.declareVariable( + LongSerializer.INSTANCE, + "_AsyncKeyedProcessOperator$sharedTimestamp", + null); + + collector = new TimestampedCollectorWithDeclaredVariable<>(output, sharedTimestamp); + + InternalTimerService<VoidNamespace> internalTimerService = + getInternalTimerService("user-timers", VoidNamespaceSerializer.INSTANCE, this); + + TimerService timerService = new SimpleTimerService(internalTimerService); + + context = new ContextImpl(userFunction, timerService, sharedTimestamp); + onTimerContext = + new OnTimerContextImpl( + userFunction, timerService, declarationContext, sharedTimestamp); + + processor = userFunction.declareProcess(declarationContext, context, collector); + timerProcessor = userFunction.declareOnTimer(declarationContext, onTimerContext, collector); + } + + @Override + public void onEventTime(InternalTimer<K, VoidNamespace> timer) throws Exception { + collector.setAbsoluteTimestamp(timer.getTimestamp()); + invokeUserFunction(TimeDomain.EVENT_TIME, timer); + } + + @Override + public void onProcessingTime(InternalTimer<K, VoidNamespace> timer) throws Exception { + collector.eraseTimestamp(); + invokeUserFunction(TimeDomain.PROCESSING_TIME, timer); + } + + @Override + public void processElement(StreamRecord<IN> element) throws Exception { + collector.setTimestamp(element); + processor.accept(element.getValue()); Review Comment: Do all other async operators have to use `declaring interface`? If they don't use `declaring interface`, can they work normally? ########## flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/declare/DeclarationChain.java: ########## @@ -0,0 +1,216 @@ +/* + * 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.runtime.asyncprocessing.declare; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.core.state.StateFutureUtils; +import org.apache.flink.util.function.FunctionWithException; +import org.apache.flink.util.function.ThrowingConsumer; + +import java.util.Deque; +import java.util.LinkedList; + +/** + * A declaration chain allows to declare multiple async operations in a single chain. + * + * @param <IN> The type of the input elements. + */ +@Experimental +public class DeclarationChain<IN> implements ThrowingConsumer<IN, Exception> { + + private final DeclarationContext context; + + private final Deque<Transformation<?, ?>> transformations; + + private DeclarationStage<?> currentStage; + + DeclarationChain(DeclarationContext context) { + this.context = context; + this.transformations = new LinkedList<>(); + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public void accept(IN in) throws Exception { + StateFuture future = StateFutureUtils.completedFuture(in); + for (Transformation trans : transformations) { + future = trans.apply(future); + } + } + + public DeclarationStage<IN> firstStage() throws DeclarationException { + if (currentStage != null) { + throw new DeclarationException( + "Diverged declaration. Please make sure you call firstStage() once."); + } + DeclarationStage<IN> declarationStage = new DeclarationStage<>(); + currentStage = declarationStage; + return declarationStage; + } + + Transformation<?, ?> getLastTransformation() { + return transformations.getLast(); + } + + /** + * A DeclarationStage is a single stage in a declaration chain. It allows a further chaining of + * operations. + * + * @param <T> The output of previous transformations. Will be the input type of further chained + * operation. + */ + public class DeclarationStage<T> { + + private boolean afterThen = false; + + private void preCheck() throws DeclarationException { + if (afterThen) { + throw new DeclarationException( + "Double thenCompose called for single declaration block."); + } + if (currentStage != this) { + throw new DeclarationException( + "Diverged declaration. Please make sure you are declaring on the last point."); + } + afterThen = true; + } + + public <U> DeclarationStage<U> thenCompose( + FunctionWithException<T, StateFuture<U>, Exception> action) + throws DeclarationException { + preCheck(); + DeclarationStage<U> next = new DeclarationStage<>(); + ComposeTransformation<T, U> trans = new ComposeTransformation<>(action, next); + transformations.add(trans); + currentStage = next; + getLastTransformation().declare(); + return next; + } + + public DeclarationStage<Void> thenAccept(ThrowingConsumer<T, Exception> action) + throws DeclarationException { + preCheck(); + DeclarationStage<Void> next = new DeclarationStage<>(); + AcceptTransformation<T> trans = new AcceptTransformation<>(action, next); + transformations.add(trans); + currentStage = next; + getLastTransformation().declare(); + return next; + } + + public DeclarationStage<T> withName(String name) throws DeclarationException { + getLastTransformation().withName(name); + return this; + } + + public DeclarationChain<IN> finish() throws DeclarationException { + preCheck(); + getLastTransformation().declare(); + return DeclarationChain.this; + } + } + + @Internal + interface Transformation<FROM, TO> { + StateFuture<TO> apply(StateFuture<FROM> upstream) throws Exception; + + void withName(String name) throws DeclarationException; + + void declare() throws DeclarationException; + } + + private abstract static class AbstractTransformation<FROM, TO> + implements Transformation<FROM, TO> { + + String name = null; + + @Override + public void withName(String newName) throws DeclarationException { + if (name != null) { + throw new DeclarationException("Double naming"); + } + name = newName; + declare(); + } + } + + private class ComposeTransformation<FROM, TO> extends AbstractTransformation<FROM, TO> { + + DeclarationStage<TO> to; + + FunctionWithException<FROM, StateFuture<TO>, ? extends Exception> action; + + NamedFunction<FROM, StateFuture<TO>> namedFunction; + + ComposeTransformation( + FunctionWithException<FROM, StateFuture<TO>, Exception> action, + DeclarationStage<TO> to) { + this.action = action; + this.to = to; + } + + @Override + public StateFuture<TO> apply(StateFuture<FROM> upstream) throws Exception { + return upstream.thenCompose(namedFunction); + } + + @Override + public void declare() throws DeclarationException { + if (namedFunction == null) { + if (name == null) { + namedFunction = context.declare(action); + } else { + namedFunction = context.declare(name, action); + } + } + } + } + + private class AcceptTransformation<FROM> extends AbstractTransformation<FROM, Void> { + + DeclarationStage<Void> to; Review Comment: Looks useless too. -- 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]
