flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [flink] pnowojski commented on a change in pull request #8124: [FLINK-11877] Implement the runtime handling of the InputSelectable interface
Date Wed, 17 Apr 2019 15:19:08 GMT
pnowojski commented on a change in pull request #8124: [FLINK-11877] Implement the runtime
handling of the InputSelectable interface
URL: https://github.com/apache/flink/pull/8124#discussion_r276291942
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputSelectableProcessor.java
 ##########
 @@ -0,0 +1,434 @@
+/*
+ * 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.io;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.SimpleCounter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Input reader for {@link org.apache.flink.streaming.runtime.tasks.TwoInputSelectableStreamTask}
+ * in the case that the operator is InputSelectable.
+ *
+ * @param <IN1> The type of the records that arrive on the first input
+ * @param <IN2> The type of the records that arrive on the second input
+ */
+@Internal
+public class StreamTwoInputSelectableProcessor<IN1, IN2> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(StreamTwoInputSelectableProcessor.class);
+
+	private volatile boolean continuousProcessing = true;
+
+	private final NetworkInput input1;
+	private final NetworkInput input2;
+
+	private final Object lock;
+
+	private final TwoInputStreamOperator<IN1, IN2, ?> streamOperator;
+
+	private final InputSelectable inputSelector;
+
+	private final AuxiliaryHandler auxiliaryHandler;
+
+	private final CompletableFuture<Integer>[] listenFutures;
+
+	private final boolean[] isFinished;
+
+	private InputSelection inputSelection;
+
+	private AtomicInteger availableInputsMask = new AtomicInteger();
 
 Review comment:
   **(hot looping without synchronisation part 2)** I would drop `AtomicInteger availableInputsMask`
and replace it with `integer availableIntpusMask` which is updated based on `BufferOrEvent#moreAvailable()`
mentioned above.
   
   As long as all of the selected inputs are available, there is no need to synchronise. If
some selected input is not available, that's the point where we could use `AtomicInteger`,
but for `TwoInputStream` (and not general N input) case more or less equally efficient would
be to check `listenFutures[x].getNow(-1)`. This is also a single volatile access under the
hood, with the same performance penalty as `availableInputsMask.get()`.
   
   But again, keep in mind that while hot looping you can not touch `listenFutures`, as they
are `volatiles` thus another synchronisation points.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message