pnowojski commented on a change in pull request #10009: [FLINK-14304] Avoid 
task starvation with mailbox
URL: https://github.com/apache/flink/pull/10009#discussion_r343536096
 
 

 ##########
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java
 ##########
 @@ -0,0 +1,146 @@
+/*
+ * 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.runtime.operators;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.MailboxExecutor;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.YieldingOperatorFactory;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
+import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+/**
+ * Test to verify that timer triggers are run according to operator precedence 
(combined with yield() at operator
+ * level).
+ */
+public class MailboxOperatorTest extends TestLogger {
+
+       @Test
+       public void testAvoidTaskStarvation() throws Exception {
+               final OneInputStreamTaskTestHarness<Integer, Integer> 
testHarness = new OneInputStreamTaskTestHarness<>(
+                       OneInputStreamTask::new,
+                       BasicTypeInfo.INT_TYPE_INFO,
+                       BasicTypeInfo.INT_TYPE_INFO);
+
+               testHarness.setupOperatorChain(new OperatorID(), new 
ReplicatingMailOperatorFactory())
+                       .chain(new OperatorID(), new 
ReplicatingMailOperatorFactory(), IntSerializer.INSTANCE)
+                       .finish();
+
+               testHarness.invoke();
+               testHarness.waitForTaskRunning();
+
+               testHarness.processElement(new StreamRecord<>(0));
+               testHarness.processElement(new StreamRecord<>(0));
+               testHarness.processElement(new StreamRecord<>(0));
+
+               testHarness.endInput();
+               testHarness.waitForTaskCompletion();
+
+               // with each input two mails should be processed, one of each 
operator in the chain
+               List<Integer> numMailsProcessed = 
testHarness.getOutput().stream()
+                       .map(element -> ((StreamRecord<Integer>) 
element).getValue())
+                       .collect(Collectors.toList());
+               assertThat(numMailsProcessed, is(Arrays.asList(0, 2, 4)));
+       }
+
+       private static class ReplicatingMailOperatorFactory implements 
OneInputStreamOperatorFactory<Integer, Integer>,
+                       YieldingOperatorFactory<Integer> {
+               private MailboxExecutor mailboxExecutor;
+
+               @Override
+               public void setMailboxExecutor(MailboxExecutor mailboxExecutor) 
{
+                       this.mailboxExecutor = mailboxExecutor;
+               }
+
+               @Override
+               public <Operator extends StreamOperator<Integer>> Operator 
createStreamOperator(
+                               StreamTask<?, ?> containingTask,
+                               StreamConfig config,
+                               Output<StreamRecord<Integer>> output) {
+                       ReplicatingMailOperator operator = new 
ReplicatingMailOperator(mailboxExecutor);
+                       operator.setup(containingTask, config, output);
+                       return (Operator) operator;
+               }
+
+               @Override
+               public void setChainingStrategy(ChainingStrategy strategy) {
+               }
+
+               @Override
+               public ChainingStrategy getChainingStrategy() {
+                       return ChainingStrategy.ALWAYS;
+               }
+
+               @Override
+               public Class<? extends StreamOperator> 
getStreamOperatorClass(ClassLoader classLoader) {
+                       return ReplicatingMailOperator.class;
+               }
+       }
+
+       private static class ReplicatingMailOperator extends 
AbstractStreamOperator<Integer>
+                       implements OneInputStreamOperator<Integer, Integer> {
+
+               private final AtomicInteger mailIndex = new AtomicInteger();
+               private final MailboxExecutor mailboxExecutor;
+
+               ReplicatingMailOperator(final MailboxExecutor mailboxExecutor) {
+                       this.mailboxExecutor = mailboxExecutor;
+               }
+
+               @Override
+               public void processElement(StreamRecord<Integer> element) 
throws Exception {
+                       // for the very first element, enqueue one mail that 
replicates itself
+                       if (mailIndex.get() == 0) {
+                               mailboxExecutor.execute(new Runnable() {
+                                       @Override
+                                       public void run() {
+                                               try {
+                                                       
mailboxExecutor.execute(this, "Blocking mail" + mailIndex.incrementAndGet());
+                                               } catch 
(RejectedExecutionException e) {
+                                                       // during shutdown the 
executor will reject new mails, which is fine for us.
+                                               }
+                                       }
+                               }, "Blocking mail" + mailIndex.get());
 
 Review comment:
   nitty nit: (here and in `testAvoidStarvation `) I think this  violates our 
[code 
style](https://flink.apache.org/contributing/code-style-and-quality-formatting.html#breaking-the-lines-of-too-long-statements).
 I think it should be:
   ```
                mailboxExecutor.execute(
                        new Runnable() {
                                @Override
                                public void run() {
                                        try {
                                                mailboxExecutor.execute(this, 
"Blocking mail" + mailIndex.incrementAndGet());
                                        } catch (RejectedExecutionException e) {
                                                // during shutdown the executor 
will reject new mails, which is fine for us.
                                        }
                                }
                        },
                        "Blocking mail" + mailIndex.get());
   ```
   
   Alternatively, name the function instead of runnable? Or add a util function 
to ignore some specified type of the exception in 
`org.apache.flink.util.function.FunctionUtils`?

----------------------------------------------------------------
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]


With regards,
Apache Git Services

Reply via email to