You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/02/14 14:18:04 UTC

[GitHub] [flink] pnowojski opened a new pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

pnowojski opened a new pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098
 
 
   This PR adds a basic support for multiple input operators and relevant interfaces.
   
   This doesn't fully support input selection, watermarks, latency markers and keyed context yet.
   
   ## Verifying this change
   
   This change adds couple of new test classes, most importantly `MultipleInputStreamTaskTest`
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (**yes** / no)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (**yes** / no / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (**yes** / no)
     - If yes, how is the feature documented? (not applicable / docs / **JavaDocs** / not documented)
   

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383187721
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
+					i));
+		}
+
+		this.operatorChain = checkNotNull(operatorChain);
+	}
+
+	@Override
+	public CompletableFuture<?> getAvailableFuture() {
+		if (inputSelectionHandler.areAllInputsSelected()) {
+			return isAnyInputAvailable();
+		} else {
+			throw new UnsupportedOperationException();
+		}
 
 Review comment:
   I haven't yet tested it with the input selection and I'm not sure if the `isAnyInputAvailable` properly supports inputs selection. 
   
   Hmm.. it looks like it might be just missing a single if check, but I'm not sure.

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383858836
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
 ##########
 @@ -73,8 +73,8 @@
 	private static final String ITERATION_ID = "iterationId";
 	private static final String OUTPUT_SELECTOR_WRAPPER = "outputSelectorWrapper";
 	private static final String BUFFER_TIMEOUT = "bufferTimeout";
-	private static final String TYPE_SERIALIZER_IN_1 = "typeSerializer_in_1";
-	private static final String TYPE_SERIALIZER_IN_2 = "typeSerializer_in_2";
+	private static final String TYPE_SERIALIZERS_IN_COUNT = "typeSerializer_in_count";
+	private static final String TYPE_SERIALIZERS_IN_PATTERN = "typeSerializer_in_%d";
 
 Review comment:
   String.format is quite slow, but I guess it's okay for setup code.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ac7b323867f647d50b170c23df6c6052b2c08d61 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150350672) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383859558
 
 

 ##########
 File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
 ##########
 @@ -633,4 +671,29 @@ protected boolean matchesSafely(ResourceSpec item) {
 			return resources.lessThanOrEqual(item) && item.lessThanOrEqual(resources);
 		}
 	}
+
+	private static class MultipleInputOperatorFactory implements StreamOperatorFactory<String> {
 
 Review comment:
   Good question. There has to be some factory for it.

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380549321
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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.transformations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This Transformation represents the application of a
+ * {@link org.apache.flink.streaming.api.operators.MultipleInputStreamOperator}
+ * to input {@code Transformations}. The result is again only one stream.
+ *
+ * @param <OUT> The type of the elements that result from this {@code MultipleInputTransformation}
+ */
+@Internal
+public class MultipleInputTransformation<OUT> extends PhysicalTransformation<OUT> {
+
+	private final ArrayList<Transformation<?>> inputs = new ArrayList<>();
+
+	private final StreamOperatorFactory<OUT> operatorFactory;
+
+	private final ArrayList<KeySelector<?, ?>> stateKeySelectors = new ArrayList<>();
+
+	private TypeInformation<?> stateKeyType;
+
+	public MultipleInputTransformation(
+			String name,
+			StreamOperatorFactory<OUT> operatorFactory,
+			TypeInformation<OUT> outputType,
+			int parallelism) {
+		super(name, outputType, parallelism);
+		this.operatorFactory = operatorFactory;
+	}
+
+	public List<Transformation<?>> getInputs() {
+		return inputs;
+	}
+
+	/**
+	 * Returns the {@code TypeInformation} for the elements from the inputs.
+	 */
+	public List<TypeInformation<?>> getInputTypes() {
+		return inputs.stream()
+			.map(Transformation::getOutputType)
+			.collect(Collectors.toList());
+	}
+
+	/**
+	 * Returns the {@code StreamOperatorFactory} of this Transformation.
+	 */
+	public StreamOperatorFactory<OUT> getOperatorFactory() {
+		return operatorFactory;
+	}
+
+	public void addInput(Transformation<?> input) {
+		checkState(
+			stateKeySelectors.isEmpty(),
+			"Trying to add non-keyed input to keyed transformation.");
 
 Review comment:
   If I understood correctly, there are two use-cases: with `stateKeySelectors` and without.
   I think it's better to have separate classes for these, or at least document these cases and invariants (on class or field level).

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383184155
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
 ##########
 @@ -235,20 +234,16 @@ public void addOutputSelector(OutputSelector<?> outputSelector) {
 		this.outputSelectors.add(outputSelector);
 	}
 
-	public TypeSerializer<?> getTypeSerializerIn1() {
-		return typeSerializerIn1;
+	public void setSerializersIn(TypeSerializer<?> ...typeSerializersIn) {
 
 Review comment:
   Empty needed for sources, I guess.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383350599
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
 
 Review comment:
   Good catch. Currently this is a dead code (watermarks not yet supported). I think this has a dedicated test in `TwoInputStreamTask` version, which I wanted to port with a follow up PR adding support for watermarks.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383387663
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
 ##########
 @@ -73,8 +73,8 @@
 	private static final String ITERATION_ID = "iterationId";
 	private static final String OUTPUT_SELECTOR_WRAPPER = "outputSelectorWrapper";
 	private static final String BUFFER_TIMEOUT = "bufferTimeout";
-	private static final String TYPE_SERIALIZER_IN_1 = "typeSerializer_in_1";
-	private static final String TYPE_SERIALIZER_IN_2 = "typeSerializer_in_2";
+	private static final String TYPE_SERIALIZERS_IN_COUNT = "typeSerializer_in_count";
+	private static final String TYPE_SERIALIZERS_IN_PATTERN = "typeSerializer_in_%d";
 
 Review comment:
   but with `%d` it's a pattern now, not a prefix, so the name fits. Or do you mean to change it to prefix (by dropping the `%d` part?)? If the latter, I think `pattern` > `prefix`.

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380553880
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
 ##########
 @@ -581,6 +611,13 @@ public void setTwoInputStateKey(Integer vertexID, KeySelector<?, ?> keySelector1
 		node.setStateKeySerializer(keySerializer);
 	}
 
+	public void setMultipleInputStateKey(
+			int id,
+			List<KeySelector<?, ?>> stateKeySelectors,
+			TypeSerializer<?> keySerializer) {
+		throw new UnsupportedOperationException();
+	}
 
 Review comment:
   Could you please explain the purpose of this 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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383203915
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
 
 Review comment:
   Should probably be -1 if we start at 0 as always.

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380755077
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ##########
 @@ -484,11 +485,15 @@ public final void invoke() throws Exception {
 		}
 	}
 
+	protected boolean runMailboxStep() throws Exception {
+		return mailboxProcessor.runMailboxStep();
+	}
+
 
 Review comment:
   If this method is only for tests, should we place it into `TestStreamTask` which `extends StreamTask`?
   If not:
   1. should we re-use it in `runMailboxLoop` below?
   2. make it package-private?

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383204891
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
+					i));
+		}
+
+		this.operatorChain = checkNotNull(operatorChain);
+	}
+
+	@Override
+	public CompletableFuture<?> getAvailableFuture() {
+		if (inputSelectionHandler.areAllInputsSelected()) {
+			return isAnyInputAvailable();
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	@Override
+	public InputStatus processInput() throws Exception {
+		int readingInputIndex;
+		if (isPrepared) {
+			readingInputIndex = selectNextReadingInputIndex();
+		} else {
+			// the preparations here are not placed in the constructor because all work in it
+			// must be executed after all operators are opened.
+			readingInputIndex = selectFirstReadingInputIndex();
 
 Review comment:
   Couldn't we introduce some open/init for this case?

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384335505
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ##########
 @@ -484,11 +485,15 @@ public final void invoke() throws Exception {
 		}
 	}
 
+	protected boolean runMailboxStep() throws Exception {
+		return mailboxProcessor.runMailboxStep();
+	}
+
 
 Review comment:
   I don't see how your proposal should work @rkhachatryan ? After passing to the test harness
   ```
   val streamTask = new TestStreamTask<>(new MultipleInputStreamTask<>(env))
   ```
   every time test harness would access the `streamTask`, it would be accessing `TestStreamTask`, not `MultipleInputStreamTask`, unless you would override/delegate  all of the methods, which would be an unstable overkill. Test harness would have to access the `delegate` reference for everything except of `runMailboxStep()`, which I would say is basically a less convoluted equivalent of having exposed `@VisibleForTesting runMailboxStep()` method in the production code.
   
   I think better solution would be to inject, mailbox into the `StreamTask`, but that would require to create `[Source|TwoInput|MultipleInput]StreamTaskBuilder` classes (constructors of `StreamTask` are already messy), and it's also on an overkill for this (imo) small problem. I think I would have done, if not for the fact how much refactoring and clean up I'm already doing here and will be doing more in other places (`AbstractStreamOperator` :()

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383180828
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
 ##########
 @@ -590,17 +626,30 @@ public void setBufferTimeout(Integer vertexID, long bufferTimeout) {
 
 	public void setSerializers(Integer vertexID, TypeSerializer<?> in1, TypeSerializer<?> in2, TypeSerializer<?> out) {
 		StreamNode vertex = getStreamNode(vertexID);
-		vertex.setSerializerIn1(in1);
-		vertex.setSerializerIn2(in2);
+		vertex.setSerializersIn(in1, in2);
+		vertex.setSerializerOut(out);
+	}
+
+	private <OUT> void setSerializers(
+			Integer vertexID,
+			List<TypeInformation<?>> inTypeInfos,
+			TypeSerializer<OUT> out) {
+
+		StreamNode vertex = getStreamNode(vertexID);
+
+		vertex.setSerializersIn(
+			inTypeInfos.stream()
+				.map(typeInfo -> typeInfo.createSerializer(executionConfig))
+				.toArray(size -> new TypeSerializer<?>[size]));
 
 Review comment:
   TypeSerializer[]::new not working?

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384335505
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ##########
 @@ -484,11 +485,15 @@ public final void invoke() throws Exception {
 		}
 	}
 
+	protected boolean runMailboxStep() throws Exception {
+		return mailboxProcessor.runMailboxStep();
+	}
+
 
 Review comment:
   I don't see how your proposal should work @rkhachatryan ? After passing to the test harness
   ```
   val streamTask = new TestStreamTask<>(new MultipleInputStreamTask<>(env))
   ```
   every time test harness would access the `streamTask`, it would be accessing `TestStreamTask`, not `MultipleInputStreamTask`, unless you would override/delegate  all of the methods, which would be an unstable overkill.
   
   I think better solution would be to inject, mailbox into the `StreamTask`, but that would require to create `[Source|TwoInput|MultipleInput]StreamTaskBuilder` classes (constructors of `StreamTask` are already messy), and it's also on an overkill for this (imo) small problem. I think I would have done, if not for the fact how much refactoring and clean up I'm already doing here and will be doing more in other places (`AbstractStreamOperator` :()

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383186214
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
 ##########
 @@ -28,7 +28,7 @@
 import org.apache.flink.runtime.util.ConfigurationParserUtils;
 import org.apache.flink.streaming.api.CheckpointingMode;
 
-import static org.apache.flink.util.Preconditions.checkState;
 
 Review comment:
   Imho not a hotfix.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383734977
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
+					i));
+		}
+
+		this.operatorChain = checkNotNull(operatorChain);
+	}
+
+	@Override
+	public CompletableFuture<?> getAvailableFuture() {
+		if (inputSelectionHandler.areAllInputsSelected()) {
+			return isAnyInputAvailable();
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	@Override
+	public InputStatus processInput() throws Exception {
+		int readingInputIndex;
+		if (isPrepared) {
+			readingInputIndex = selectNextReadingInputIndex();
+		} else {
+			// the preparations here are not placed in the constructor because all work in it
+			// must be executed after all operators are opened.
+			readingInputIndex = selectFirstReadingInputIndex();
+		}
+		if (readingInputIndex == -1) {
+			return InputStatus.NOTHING_AVAILABLE;
+		}
+
+		lastReadInputIndex = readingInputIndex;
+		InputStatus inputStatus = inputProcessors[readingInputIndex].processInput();
+		checkFinished(inputStatus, readingInputIndex);
+		return inputSelectionHandler.reportInputStatus(inputStatus, readingInputIndex);
+	}
+
+	private int selectFirstReadingInputIndex() {
+		// Note: the first call to nextSelection () on the operator must be made after this operator
+		// is opened to ensure that any changes about the input selection in its open()
+		// method take effect.
+		inputSelectionHandler.nextSelection();
+
+		isPrepared = true;
+
+		return selectNextReadingInputIndex();
+	}
+
+	private void checkFinished(InputStatus status, int inputIndex) throws Exception {
+		if (status == InputStatus.END_OF_INPUT) {
+			operatorChain.endHeadOperatorInput(getInputId(inputIndex));
+			inputSelectionHandler.nextSelection();
+		}
+	}
+
+	@Override
+	public void close() throws IOException {
+		IOException ex = null;
+		for (InputProcessor<?> input : inputProcessors) {
+			try {
+				input.close();
+			} catch (IOException e) {
+				ex = ExceptionUtils.firstOrSuppressed(e, ex);
+			}
+		}
+
+		if (ex != null) {
+			throw ex;
+		}
+	}
+
+	private int selectNextReadingInputIndex() {
+		if (!inputSelectionHandler.isAnyInputAvailable()) {
+			fullCheckAndSetAvailable();
+		}
+
+		int readingInputIndex = inputSelectionHandler.selectNextInputIndex(lastReadInputIndex);
+		if (readingInputIndex == -1) {
+			return -1;
+		}
+
+		// to avoid starvation, if the input selection is ALL and availableInputsMask is not ALL,
+		// always try to check and set the availability of another input
+		if (inputSelectionHandler.shouldSetAvailableForAnotherInput()) {
+			fullCheckAndSetAvailable();
+		}
+
+		return readingInputIndex;
+	}
+
+	private void fullCheckAndSetAvailable() {
+		for (int i = 0; i < inputProcessors.length; i++) {
+			InputProcessor<?> inputProcessor = inputProcessors[i];
+			// TODO: isAvailable() can be a costly operation (checking volatile). If one of
+			// the input is constantly available and another is not, we will be checking this volatile
+			// once per every record. This might be optimized to only check once per processed NetworkBuffer
+			if (inputProcessor.networkInput.isApproximatelyAvailable() || inputProcessor.networkInput.isAvailable()) {
+				inputSelectionHandler.setAvailableInput(i);
+			}
+		}
+	}
+
+	private CompletableFuture<?> isAnyInputAvailable() {
+		if (inputSelectionHandler.isAnyInputAvailable()) {
+			return AVAILABLE;
+		}
+		final CompletableFuture<?> anyInputAvailable = new CompletableFuture<>();
+		for (int i = 0; i < inputProcessors.length; i++) {
+			if (!inputSelectionHandler.isInputFinished(i)) {
 
 Review comment:
   > Would probably be better to use CompletableFuture#any in any case.
   
   I was hoping to do just that, but we can not include all of the futures, so we would have to be constructing new array every time we access `CompletableFuture#anyOf(...)` - which might be quite often.

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384354604
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ##########
 @@ -484,11 +485,15 @@ public final void invoke() throws Exception {
 		}
 	}
 
+	protected boolean runMailboxStep() throws Exception {
+		return mailboxProcessor.runMailboxStep();
+	}
+
 
 Review comment:
   Yes, we would have to delegate all methods.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150614271",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150672723",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150856839",
       "triggerID" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5701",
       "triggerID" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b85332b20e2765101e28eca8ac45584ab57c77a0 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150856839) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5701) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] pnowojski commented on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-592010424
 
 
   Yes, we can remove it or keep it as we prefer. Thanks for the reviews. Will rename the method to `transform` and merge.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382653408
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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.transformations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This Transformation represents the application of a
+ * {@link org.apache.flink.streaming.api.operators.MultipleInputStreamOperator}
+ * to input {@code Transformations}. The result is again only one stream.
+ *
+ * @param <OUT> The type of the elements that result from this {@code MultipleInputTransformation}
+ */
+@Internal
+public class MultipleInputTransformation<OUT> extends PhysicalTransformation<OUT> {
+
+	private final ArrayList<Transformation<?>> inputs = new ArrayList<>();
+
+	private final StreamOperatorFactory<OUT> operatorFactory;
+
+	private final ArrayList<KeySelector<?, ?>> stateKeySelectors = new ArrayList<>();
+
+	private TypeInformation<?> stateKeyType;
+
+	public MultipleInputTransformation(
+			String name,
+			StreamOperatorFactory<OUT> operatorFactory,
+			TypeInformation<OUT> outputType,
+			int parallelism) {
+		super(name, outputType, parallelism);
+		this.operatorFactory = operatorFactory;
+	}
+
+	public List<Transformation<?>> getInputs() {
+		return inputs;
+	}
+
+	/**
+	 * Returns the {@code TypeInformation} for the elements from the inputs.
+	 */
+	public List<TypeInformation<?>> getInputTypes() {
+		return inputs.stream()
+			.map(Transformation::getOutputType)
+			.collect(Collectors.toList());
+	}
+
+	/**
+	 * Returns the {@code StreamOperatorFactory} of this Transformation.
+	 */
+	public StreamOperatorFactory<OUT> getOperatorFactory() {
+		return operatorFactory;
+	}
+
+	public void addInput(Transformation<?> input) {
+		checkState(
+			stateKeySelectors.isEmpty(),
+			"Trying to add non-keyed input to keyed transformation.");
 
 Review comment:
   I think it might be a good idea to provide both `MultipleInputTransformation` and `MultipleInputKeyedTransformation`, with some common base class. Maybe for this PR, I would just provide non keyed version, and in the later one, where I'm planning to add actual support for key selectors and multiple input, I would try to add separate class for handling key selectors.
   
   This way, if there will come up any reason why having a single class is actually a better idea, we won't have to revert the changes.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383196743
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ##########
 @@ -484,11 +485,15 @@ public final void invoke() throws Exception {
 		}
 	}
 
+	protected boolean runMailboxStep() throws Exception {
+		return mailboxProcessor.runMailboxStep();
+	}
+
 
 Review comment:
   We could just expose mailboxProcessor to tests and delegate to `MailboxProcessor#runMailboxStep` in harness.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150614271",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150672723",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 352eaf94ac42b2428e2f2f90081e7a023136d9fd Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150614271) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606) 
   * 8e56fa6affa8e58ed6ad9233345c7c5b697de4ea Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150672723) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150614271",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 352eaf94ac42b2428e2f2f90081e7a023136d9fd Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150614271) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot commented on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586308473
 
 
   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit df7f0151d94bc7705c87baf855ae3d8d57f7e463 (Fri Feb 14 14:21:18 UTC 2020)
   
    ✅no warnings
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380741055
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InputSelection.java
 ##########
 @@ -110,8 +101,39 @@ public int fairSelectNextIndexOutOf2(int availableInputsMask, int lastReadInputI
 		throw new UnsupportedOperationException("Only two inputs are supported.");
 	}
 
-	private static boolean isALLMaskOf2(long inputMask) {
-		return (3 & inputMask) == 3;
+	/**
+	 * Fairly select one of the available inputs for reading.
+	 *
+	 * @param availableInputsMask The mask of all available inputs.
+	 * @param lastReadInputIndex The index of last read input.
+	 * @return the index of the input for reading or -1, and -1 indicates no input is selected (
+	 *         {@code inputMask} is empty or the inputs in {@code inputMask} are unavailable).
+	 */
+	public int fairSelectNextIndex(int availableInputsMask, int lastReadInputIndex) {
+		int selectionMask = (int) inputMask;
+		int combineMask = availableInputsMask & selectionMask;
 
 Review comment:
   Do we need special mask for the case when all inputs are selected (`-1L`)?
   Or how is this case handled?

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150614271",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150672723",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8e56fa6affa8e58ed6ad9233345c7c5b697de4ea Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150672723) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149077809 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149160727 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:e9472785fa25680b297986397e3c732875d4ccff Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149163748 TriggerType:PUSH TriggerID:e9472785fa25680b297986397e3c732875d4ccff
   Hash:e9472785fa25680b297986397e3c732875d4ccff Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219 TriggerType:PUSH TriggerID:e9472785fa25680b297986397e3c732875d4ccff
   Hash:82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149269902 TriggerType:PUSH TriggerID:82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f
   Hash:82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244 TriggerType:PUSH TriggerID:82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149077809) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195) 
   * ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149160727) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217) 
   * e9472785fa25680b297986397e3c732875d4ccff Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149163748) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219) 
   * 82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149269902) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384340413
 
 

 ##########
 File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java
 ##########
 @@ -42,15 +42,17 @@
 /**
  * Helper class to build StreamConfig for chain of operators.
  */
-public class StreamConfigChainer {
+public class StreamConfigChainer<OWNER> {
+	private final OWNER owner;
 
 Review comment:
   Yes, exactly. It is a builder to setup operator chain.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382658325
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
 ##########
 @@ -581,6 +611,13 @@ public void setTwoInputStateKey(Integer vertexID, KeySelector<?, ?> keySelector1
 		node.setStateKeySerializer(keySerializer);
 	}
 
+	public void setMultipleInputStateKey(
+			int id,
+			List<KeySelector<?, ?>> stateKeySelectors,
+			TypeSerializer<?> keySerializer) {
+		throw new UnsupportedOperationException();
+	}
 
 Review comment:
   I dropped the method as a result of a previous comment https://github.com/apache/flink/pull/11098/#discussion_r380549321

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382658325
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
 ##########
 @@ -581,6 +611,13 @@ public void setTwoInputStateKey(Integer vertexID, KeySelector<?, ?> keySelector1
 		node.setStateKeySerializer(keySerializer);
 	}
 
+	public void setMultipleInputStateKey(
+			int id,
+			List<KeySelector<?, ?>> stateKeySelectors,
+			TypeSerializer<?> keySerializer) {
+		throw new UnsupportedOperationException();
+	}
 
 Review comment:
   I dropped the method as a result of https://github.com/apache/flink/pull/11098/#discussion_r380549321

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382657856
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
 ##########
 @@ -330,6 +332,34 @@ public boolean isIterative() {
 		}
 	}
 
+	public <OUT> void addMultipleInputOperator(
+			Integer vertexID,
+			String slotSharingGroup,
+			@Nullable String coLocationGroup,
+			StreamOperatorFactory<OUT> operatorFactory,
+			List<TypeInformation<?>> inTypeInfos,
+			TypeInformation<OUT> outTypeInfo,
+			String operatorName) {
+
+		Class<? extends AbstractInvokable> vertexClass = MultipleInputStreamTask.class;
+
+		addNode(vertexID, slotSharingGroup, coLocationGroup, vertexClass, operatorFactory, operatorName);
+
+		TypeSerializer<OUT> outSerializer = (outTypeInfo != null) && !(outTypeInfo instanceof MissingTypeInfo) ?
+			outTypeInfo.createSerializer(executionConfig) : null;
+
 
 Review comment:
   ~what is repeated 4 times? I see that `addMultipleInputOperator ` duplicates (once) `addCoOperator`, but where are other two duplications?~
   
   never mind, found it.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150614271",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150672723",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8e56fa6affa8e58ed6ad9233345c7c5b697de4ea Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150672723) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646) 
   * b85332b20e2765101e28eca8ac45584ab57c77a0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382696744
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InputSelection.java
 ##########
 @@ -110,8 +101,39 @@ public int fairSelectNextIndexOutOf2(int availableInputsMask, int lastReadInputI
 		throw new UnsupportedOperationException("Only two inputs are supported.");
 	}
 
-	private static boolean isALLMaskOf2(long inputMask) {
-		return (3 & inputMask) == 3;
+	/**
+	 * Fairly select one of the available inputs for reading.
+	 *
+	 * @param availableInputsMask The mask of all available inputs.
+	 * @param lastReadInputIndex The index of last read input.
+	 * @return the index of the input for reading or -1, and -1 indicates no input is selected (
+	 *         {@code inputMask} is empty or the inputs in {@code inputMask} are unavailable).
+	 */
+	public int fairSelectNextIndex(int availableInputsMask, int lastReadInputIndex) {
+		int selectionMask = (int) inputMask;
+		int combineMask = availableInputsMask & selectionMask;
+
+		if (combineMask == 0) {
+			return -1;
+		}
+
+		int nextReadInputIndex = fairSelectFromRightBits(combineMask, lastReadInputIndex + 1);
+		if (nextReadInputIndex >= 0) {
+			return nextReadInputIndex;
+		}
+		return fairSelectFromRightBits(combineMask, 0);
+	}
+
+	private int fairSelectFromRightBits(int combineMask, int nextReadInputIndex) {
+		int rightBits = combineMask >> nextReadInputIndex;
+		while (rightBits > 0) {
+			if (rightBits % 2 == 1) {
+				return nextReadInputIndex;
+			}
+			nextReadInputIndex++;
+			rightBits >>= 1;
 
 Review comment:
   Yes. There was also another bug  in this method. If `nextReadInputIndex` was larger or equal to 32, method would enter endless loop, as bitshifts have undefined behaviour for shifting more than number of bits. 

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380728371
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InputSelection.java
 ##########
 @@ -110,8 +101,39 @@ public int fairSelectNextIndexOutOf2(int availableInputsMask, int lastReadInputI
 		throw new UnsupportedOperationException("Only two inputs are supported.");
 	}
 
-	private static boolean isALLMaskOf2(long inputMask) {
-		return (3 & inputMask) == 3;
+	/**
+	 * Fairly select one of the available inputs for reading.
+	 *
+	 * @param availableInputsMask The mask of all available inputs.
+	 * @param lastReadInputIndex The index of last read input.
+	 * @return the index of the input for reading or -1, and -1 indicates no input is selected (
+	 *         {@code inputMask} is empty or the inputs in {@code inputMask} are unavailable).
+	 */
+	public int fairSelectNextIndex(int availableInputsMask, int lastReadInputIndex) {
+		int selectionMask = (int) inputMask;
+		int combineMask = availableInputsMask & selectionMask;
+
+		if (combineMask == 0) {
+			return -1;
+		}
+
+		int nextReadInputIndex = fairSelectFromRightBits(combineMask, lastReadInputIndex + 1);
+		if (nextReadInputIndex >= 0) {
+			return nextReadInputIndex;
+		}
+		return fairSelectFromRightBits(combineMask, 0);
+	}
+
+	private int fairSelectFromRightBits(int combineMask, int nextReadInputIndex) {
 
 Review comment:
   I think the whole function can be simplified as:
   ```
   private int fairSelectFromRightBits(int bits, int next) {
       for (bits >>= next; bits > 0 && bits % 2 != 1; bits >>= 1, next++) {
       }
       return bits > 0 ? next : -1;
   }
   ```
   Removing `if` makes loop more predictable (I understand it's a critical code path, right?)
   

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384129362
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ##########
 @@ -484,11 +485,15 @@ public final void invoke() throws Exception {
 		}
 	}
 
+	protected boolean runMailboxStep() throws Exception {
+		return mailboxProcessor.runMailboxStep();
+	}
+
 
 Review comment:
   I came up with this weird construction:
   ```
   public class TestStreamTask<OUT, OP extends StreamOperator<OUT>> extends StreamTask<OUT, OP> {
   	private final StreamTask<OUT, OP> delegatee;
   
   	public TestStreamTask(StreamTask<OUT, OP> streamTask) {
   		super(streamTask.getEnvironment());
   		this.delegatee = streamTask;
   	}
   
   	public boolean runMailboxStep() throws Exception {
   		return delegate.mailboxProcessor.runMailboxStep();
   	}
           // init()
   }
   ```
   Then replace type `StreamTask` to `TestStreamTask` and create it as:
   ```
   env -> new TestStreamTask<>(new MultipleInputStreamTask<>(env))
   ```
   (should be a factory)
   
   I think this is better than having "test" methods in `StreamTask`.
   
   Exposing `mailboxProcessor` (via constructor?) is also an option. But IMO less preferable because it exposes more details.

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383861318
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
+					i));
+		}
+
+		this.operatorChain = checkNotNull(operatorChain);
+	}
+
+	@Override
+	public CompletableFuture<?> getAvailableFuture() {
+		if (inputSelectionHandler.areAllInputsSelected()) {
+			return isAnyInputAvailable();
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	@Override
+	public InputStatus processInput() throws Exception {
+		int readingInputIndex;
+		if (isPrepared) {
+			readingInputIndex = selectNextReadingInputIndex();
+		} else {
+			// the preparations here are not placed in the constructor because all work in it
+			// must be executed after all operators are opened.
+			readingInputIndex = selectFirstReadingInputIndex();
 
 Review comment:
   Leave as is. Ideally we would have fully initialized operators before any processor is used.

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383201388
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java
 ##########
 @@ -42,8 +42,11 @@ private MetricNames() {
 	public static final String IO_NUM_BUFFERS_OUT_RATE = IO_NUM_BUFFERS_OUT + SUFFIX_RATE;
 
 	public static final String IO_CURRENT_INPUT_WATERMARK = "currentInputWatermark";
+	@Deprecated
 	public static final String IO_CURRENT_INPUT_1_WATERMARK = "currentInput1Watermark";
+	@Deprecated
 	public static final String IO_CURRENT_INPUT_2_WATERMARK = "currentInput2Watermark";
+	public static final String IO_CURRENT_INPUT_WATERMARK_PATERN = "currentInput%dWatermark";
 
 Review comment:
   Could also be done with prefix.

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384161410
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+	@Nullable
+	private final InputSelectable inputSelector;
+
+	private InputSelection inputSelection = InputSelection.ALL;
+
+	private final long allSelectedMask;
+
+	private long availableInputsMask;
+
+	private long endOfInputMask;
+
+	public MultipleInputSelectionHandler(@Nullable InputSelectable inputSelectable, int inputCount) {
+		this.inputSelector = inputSelectable;
+		this.allSelectedMask = (1 << inputCount) - 1;
+		this.availableInputsMask = allSelectedMask;
+	}
+
+	public InputStatus reportInputStatus(InputStatus inputStatus, int inputIndex) throws IOException {
 
 Review comment:
   I find the convention of splitting methods to read-only and writing very helpful.
   This says much less to me:
   ```
   return inputSelectionHandler.reportInputStatus(inputStatus, readingInputIndex);
   ```
   than
   ```
   inputSelectionHandler.updateStatus(inputStatus, readingInputIndex);
   return inputSelectionHandler.getOverallStatus();
   ```
   (I don't even have to go into methods).

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380789836
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
+					i));
+		}
+
+		this.operatorChain = checkNotNull(operatorChain);
+	}
+
+	@Override
+	public CompletableFuture<?> getAvailableFuture() {
+		if (inputSelectionHandler.areAllInputsSelected()) {
+			return isAnyInputAvailable();
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	@Override
+	public InputStatus processInput() throws Exception {
+		int readingInputIndex;
+		if (isPrepared) {
+			readingInputIndex = selectNextReadingInputIndex();
+		} else {
+			// the preparations here are not placed in the constructor because all work in it
+			// must be executed after all operators are opened.
+			readingInputIndex = selectFirstReadingInputIndex();
+		}
+		if (readingInputIndex == -1) {
+			return InputStatus.NOTHING_AVAILABLE;
+		}
+
+		lastReadInputIndex = readingInputIndex;
+		InputStatus inputStatus = inputProcessors[readingInputIndex].processInput();
+		checkFinished(inputStatus, readingInputIndex);
+		return inputSelectionHandler.reportInputStatus(inputStatus, readingInputIndex);
+	}
+
+	private int selectFirstReadingInputIndex() {
+		// Note: the first call to nextSelection () on the operator must be made after this operator
+		// is opened to ensure that any changes about the input selection in its open()
+		// method take effect.
+		inputSelectionHandler.nextSelection();
+
+		isPrepared = true;
+
+		return selectNextReadingInputIndex();
+	}
+
+	private void checkFinished(InputStatus status, int inputIndex) throws Exception {
+		if (status == InputStatus.END_OF_INPUT) {
+			operatorChain.endHeadOperatorInput(getInputId(inputIndex));
+			inputSelectionHandler.nextSelection();
+		}
+	}
+
+	@Override
+	public void close() throws IOException {
+		IOException ex = null;
+		for (InputProcessor<?> input : inputProcessors) {
+			try {
+				input.close();
+			} catch (IOException e) {
+				ex = ExceptionUtils.firstOrSuppressed(e, ex);
+			}
+		}
+
+		if (ex != null) {
+			throw ex;
+		}
+	}
+
+	private int selectNextReadingInputIndex() {
+		if (!inputSelectionHandler.isAnyInputAvailable()) {
+			fullCheckAndSetAvailable();
+		}
+
+		int readingInputIndex = inputSelectionHandler.selectNextInputIndex(lastReadInputIndex);
+		if (readingInputIndex == -1) {
+			return -1;
 
 Review comment:
   nit: extract constant?

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383415907
 
 

 ##########
 File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java
 ##########
 @@ -42,15 +42,17 @@
 /**
  * Helper class to build StreamConfig for chain of operators.
  */
-public class StreamConfigChainer {
+public class StreamConfigChainer<OWNER> {
+	private final OWNER owner;
 
 Review comment:
   It's in order to provide a nice builder like API, for nested builder. That after `finish()` call on setting up the chaining, we  can continue configuring or build the test harness:
   ```
   			new MultipleInputStreamTaskTestHarnessBuilder<>(MultipleInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO)
   				.addInput(BasicTypeInfo.STRING_TYPE_INFO)
   				.addInput(BasicTypeInfo.STRING_TYPE_INFO)
   				.addInput(BasicTypeInfo.STRING_TYPE_INFO)
   				.setupOperatorChain(new DuplicatingOperator())
   				.chain(new OneInputStreamTaskTest.DuplicatingOperator(), BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()))
   				.chain(new OneInputStreamTaskTest.DuplicatingOperator(), BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()))
   				.finish()
   				.setTaskMetricGroup(taskMetricGroup)
   				.build()
   ```
   And `OWNER` is template, so that `finish()` and hence `build()` would return correct specific type of the test harness (`MultipleInput`)

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383206954
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
+					i));
+		}
+
+		this.operatorChain = checkNotNull(operatorChain);
+	}
+
+	@Override
+	public CompletableFuture<?> getAvailableFuture() {
+		if (inputSelectionHandler.areAllInputsSelected()) {
+			return isAnyInputAvailable();
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	@Override
+	public InputStatus processInput() throws Exception {
+		int readingInputIndex;
+		if (isPrepared) {
+			readingInputIndex = selectNextReadingInputIndex();
+		} else {
+			// the preparations here are not placed in the constructor because all work in it
+			// must be executed after all operators are opened.
+			readingInputIndex = selectFirstReadingInputIndex();
+		}
+		if (readingInputIndex == -1) {
+			return InputStatus.NOTHING_AVAILABLE;
+		}
+
+		lastReadInputIndex = readingInputIndex;
+		InputStatus inputStatus = inputProcessors[readingInputIndex].processInput();
+		checkFinished(inputStatus, readingInputIndex);
+		return inputSelectionHandler.reportInputStatus(inputStatus, readingInputIndex);
+	}
+
+	private int selectFirstReadingInputIndex() {
+		// Note: the first call to nextSelection () on the operator must be made after this operator
+		// is opened to ensure that any changes about the input selection in its open()
+		// method take effect.
+		inputSelectionHandler.nextSelection();
+
+		isPrepared = true;
+
+		return selectNextReadingInputIndex();
+	}
+
+	private void checkFinished(InputStatus status, int inputIndex) throws Exception {
+		if (status == InputStatus.END_OF_INPUT) {
+			operatorChain.endHeadOperatorInput(getInputId(inputIndex));
+			inputSelectionHandler.nextSelection();
+		}
+	}
+
+	@Override
+	public void close() throws IOException {
+		IOException ex = null;
+		for (InputProcessor<?> input : inputProcessors) {
+			try {
+				input.close();
+			} catch (IOException e) {
+				ex = ExceptionUtils.firstOrSuppressed(e, ex);
+			}
+		}
+
+		if (ex != null) {
+			throw ex;
+		}
+	}
+
+	private int selectNextReadingInputIndex() {
+		if (!inputSelectionHandler.isAnyInputAvailable()) {
+			fullCheckAndSetAvailable();
+		}
+
+		int readingInputIndex = inputSelectionHandler.selectNextInputIndex(lastReadInputIndex);
+		if (readingInputIndex == -1) {
+			return -1;
+		}
+
+		// to avoid starvation, if the input selection is ALL and availableInputsMask is not ALL,
+		// always try to check and set the availability of another input
+		if (inputSelectionHandler.shouldSetAvailableForAnotherInput()) {
+			fullCheckAndSetAvailable();
+		}
+
+		return readingInputIndex;
+	}
+
+	private void fullCheckAndSetAvailable() {
+		for (int i = 0; i < inputProcessors.length; i++) {
+			InputProcessor<?> inputProcessor = inputProcessors[i];
+			// TODO: isAvailable() can be a costly operation (checking volatile). If one of
+			// the input is constantly available and another is not, we will be checking this volatile
+			// once per every record. This might be optimized to only check once per processed NetworkBuffer
+			if (inputProcessor.networkInput.isApproximatelyAvailable() || inputProcessor.networkInput.isAvailable()) {
+				inputSelectionHandler.setAvailableInput(i);
+			}
+		}
+	}
+
+	private CompletableFuture<?> isAnyInputAvailable() {
+		if (inputSelectionHandler.isAnyInputAvailable()) {
+			return AVAILABLE;
+		}
+		final CompletableFuture<?> anyInputAvailable = new CompletableFuture<>();
+		for (int i = 0; i < inputProcessors.length; i++) {
+			if (!inputSelectionHandler.isInputFinished(i)) {
 
 Review comment:
   Would probably be better to use `CompletableFuture#any` in any case.

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383195008
 
 

 ##########
 File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InputSelectionTest.java
 ##########
 @@ -75,6 +75,28 @@ public void testFairSelectNextIndexOutOf2() {
 		assertEquals(-1, InputSelection.SECOND.fairSelectNextIndexOutOf2(0, 1));
 	}
 
+	@Test
+	public void testFairSelectNextIndex() {
+		assertEquals(1, InputSelection.ALL.fairSelectNextIndex(7, 0));
+		assertEquals(2, InputSelection.ALL.fairSelectNextIndex(7, 1));
+		assertEquals(0, InputSelection.ALL.fairSelectNextIndex(7, 2));
+		assertEquals(1, InputSelection.ALL.fairSelectNextIndex(7, 0));
+
+		// combination of selection and availability is supposed to be 3, 5, 8:
+		InputSelection selection = new Builder().select(2).select(3).select(4).select(5).select(8).build();
 
 Review comment:
   This looks like two test cases to me.

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383185729
 
 

 ##########
 File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
 ##########
 @@ -633,4 +671,29 @@ protected boolean matchesSafely(ResourceSpec item) {
 			return resources.lessThanOrEqual(item) && item.lessThanOrEqual(resources);
 		}
 	}
+
+	private static class MultipleInputOperatorFactory implements StreamOperatorFactory<String> {
 
 Review comment:
   Weird that we need that here... the real factories are still SimpleX? Or can we not add a real factory.

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383860400
 
 

 ##########
 File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java
 ##########
 @@ -42,15 +42,17 @@
 /**
  * Helper class to build StreamConfig for chain of operators.
  */
-public class StreamConfigChainer {
+public class StreamConfigChainer<OWNER> {
+	private final OWNER owner;
 
 Review comment:
   Gotcha. StreamConfigChainer is only used as a builder anyway, right?

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383178461
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
 ##########
 @@ -158,12 +158,11 @@ public TimeCharacteristic getTimeCharacteristic() {
 		}
 	}
 
-	public void setTypeSerializerIn1(TypeSerializer<?> serializer) {
-		setTypeSerializer(TYPE_SERIALIZER_IN_1, serializer);
-	}
-
-	public void setTypeSerializerIn2(TypeSerializer<?> serializer) {
-		setTypeSerializer(TYPE_SERIALIZER_IN_2, serializer);
+	public void setTypeSerializersIn(TypeSerializer<?> ...serializers) {
+		config.setInteger(TYPE_SERIALIZERS_IN_COUNT, serializers.length);
+		for (int i = 0; i < serializers.length; i++) {
 
 Review comment:
   Start at 1 for backwards compatibility?

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383377827
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
 ##########
 @@ -59,7 +59,7 @@
 
 	private boolean wasSplitApplied = false;
 
-	protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, Transformation<T> transformation) {
+	public SingleOutputStreamOperator(StreamExecutionEnvironment environment, Transformation<T> transformation) {
 
 Review comment:
   actually not :) This constructor is the only way to actually use the `MutlipleInputTransformation`. But as this is a `@Public` class, good that you are rising this issue.
   
   @aljoscha you might want to voice your opinion about this and the first commit of this PR in general. 

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382653408
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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.transformations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This Transformation represents the application of a
+ * {@link org.apache.flink.streaming.api.operators.MultipleInputStreamOperator}
+ * to input {@code Transformations}. The result is again only one stream.
+ *
+ * @param <OUT> The type of the elements that result from this {@code MultipleInputTransformation}
+ */
+@Internal
+public class MultipleInputTransformation<OUT> extends PhysicalTransformation<OUT> {
+
+	private final ArrayList<Transformation<?>> inputs = new ArrayList<>();
+
+	private final StreamOperatorFactory<OUT> operatorFactory;
+
+	private final ArrayList<KeySelector<?, ?>> stateKeySelectors = new ArrayList<>();
+
+	private TypeInformation<?> stateKeyType;
+
+	public MultipleInputTransformation(
+			String name,
+			StreamOperatorFactory<OUT> operatorFactory,
+			TypeInformation<OUT> outputType,
+			int parallelism) {
+		super(name, outputType, parallelism);
+		this.operatorFactory = operatorFactory;
+	}
+
+	public List<Transformation<?>> getInputs() {
+		return inputs;
+	}
+
+	/**
+	 * Returns the {@code TypeInformation} for the elements from the inputs.
+	 */
+	public List<TypeInformation<?>> getInputTypes() {
+		return inputs.stream()
+			.map(Transformation::getOutputType)
+			.collect(Collectors.toList());
+	}
+
+	/**
+	 * Returns the {@code StreamOperatorFactory} of this Transformation.
+	 */
+	public StreamOperatorFactory<OUT> getOperatorFactory() {
+		return operatorFactory;
+	}
+
+	public void addInput(Transformation<?> input) {
+		checkState(
+			stateKeySelectors.isEmpty(),
+			"Trying to add non-keyed input to keyed transformation.");
 
 Review comment:
   I think it might be a good idea to provide both `MultipleInputTransformation` and `MultipleInputKeyedTransformation` (I was duplicating pre-existing pattern), with some common base class. Maybe for this PR, I would just provide non keyed version, and in the later one, where I'm planning to add actual support for key selectors and multiple input, I would try to add separate class for handling key selectors.
   
   This way, if there will come up any reason why having a single class is actually a better idea, we won't have to revert the changes.

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383179373
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
 ##########
 @@ -174,19 +173,35 @@ public void setTypeSerializerSideOut(OutputTag<?> outputTag, TypeSerializer<?> s
 		setTypeSerializer(TYPE_SERIALIZER_SIDEOUT_PREFIX + outputTag.getId(), serializer);
 	}
 
+	@Deprecated
 	public <T> TypeSerializer<T> getTypeSerializerIn1(ClassLoader cl) {
-		try {
-			return InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_IN_1, cl);
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate serializer.", e);
-		}
+		return getTypeSerializerIn(0, cl);
 	}
 
+	@Deprecated
 	public <T> TypeSerializer<T> getTypeSerializerIn2(ClassLoader cl) {
+		return getTypeSerializerIn(1, cl);
+	}
+
+	public TypeSerializer<?>[] getTypeSerializersIn(ClassLoader cl) {
+		int typeSerializersCount = config.getInteger(TYPE_SERIALIZERS_IN_COUNT, -1);
+		TypeSerializer<?>[] typeSerializers = new TypeSerializer<?>[typeSerializersCount];
 
 Review comment:
   handle -1? Would give a weird exception.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149077809 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149160727 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:e9472785fa25680b297986397e3c732875d4ccff Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149163748 TriggerType:PUSH TriggerID:e9472785fa25680b297986397e3c732875d4ccff
   Hash:e9472785fa25680b297986397e3c732875d4ccff Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219 TriggerType:PUSH TriggerID:e9472785fa25680b297986397e3c732875d4ccff
   Hash:82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149269902 TriggerType:PUSH TriggerID:82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f
   Hash:82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244 TriggerType:PUSH TriggerID:82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149077809) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195) 
   * ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149160727) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217) 
   * e9472785fa25680b297986397e3c732875d4ccff Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149163748) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219) 
   * 82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149269902) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380658811
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
 ##########
 @@ -735,6 +739,56 @@ public StreamGraph generate() {
 		return Collections.singleton(transform.getId());
 	}
 
+	private <OUT> Collection<Integer> transformMultipleInputTransform(MultipleInputTransformation<OUT> transform) {
+		List<Collection<Integer>> allInputIds = new ArrayList<>();
+
+		for (Transformation<?> input : transform.getInputs()) {
+			allInputIds.add(transform(input));
+		}
+
+		// the recursive call might have already transformed this
+		if (alreadyTransformed.containsKey(transform)) {
+			return alreadyTransformed.get(transform);
+		}
+
 
 Review comment:
   Could you please explain:
   1. why do we need this check here? there is already one in `transform()`?
   2. why the check doesn't precede `transform()` of inputs?

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383191727
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InputSelection.java
 ##########
 @@ -110,6 +110,41 @@ public int fairSelectNextIndexOutOf2(int availableInputsMask, int lastReadInputI
 		throw new UnsupportedOperationException("Only two inputs are supported.");
 	}
 
+	/**
+	 * Fairly select one of the available inputs for reading.
+	 *
+	 * @param availableInputsMask The mask of all available inputs.
+	 * @param lastReadInputIndex The index of last read input.
+	 * @return the index of the input for reading or -1, and -1 indicates no input is selected (
+	 *         {@code inputMask} is empty or the inputs in {@code inputMask} are unavailable).
+	 */
+	public int fairSelectNextIndex(int availableInputsMask, int lastReadInputIndex) {
 
 Review comment:
   any reason why inputMask is long but availableInputsMask is not?

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383194378
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InputSelection.java
 ##########
 @@ -110,6 +110,41 @@ public int fairSelectNextIndexOutOf2(int availableInputsMask, int lastReadInputI
 		throw new UnsupportedOperationException("Only two inputs are supported.");
 	}
 
+	/**
+	 * Fairly select one of the available inputs for reading.
+	 *
+	 * @param availableInputsMask The mask of all available inputs.
+	 * @param lastReadInputIndex The index of last read input.
+	 * @return the index of the input for reading or -1, and -1 indicates no input is selected (
+	 *         {@code inputMask} is empty or the inputs in {@code inputMask} are unavailable).
+	 */
+	public int fairSelectNextIndex(int availableInputsMask, int lastReadInputIndex) {
+		int selectionMask = (int) inputMask;
+		int combineMask = availableInputsMask & selectionMask;
+
+		if (combineMask == 0) {
+			return -1;
+		}
+
+		int nextReadInputIndex = fairSelectFromRightBits(combineMask, lastReadInputIndex + 1);
+		if (nextReadInputIndex >= 0) {
+			return nextReadInputIndex;
+		}
+		return fairSelectFromRightBits(combineMask, 0);
+	}
+
+	private int fairSelectFromRightBits(int combineMask, int nextReadInputIndex) {
+		int rightBits = combineMask >> nextReadInputIndex;
+		while (rightBits > 0) {
 
 Review comment:
   Since this is very performance critical, we might need to check if we can use the ideas behind `Integer#numberOfTrailingZeros` to speed things up.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150614271",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 352eaf94ac42b2428e2f2f90081e7a023136d9fd Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150614271) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606) 
   * 8e56fa6affa8e58ed6ad9233345c7c5b697de4ea UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382698544
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ##########
 @@ -484,11 +485,15 @@ public final void invoke() throws Exception {
 		}
 	}
 
+	protected boolean runMailboxStep() throws Exception {
+		return mailboxProcessor.runMailboxStep();
+	}
+
 
 Review comment:
   I don't see an easy way how to do it, as we would have to provide multiple different test classes (one per each production `StreamTask` subclass?). 

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380730451
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InputSelection.java
 ##########
 @@ -110,8 +101,39 @@ public int fairSelectNextIndexOutOf2(int availableInputsMask, int lastReadInputI
 		throw new UnsupportedOperationException("Only two inputs are supported.");
 	}
 
-	private static boolean isALLMaskOf2(long inputMask) {
-		return (3 & inputMask) == 3;
+	/**
+	 * Fairly select one of the available inputs for reading.
+	 *
+	 * @param availableInputsMask The mask of all available inputs.
+	 * @param lastReadInputIndex The index of last read input.
+	 * @return the index of the input for reading or -1, and -1 indicates no input is selected (
+	 *         {@code inputMask} is empty or the inputs in {@code inputMask} are unavailable).
+	 */
+	public int fairSelectNextIndex(int availableInputsMask, int lastReadInputIndex) {
+		int selectionMask = (int) inputMask;
+		int combineMask = availableInputsMask & selectionMask;
+
+		if (combineMask == 0) {
+			return -1;
+		}
+
+		int nextReadInputIndex = fairSelectFromRightBits(combineMask, lastReadInputIndex + 1);
+		if (nextReadInputIndex >= 0) {
+			return nextReadInputIndex;
+		}
+		return fairSelectFromRightBits(combineMask, 0);
+	}
+
+	private int fairSelectFromRightBits(int combineMask, int nextReadInputIndex) {
+		int rightBits = combineMask >> nextReadInputIndex;
+		while (rightBits > 0) {
+			if (rightBits % 2 == 1) {
+				return nextReadInputIndex;
+			}
+			nextReadInputIndex++;
+			rightBits >>= 1;
 
 Review comment:
   Should we use `>>>` instead of `>>`? (unlikely we hit this situation... but IPv4... :) )
   (same in other places where we shift)

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383201040
 
 

 ##########
 File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java
 ##########
 @@ -42,15 +42,17 @@
 /**
  * Helper class to build StreamConfig for chain of operators.
  */
-public class StreamConfigChainer {
+public class StreamConfigChainer<OWNER> {
+	private final OWNER owner;
 
 Review comment:
   I didn't get why we'd need the owner in this commit.

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380770621
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+	@Nullable
+	private final InputSelectable inputSelector;
+
+	private InputSelection inputSelection = InputSelection.ALL;
+
+	private final long allSelectedMask;
+
+	private long availableInputsMask;
+
+	private long endOfInputMask;
+
+	public MultipleInputSelectionHandler(@Nullable InputSelectable inputSelectable, int inputCount) {
+		this.inputSelector = inputSelectable;
+		this.allSelectedMask = (1 << inputCount) - 1;
+		this.availableInputsMask = allSelectedMask;
+	}
+
+	public InputStatus reportInputStatus(InputStatus inputStatus, int inputIndex) throws IOException {
 
 Review comment:
   I think this method shouldn't mix these responsibilities:
   1. update state for the given input
   2. calculate the aggregate status
   
   (`reportInputStatus` could be `void` and client could additinally call `calculateInputStatus` )

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383181941
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
 ##########
 @@ -735,6 +739,56 @@ public StreamGraph generate() {
 		return Collections.singleton(transform.getId());
 	}
 
+	private <OUT> Collection<Integer> transformMultipleInputTransform(MultipleInputTransformation<OUT> transform) {
+		List<Collection<Integer>> allInputIds = new ArrayList<>();
+
+		for (Transformation<?> input : transform.getInputs()) {
+			allInputIds.add(transform(input));
+		}
+
+		// the recursive call might have already transformed this
+		if (alreadyTransformed.containsKey(transform)) {
+			return alreadyTransformed.get(transform);
+		}
+
 
 Review comment:
   Should be before transform() of inputs.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383231215
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
+					i));
+		}
+
+		this.operatorChain = checkNotNull(operatorChain);
+	}
+
+	@Override
+	public CompletableFuture<?> getAvailableFuture() {
+		if (inputSelectionHandler.areAllInputsSelected()) {
+			return isAnyInputAvailable();
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	@Override
+	public InputStatus processInput() throws Exception {
+		int readingInputIndex;
+		if (isPrepared) {
+			readingInputIndex = selectNextReadingInputIndex();
+		} else {
+			// the preparations here are not placed in the constructor because all work in it
+			// must be executed after all operators are opened.
+			readingInputIndex = selectFirstReadingInputIndex();
+		}
+		if (readingInputIndex == -1) {
+			return InputStatus.NOTHING_AVAILABLE;
+		}
+
+		lastReadInputIndex = readingInputIndex;
+		InputStatus inputStatus = inputProcessors[readingInputIndex].processInput();
+		checkFinished(inputStatus, readingInputIndex);
+		return inputSelectionHandler.reportInputStatus(inputStatus, readingInputIndex);
+	}
+
+	private int selectFirstReadingInputIndex() {
+		// Note: the first call to nextSelection () on the operator must be made after this operator
+		// is opened to ensure that any changes about the input selection in its open()
+		// method take effect.
+		inputSelectionHandler.nextSelection();
+
+		isPrepared = true;
+
+		return selectNextReadingInputIndex();
+	}
+
+	private void checkFinished(InputStatus status, int inputIndex) throws Exception {
+		if (status == InputStatus.END_OF_INPUT) {
+			operatorChain.endHeadOperatorInput(getInputId(inputIndex));
+			inputSelectionHandler.nextSelection();
+		}
+	}
+
+	@Override
+	public void close() throws IOException {
+		IOException ex = null;
+		for (InputProcessor<?> input : inputProcessors) {
+			try {
+				input.close();
+			} catch (IOException e) {
+				ex = ExceptionUtils.firstOrSuppressed(e, ex);
+			}
+		}
+
+		if (ex != null) {
+			throw ex;
+		}
+	}
+
+	private int selectNextReadingInputIndex() {
+		if (!inputSelectionHandler.isAnyInputAvailable()) {
+			fullCheckAndSetAvailable();
+		}
+
+		int readingInputIndex = inputSelectionHandler.selectNextInputIndex(lastReadInputIndex);
+		if (readingInputIndex == -1) {
+			return -1;
+		}
+
+		// to avoid starvation, if the input selection is ALL and availableInputsMask is not ALL,
+		// always try to check and set the availability of another input
+		if (inputSelectionHandler.shouldSetAvailableForAnotherInput()) {
+			fullCheckAndSetAvailable();
+		}
+
+		return readingInputIndex;
+	}
+
+	private void fullCheckAndSetAvailable() {
+		for (int i = 0; i < inputProcessors.length; i++) {
+			InputProcessor<?> inputProcessor = inputProcessors[i];
+			// TODO: isAvailable() can be a costly operation (checking volatile). If one of
+			// the input is constantly available and another is not, we will be checking this volatile
+			// once per every record. This might be optimized to only check once per processed NetworkBuffer
+			if (inputProcessor.networkInput.isApproximatelyAvailable() || inputProcessor.networkInput.isAvailable()) {
+				inputSelectionHandler.setAvailableInput(i);
+			}
+		}
+	}
+
+	private CompletableFuture<?> isAnyInputAvailable() {
+		if (inputSelectionHandler.isAnyInputAvailable()) {
+			return AVAILABLE;
+		}
+		final CompletableFuture<?> anyInputAvailable = new CompletableFuture<>();
+		for (int i = 0; i < inputProcessors.length; i++) {
+			if (!inputSelectionHandler.isInputFinished(i)) {
 
 Review comment:
   Good catch, there should be one more check +/-:
   
   ```
   if (inputSelectionHandler.isAnyInputAvailable() || inputSelectionHandler.areAllInputsFinished()) {
   	return AVAILABLE;
   }
   ```

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384338237
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+	@Nullable
+	private final InputSelectable inputSelector;
+
+	private InputSelection inputSelection = InputSelection.ALL;
+
+	private final long allSelectedMask;
+
+	private long availableInputsMask;
+
+	private long endOfInputMask;
+
+	public MultipleInputSelectionHandler(@Nullable InputSelectable inputSelectable, int inputCount) {
+		this.inputSelector = inputSelectable;
+		this.allSelectedMask = (1 << inputCount) - 1;
+		this.availableInputsMask = allSelectedMask;
+	}
+
+	public InputStatus reportInputStatus(InputStatus inputStatus, int inputIndex) throws IOException {
 
 Review comment:
   Yes and no. Yes, it's a bit easier to understand, but no, as it creates opportunity for code duplication. I think I would change it as you are suggesting if not for that hot looping argument.
   
   So I would leave it as it is, with @AHeise 's suggestion to rename to `updateStatus`?

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380654321
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
 ##########
 @@ -330,6 +332,34 @@ public boolean isIterative() {
 		}
 	}
 
+	public <OUT> void addMultipleInputOperator(
+			Integer vertexID,
+			String slotSharingGroup,
+			@Nullable String coLocationGroup,
+			StreamOperatorFactory<OUT> operatorFactory,
+			List<TypeInformation<?>> inTypeInfos,
+			TypeInformation<OUT> outTypeInfo,
+			String operatorName) {
+
+		Class<? extends AbstractInvokable> vertexClass = MultipleInputStreamTask.class;
+
+		addNode(vertexID, slotSharingGroup, coLocationGroup, vertexClass, operatorFactory, operatorName);
+
+		TypeSerializer<OUT> outSerializer = (outTypeInfo != null) && !(outTypeInfo instanceof MissingTypeInfo) ?
+			outTypeInfo.createSerializer(executionConfig) : null;
+
 
 Review comment:
   nit: extract method (repeated 4 times)?

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383385507
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
 ##########
 @@ -637,19 +638,24 @@ public StreamNode getStreamNode(Integer vertexID) {
 		return streamNodes.keySet();
 	}
 
+	@VisibleForTesting
 	public List<StreamEdge> getStreamEdges(int sourceId, int targetId) {
-
 		List<StreamEdge> result = new ArrayList<>();
 		for (StreamEdge edge : getStreamNode(sourceId).getOutEdges()) {
 			if (edge.getTargetId() == targetId) {
 				result.add(edge);
 			}
 		}
+		return result;
+	}
 
+	@VisibleForTesting
+	@Deprecated
+	public List<StreamEdge> getStreamEdgesOrThrow(int sourceId, int targetId) {
 
 Review comment:
   Keep in mind that this method is only used in the tests, to make some assertions. 
   
   Generally speaking this old `getStreamEdges()` with throwing an exception was a bad & confusing design. It should never be like that. If it wasn't supposed to be empty, test should just assert manually content of the actual list, instead of relaying on this method throwing an exception. I tried to use it to actually make an assertion that the list is empty and it exploded in my face with some `RuntimeException`...
   
   If not for the fact that this method is used a lot in the tests and annoyingly, couple percent of those tests are actually relaying on the fact that this method throws, I would just drop it and use `getStreamEdges()` everywhere properly , but there were too many usages and I gave up...

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384544359
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/MultipleConnectedStreams.java
 ##########
 @@ -0,0 +1,47 @@
+/*
+ * 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.datastream;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.transformations.MultipleInputTransformation;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This is a very basic and rough stub for a class connecting multiple input {@link DataStream}s
+ * into one, using {@link MultipleInputStreamOperator}.
+ */
+@Experimental
+public class MultipleConnectedStreams {
 
 Review comment:
   @AHeise could you take a look at the first commit once more time? I've reverted the `@Public` api change.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150614271",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 68393ef701e7b462638a26e89df167fa4a36ec14 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150438127) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551) 
   * 352eaf94ac42b2428e2f2f90081e7a023136d9fd Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150614271) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383203205
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+	@Nullable
+	private final InputSelectable inputSelector;
+
+	private InputSelection inputSelection = InputSelection.ALL;
+
+	private final long allSelectedMask;
+
+	private long availableInputsMask;
+
+	private long endOfInputMask;
+
+	public MultipleInputSelectionHandler(@Nullable InputSelectable inputSelectable, int inputCount) {
+		this.inputSelector = inputSelectable;
+		this.allSelectedMask = (1 << inputCount) - 1;
+		this.availableInputsMask = allSelectedMask;
+	}
+
+	public InputStatus reportInputStatus(InputStatus inputStatus, int inputIndex) throws IOException {
 
 Review comment:
   I don't see an issue with that. For me this is the transition function of an finite automate.
   nit: reportInputStatus -> updateInputStatus

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383181327
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+	@Nullable
+	private final InputSelectable inputSelector;
+
+	private InputSelection inputSelection = InputSelection.ALL;
+
+	private final long allSelectedMask;
+
+	private long availableInputsMask;
+
+	private long endOfInputMask;
+
+	public MultipleInputSelectionHandler(@Nullable InputSelectable inputSelectable, int inputCount) {
+		this.inputSelector = inputSelectable;
+		this.allSelectedMask = (1 << inputCount) - 1;
+		this.availableInputsMask = allSelectedMask;
+	}
+
+	public InputStatus reportInputStatus(InputStatus inputStatus, int inputIndex) throws IOException {
 
 Review comment:
   I'm not sure. Also as it is now, the hot looping path
   ```
   			case MORE_AVAILABLE:
   				checkState(checkBitMask(availableInputsMask, inputIndex));
   				return InputStatus.MORE_AVAILABLE;
   ```
   is a bit shorter, this probably doesn't matter much, but I'm not sure in the first place if it makes sense to push the responsibility on a caller, to always call two methods (especially since they are logically inseparable), .

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383173795
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
 ##########
 @@ -59,7 +59,7 @@
 
 	private boolean wasSplitApplied = false;
 
-	protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, Transformation<T> transformation) {
+	public SingleOutputStreamOperator(StreamExecutionEnvironment environment, Transformation<T> transformation) {
 
 Review comment:
   Random change?

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383402049
 
 

 ##########
 File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
 ##########
 @@ -312,6 +316,40 @@ public void testOutputTypeConfigurationWithTwoInputTransformation() throws Excep
 		assertEquals(BasicTypeInfo.INT_TYPE_INFO, outputTypeConfigurableOperation.getTypeInformation());
 	}
 
+	@Test
+	public void testMultipleInputTransformation() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		//TODO: referring to getNewNodeId() is hacky, but I'm not sure if there is a better idea?
+		int startingNewNodeId = Transformation.getNewNodeId();
+
+		DataStream<Integer> source1 = env.fromElements(1, 10);
+		DataStream<Long> source2 = env.fromElements(2L, 11L);
+		DataStream<String> source3 = env.fromElements("42", "44");
+
+		MultipleInputTransformation<String> transform = new MultipleInputTransformation<String>(
+			"My Operator",
+			new MultipleInputOperatorFactory(),
+			BasicTypeInfo.STRING_TYPE_INFO,
+			3);
+
+		transform.addInput(source1.getTransformation());
+		transform.addInput(source2.getTransformation());
+		transform.addInput(source3.getTransformation());
+
+		env.addOperator(transform);
+		StreamGraph streamGraph = env.getStreamGraph();
+		assertEquals(4, streamGraph.getStreamNodes().size());
+
+		int id = startingNewNodeId;
+		assertEquals(1, streamGraph.getStreamEdges(id + 1, id + 4).size());
+		assertEquals(1, streamGraph.getStreamEdges(id + 2, id + 4).size());
+		assertEquals(1, streamGraph.getStreamEdges(id + 3, id + 4).size());
+		assertEquals(1, streamGraph.getStreamEdges(id + 1).size());
+		assertEquals(1, streamGraph.getStreamEdges(id + 2).size());
+		assertEquals(1, streamGraph.getStreamEdges(id + 3).size());
+		assertEquals(0, streamGraph.getStreamEdges(id + 4).size());
 
 Review comment:
   I don't know why I haven't figured this out 😓 

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382681408
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InputSelection.java
 ##########
 @@ -110,8 +101,39 @@ public int fairSelectNextIndexOutOf2(int availableInputsMask, int lastReadInputI
 		throw new UnsupportedOperationException("Only two inputs are supported.");
 	}
 
-	private static boolean isALLMaskOf2(long inputMask) {
-		return (3 & inputMask) == 3;
+	/**
+	 * Fairly select one of the available inputs for reading.
+	 *
+	 * @param availableInputsMask The mask of all available inputs.
+	 * @param lastReadInputIndex The index of last read input.
+	 * @return the index of the input for reading or -1, and -1 indicates no input is selected (
+	 *         {@code inputMask} is empty or the inputs in {@code inputMask} are unavailable).
+	 */
+	public int fairSelectNextIndex(int availableInputsMask, int lastReadInputIndex) {
+		int selectionMask = (int) inputMask;
+		int combineMask = availableInputsMask & selectionMask;
+
+		if (combineMask == 0) {
+			return -1;
+		}
+
+		int nextReadInputIndex = fairSelectFromRightBits(combineMask, lastReadInputIndex + 1);
+		if (nextReadInputIndex >= 0) {
+			return nextReadInputIndex;
+		}
+		return fairSelectFromRightBits(combineMask, 0);
+	}
+
+	private int fairSelectFromRightBits(int combineMask, int nextReadInputIndex) {
+		int rightBits = combineMask >> nextReadInputIndex;
+		while (rightBits > 0) {
+			if (rightBits % 2 == 1) {
 
 Review comment:
   ~For me `% 2` is easier to understand, as `%` operator is used much more frequently compared to `&`. And there is zero performance difference between those two (both will be optimised to the same machine code)~
   
   scratch that, `& 1` behaves somehow better for negative inputs. 

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149077809 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149160727 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:e9472785fa25680b297986397e3c732875d4ccff Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149163748 TriggerType:PUSH TriggerID:e9472785fa25680b297986397e3c732875d4ccff
   Hash:e9472785fa25680b297986397e3c732875d4ccff Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219 TriggerType:PUSH TriggerID:e9472785fa25680b297986397e3c732875d4ccff
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149077809) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195) 
   * ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149160727) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217) 
   * e9472785fa25680b297986397e3c732875d4ccff Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149163748) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380789462
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
+					i));
+		}
+
+		this.operatorChain = checkNotNull(operatorChain);
+	}
+
+	@Override
+	public CompletableFuture<?> getAvailableFuture() {
+		if (inputSelectionHandler.areAllInputsSelected()) {
+			return isAnyInputAvailable();
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	@Override
+	public InputStatus processInput() throws Exception {
+		int readingInputIndex;
+		if (isPrepared) {
+			readingInputIndex = selectNextReadingInputIndex();
+		} else {
+			// the preparations here are not placed in the constructor because all work in it
+			// must be executed after all operators are opened.
+			readingInputIndex = selectFirstReadingInputIndex();
+		}
+		if (readingInputIndex == -1) {
+			return InputStatus.NOTHING_AVAILABLE;
+		}
+
+		lastReadInputIndex = readingInputIndex;
+		InputStatus inputStatus = inputProcessors[readingInputIndex].processInput();
+		checkFinished(inputStatus, readingInputIndex);
+		return inputSelectionHandler.reportInputStatus(inputStatus, readingInputIndex);
+	}
+
+	private int selectFirstReadingInputIndex() {
+		// Note: the first call to nextSelection () on the operator must be made after this operator
+		// is opened to ensure that any changes about the input selection in its open()
+		// method take effect.
+		inputSelectionHandler.nextSelection();
+
+		isPrepared = true;
+
+		return selectNextReadingInputIndex();
+	}
+
+	private void checkFinished(InputStatus status, int inputIndex) throws Exception {
+		if (status == InputStatus.END_OF_INPUT) {
+			operatorChain.endHeadOperatorInput(getInputId(inputIndex));
+			inputSelectionHandler.nextSelection();
+		}
+	}
+
+	@Override
+	public void close() throws IOException {
+		IOException ex = null;
+		for (InputProcessor<?> input : inputProcessors) {
+			try {
+				input.close();
+			} catch (IOException e) {
+				ex = ExceptionUtils.firstOrSuppressed(e, ex);
+			}
+		}
+
+		if (ex != null) {
+			throw ex;
+		}
+	}
+
+	private int selectNextReadingInputIndex() {
+		if (!inputSelectionHandler.isAnyInputAvailable()) {
+			fullCheckAndSetAvailable();
+		}
+
+		int readingInputIndex = inputSelectionHandler.selectNextInputIndex(lastReadInputIndex);
+		if (readingInputIndex == -1) {
+			return -1;
+		}
+
+		// to avoid starvation, if the input selection is ALL and availableInputsMask is not ALL,
+		// always try to check and set the availability of another input
+		if (inputSelectionHandler.shouldSetAvailableForAnotherInput()) {
+			fullCheckAndSetAvailable();
+		}
+
+		return readingInputIndex;
+	}
+
+	private void fullCheckAndSetAvailable() {
+		for (int i = 0; i < inputProcessors.length; i++) {
+			InputProcessor<?> inputProcessor = inputProcessors[i];
+			// TODO: isAvailable() can be a costly operation (checking volatile). If one of
+			// the input is constantly available and another is not, we will be checking this volatile
+			// once per every record. This might be optimized to only check once per processed NetworkBuffer
+			if (inputProcessor.networkInput.isApproximatelyAvailable() || inputProcessor.networkInput.isAvailable()) {
+				inputSelectionHandler.setAvailableInput(i);
+			}
+		}
+	}
+
+	private CompletableFuture<?> isAnyInputAvailable() {
+		if (inputSelectionHandler.isAnyInputAvailable()) {
+			return AVAILABLE;
+		}
+		final CompletableFuture<?> anyInputAvailable = new CompletableFuture<>();
+		for (int i = 0; i < inputProcessors.length; i++) {
+			if (!inputSelectionHandler.isInputFinished(i)) {
 
 Review comment:
   The future will never complete if all inputs are finished, won't it? 

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380766284
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+	@Nullable
+	private final InputSelectable inputSelector;
+
+	private InputSelection inputSelection = InputSelection.ALL;
+
+	private final long allSelectedMask;
+
+	private long availableInputsMask;
+
+	private long endOfInputMask;
 
 Review comment:
   Looks like `endOfInputMask` is always negated.
   Should we reverse the logic then?
   I.e., have `nonEndedInputs` set to all ones initially, and zeroed on `END_OF_INPUT`?

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384368915
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+	@Nullable
+	private final InputSelectable inputSelector;
+
+	private InputSelection inputSelection = InputSelection.ALL;
+
+	private final long allSelectedMask;
+
+	private long availableInputsMask;
+
+	private long endOfInputMask;
+
+	public MultipleInputSelectionHandler(@Nullable InputSelectable inputSelectable, int inputCount) {
+		this.inputSelector = inputSelectable;
+		this.allSelectedMask = (1 << inputCount) - 1;
+		this.availableInputsMask = allSelectedMask;
+	}
+
+	public InputStatus reportInputStatus(InputStatus inputStatus, int inputIndex) throws IOException {
 
 Review comment:
   I'd choose readability now vs avoiding *Opportunity* for duplication of 2 lines. And even if we'd have two call sites, I'd prefer to see them more readable.
   
   Hot looping argument seems like premature optimization to me. 
   This is the 2nd or 3rd time this argument is used in this PR without a benchmark for a hot code path (right?). So probably it makes sense to add a benchmark for this and maybe as a follow-up PR. So let's keep the current version or the suggested by @AHeise and hope to improve it later.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ac7b323867f647d50b170c23df6c6052b2c08d61 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150350672) 
   * 68393ef701e7b462638a26e89df167fa4a36ec14 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150438127) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384161410
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+	@Nullable
+	private final InputSelectable inputSelector;
+
+	private InputSelection inputSelection = InputSelection.ALL;
+
+	private final long allSelectedMask;
+
+	private long availableInputsMask;
+
+	private long endOfInputMask;
+
+	public MultipleInputSelectionHandler(@Nullable InputSelectable inputSelectable, int inputCount) {
+		this.inputSelector = inputSelectable;
+		this.allSelectedMask = (1 << inputCount) - 1;
+		this.availableInputsMask = allSelectedMask;
+	}
+
+	public InputStatus reportInputStatus(InputStatus inputStatus, int inputIndex) throws IOException {
 
 Review comment:
   I find the convention of splitting methods to read-only and writing very helpful.
   This says much less to me:
   ```
   return inputSelectionHandler.reportInputStatus(inputStatus, readingInputIndex);
   ```
   than
   ```
   inputSelectionHandler.updateStatus(inputStatus, readingInputIndex);
   return inputSelectionHandler.getOverallStatus();
   ```
   (I don't even have to go into methods to understand what they do).

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383185280
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+	@Nullable
+	private final InputSelectable inputSelector;
+
+	private InputSelection inputSelection = InputSelection.ALL;
+
+	private final long allSelectedMask;
+
+	private long availableInputsMask;
+
+	private long endOfInputMask;
+
+	public MultipleInputSelectionHandler(@Nullable InputSelectable inputSelectable, int inputCount) {
+		this.inputSelector = inputSelectable;
+		this.allSelectedMask = (1 << inputCount) - 1;
+		this.availableInputsMask = allSelectedMask;
+	}
+
+	public InputStatus reportInputStatus(InputStatus inputStatus, int inputIndex) throws IOException {
+		switch (inputStatus) {
+			case MORE_AVAILABLE:
+				checkState(checkBitMask(availableInputsMask, inputIndex));
+				return InputStatus.MORE_AVAILABLE;
+			case NOTHING_AVAILABLE:
+				availableInputsMask = unsetBitMask(availableInputsMask, inputIndex);
+				break;
+			case END_OF_INPUT:
+				endOfInputMask = setBitMask(endOfInputMask, inputIndex);
+				break;
+			default:
+				throw new UnsupportedOperationException("Unsupported inputStatus = " + inputStatus);
+		}
+
+		return calculateInputStatus();
+	}
+
+	public InputStatus calculateInputStatus() throws IOException {
+		if (endOfInputMask == allSelectedMask) {
+			return InputStatus.END_OF_INPUT;
+		}
+
+		if (isAnyInputAvailable()) {
+			return InputStatus.MORE_AVAILABLE;
+		}
+		else {
+			long selectedNotFinishedInputMask = inputSelection.getInputMask() & ~(endOfInputMask);
+			if (selectedNotFinishedInputMask == 0) {
+				throw new IOException("Can not make a progress: all selected inputs are already finished");
+			}
+			return InputStatus.NOTHING_AVAILABLE;
+		}
+	}
+
+	void nextSelection() {
+		if (inputSelector == null) {
+			inputSelection = InputSelection.ALL;
 
 Review comment:
   It would be indeed nicer. However this brings extra overhead of virtualisation, and might mess up with the optimiser for the common path (no  `InputSelector`), preventing inlining etc. I'm not sure how much does it matter in here though. 
   
   However if we wanted to do it, it should also be changed in the two input processor as well, and would require more cautious regression benchmarks. So I would suggest to do it independently.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383388037
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
 ##########
 @@ -158,12 +158,11 @@ public TimeCharacteristic getTimeCharacteristic() {
 		}
 	}
 
-	public void setTypeSerializerIn1(TypeSerializer<?> serializer) {
-		setTypeSerializer(TYPE_SERIALIZER_IN_1, serializer);
-	}
-
-	public void setTypeSerializerIn2(TypeSerializer<?> serializer) {
-		setTypeSerializer(TYPE_SERIALIZER_IN_2, serializer);
+	public void setTypeSerializersIn(TypeSerializer<?> ...serializers) {
+		config.setInteger(TYPE_SERIALIZERS_IN_COUNT, serializers.length);
+		for (int i = 0; i < serializers.length; i++) {
 
 Review comment:
   no need for the backwards compatibility - I've double checked that with SDK team.

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380761067
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+	@Nullable
+	private final InputSelectable inputSelector;
+
+	private InputSelection inputSelection = InputSelection.ALL;
+
+	private final long allSelectedMask;
+
+	private long availableInputsMask;
+
+	private long endOfInputMask;
+
+	public MultipleInputSelectionHandler(@Nullable InputSelectable inputSelectable, int inputCount) {
+		this.inputSelector = inputSelectable;
+		this.allSelectedMask = (1 << inputCount) - 1;
+		this.availableInputsMask = allSelectedMask;
+	}
+
+	public InputStatus reportInputStatus(InputStatus inputStatus, int inputIndex) throws IOException {
+		switch (inputStatus) {
+			case MORE_AVAILABLE:
+				checkState(checkBitMask(availableInputsMask, inputIndex));
+				return InputStatus.MORE_AVAILABLE;
+			case NOTHING_AVAILABLE:
+				availableInputsMask = unsetBitMask(availableInputsMask, inputIndex);
+				break;
+			case END_OF_INPUT:
+				endOfInputMask = setBitMask(endOfInputMask, inputIndex);
+				break;
+			default:
+				throw new UnsupportedOperationException("Unsupported inputStatus = " + inputStatus);
+		}
+
+		return calculateInputStatus();
+	}
+
+	public InputStatus calculateInputStatus() throws IOException {
+		if (endOfInputMask == allSelectedMask) {
+			return InputStatus.END_OF_INPUT;
+		}
+
+		if (isAnyInputAvailable()) {
+			return InputStatus.MORE_AVAILABLE;
+		}
+		else {
+			long selectedNotFinishedInputMask = inputSelection.getInputMask() & ~(endOfInputMask);
+			if (selectedNotFinishedInputMask == 0) {
+				throw new IOException("Can not make a progress: all selected inputs are already finished");
+			}
+			return InputStatus.NOTHING_AVAILABLE;
+		}
+	}
+
+	void nextSelection() {
+		if (inputSelector == null) {
+			inputSelection = InputSelection.ALL;
 
 Review comment:
   Can we instead of null add something like
   ```
   public interface InputSelectable {
       InputSelectable ALL = () -> InputSelection.ALL;
   ```
   and then use it in this class?

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380787617
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
+					i));
+		}
+
+		this.operatorChain = checkNotNull(operatorChain);
+	}
+
+	@Override
+	public CompletableFuture<?> getAvailableFuture() {
+		if (inputSelectionHandler.areAllInputsSelected()) {
+			return isAnyInputAvailable();
+		} else {
+			throw new UnsupportedOperationException();
+		}
 
 Review comment:
   Could you please explain why is it not supported?

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384338237
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.streaming.api.operators.InputSelectable;
+import org.apache.flink.streaming.api.operators.InputSelection;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This handler is mainly used for selecting the next available input index
+ * in {@link StreamMultipleInputProcessor}.
+ */
+@Internal
+public class MultipleInputSelectionHandler {
+
+	@Nullable
+	private final InputSelectable inputSelector;
+
+	private InputSelection inputSelection = InputSelection.ALL;
+
+	private final long allSelectedMask;
+
+	private long availableInputsMask;
+
+	private long endOfInputMask;
+
+	public MultipleInputSelectionHandler(@Nullable InputSelectable inputSelectable, int inputCount) {
+		this.inputSelector = inputSelectable;
+		this.allSelectedMask = (1 << inputCount) - 1;
+		this.availableInputsMask = allSelectedMask;
+	}
+
+	public InputStatus reportInputStatus(InputStatus inputStatus, int inputIndex) throws IOException {
 
 Review comment:
   Yes and no. Yes, it's a bit easier to understand, but no, as it creates opportunity for code duplication. On top of that, there is this hot looping argument, so I would leave it as it is, with @AHeise 's suggestion to rename to `updateStatus`.

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383858457
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
 ##########
 @@ -637,19 +638,24 @@ public StreamNode getStreamNode(Integer vertexID) {
 		return streamNodes.keySet();
 	}
 
+	@VisibleForTesting
 	public List<StreamEdge> getStreamEdges(int sourceId, int targetId) {
-
 		List<StreamEdge> result = new ArrayList<>();
 		for (StreamEdge edge : getStreamNode(sourceId).getOutEdges()) {
 			if (edge.getTargetId() == targetId) {
 				result.add(edge);
 			}
 		}
+		return result;
+	}
 
+	@VisibleForTesting
+	@Deprecated
+	public List<StreamEdge> getStreamEdgesOrThrow(int sourceId, int targetId) {
 
 Review comment:
   Like I said, we should move the old behavior into the tests. If we don't do it now, it's probably okay.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383418442
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
+					i));
+		}
+
+		this.operatorChain = checkNotNull(operatorChain);
+	}
+
+	@Override
+	public CompletableFuture<?> getAvailableFuture() {
+		if (inputSelectionHandler.areAllInputsSelected()) {
+			return isAnyInputAvailable();
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	@Override
+	public InputStatus processInput() throws Exception {
+		int readingInputIndex;
+		if (isPrepared) {
+			readingInputIndex = selectNextReadingInputIndex();
+		} else {
+			// the preparations here are not placed in the constructor because all work in it
+			// must be executed after all operators are opened.
+			readingInputIndex = selectFirstReadingInputIndex();
 
 Review comment:
   +/- 0 from my side. Complicating the interface of the class vs a tinny implementation hiccup ?

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383172192
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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.transformations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This Transformation represents the application of a
+ * {@link org.apache.flink.streaming.api.operators.MultipleInputStreamOperator}
+ * to input {@code Transformations}. The result is again only one stream.
+ *
+ * @param <OUT> The type of the elements that result from this {@code MultipleInputTransformation}
+ */
+@Internal
+public class MultipleInputTransformation<OUT> extends PhysicalTransformation<OUT> {
+
+	private final ArrayList<Transformation<?>> inputs = new ArrayList<>();
+
+	private final StreamOperatorFactory<OUT> operatorFactory;
+
+	private final ArrayList<KeySelector<?, ?>> stateKeySelectors = new ArrayList<>();
 
 Review comment:
   `List<KeySelector<?, ?>> stateKeySelectors`

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150614271",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150672723",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150856839",
       "triggerID" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5701",
       "triggerID" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8e56fa6affa8e58ed6ad9233345c7c5b697de4ea Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150672723) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646) 
   * b85332b20e2765101e28eca8ac45584ab57c77a0 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150856839) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5701) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383187455
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
 ##########
 @@ -63,40 +63,57 @@ public static CheckpointedInputGate createCheckpointedInputGate(
 	public static CheckpointedInputGate[] createCheckpointedInputGatePair(
 			AbstractInvokable toNotifyOnCheckpoint,
 			CheckpointingMode checkpointMode,
-			InputGate inputGate1,
-			InputGate inputGate2,
 			Configuration taskManagerConfig,
 			TaskIOMetricGroup taskIOMetricGroup,
-			String taskName) {
+			String taskName,
+			InputGate ...inputGates) {
 
 		int pageSize = ConfigurationParserUtils.getPageSize(taskManagerConfig);
 
-		BufferStorage mainBufferStorage1 = createBufferStorage(
-			checkpointMode, pageSize, taskManagerConfig, taskName);
-		BufferStorage mainBufferStorage2 = createBufferStorage(
-			checkpointMode, pageSize, taskManagerConfig, taskName);
-		checkState(mainBufferStorage1.getMaxBufferedBytes() == mainBufferStorage2.getMaxBufferedBytes());
+		BufferStorage[] mainBufferStorages = new BufferStorage[inputGates.length];
+		for (int i = 0; i < inputGates.length; i++) {
+			mainBufferStorages[i] = createBufferStorage(
+				checkpointMode, pageSize, taskManagerConfig, taskName);
+		}
+
+		BufferStorage[] linkedBufferStorages = new BufferStorage[inputGates.length];
 
-		BufferStorage linkedBufferStorage1 = new LinkedBufferStorage(
-			mainBufferStorage1,
-			mainBufferStorage2,
-			mainBufferStorage1.getMaxBufferedBytes());
-		BufferStorage linkedBufferStorage2 = new LinkedBufferStorage(
-			mainBufferStorage2,
-			mainBufferStorage1,
-			mainBufferStorage1.getMaxBufferedBytes());
+		for (int i = 0; i < inputGates.length; i++) {
+			linkedBufferStorages[i] = new LinkedBufferStorage(
+				mainBufferStorages[i],
+				mainBufferStorages[i].getMaxBufferedBytes(),
+				copyBufferStoragesExceptOf(i, mainBufferStorages));
+		}
 
 		CheckpointBarrierHandler barrierHandler = createCheckpointBarrierHandler(
 			checkpointMode,
-			inputGate1.getNumberOfInputChannels() + inputGate2.getNumberOfInputChannels(),
+			Arrays.stream(inputGates).mapToInt(InputGate::getNumberOfInputChannels).sum(),
 			taskName,
 			toNotifyOnCheckpoint);
 		registerCheckpointMetrics(taskIOMetricGroup, barrierHandler);
 
-		return new CheckpointedInputGate[] {
-			new CheckpointedInputGate(inputGate1, linkedBufferStorage1, barrierHandler),
-			new CheckpointedInputGate(inputGate2, linkedBufferStorage2, barrierHandler, inputGate1.getNumberOfInputChannels())
-		};
+		CheckpointedInputGate[] checkpointedInputGates = new CheckpointedInputGate[inputGates.length];
+
+		int channelIndexOffset = 0;
+		for (int i = 0; i < inputGates.length; i++) {
+			checkpointedInputGates[i] = new CheckpointedInputGate(inputGates[i], linkedBufferStorages[i], barrierHandler, channelIndexOffset);
+			channelIndexOffset += inputGates[i].getNumberOfInputChannels();
+		}
+
+		return checkpointedInputGates;
+	}
+
+	private static BufferStorage[] copyBufferStoragesExceptOf(
+			int skipStorage,
+			BufferStorage[] bufferStorages) {
+		BufferStorage[] copy = new BufferStorage[bufferStorages.length - 1];
+		int copyTo = 0;
 
 Review comment:
   nit declare in loop.

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

[GitHub] [flink] pnowojski commented on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-587351638
 
 
   Test failures are unrelated. Travis was already fixed on latest master while azure is a[ new one ](https://issues.apache.org/jira/browse/FLINK-16134)

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149077809 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149077809) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384354604
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ##########
 @@ -484,11 +485,15 @@ public final void invoke() throws Exception {
 		}
 	}
 
+	protected boolean runMailboxStep() throws Exception {
+		return mailboxProcessor.runMailboxStep();
+	}
+
 
 Review comment:
   Yes, we would have to delegate all methods.
   (I agree it can be overkill)

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383409342
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InputSelection.java
 ##########
 @@ -110,6 +110,41 @@ public int fairSelectNextIndexOutOf2(int availableInputsMask, int lastReadInputI
 		throw new UnsupportedOperationException("Only two inputs are supported.");
 	}
 
+	/**
+	 * Fairly select one of the available inputs for reading.
+	 *
+	 * @param availableInputsMask The mask of all available inputs.
+	 * @param lastReadInputIndex The index of last read input.
+	 * @return the index of the input for reading or -1, and -1 indicates no input is selected (
+	 *         {@code inputMask} is empty or the inputs in {@code inputMask} are unavailable).
+	 */
+	public int fairSelectNextIndex(int availableInputsMask, int lastReadInputIndex) {
 
 Review comment:
   I've forgotten about a TODO in the code to investigate it. I think we should be fine by supporting longs. Previously for two input case it was `int`, as  it was max 2.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150614271",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 352eaf94ac42b2428e2f2f90081e7a023136d9fd Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150614271) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383172141
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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.transformations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This Transformation represents the application of a
+ * {@link org.apache.flink.streaming.api.operators.MultipleInputStreamOperator}
+ * to input {@code Transformations}. The result is again only one stream.
+ *
+ * @param <OUT> The type of the elements that result from this {@code MultipleInputTransformation}
+ */
+@Internal
+public class MultipleInputTransformation<OUT> extends PhysicalTransformation<OUT> {
+
+	private final ArrayList<Transformation<?>> inputs = new ArrayList<>();
 
 Review comment:
   List<Transformation<?>> inputs

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383402906
 
 

 ##########
 File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
 ##########
 @@ -633,4 +671,29 @@ protected boolean matchesSafely(ResourceSpec item) {
 			return resources.lessThanOrEqual(item) && item.lessThanOrEqual(resources);
 		}
 	}
+
+	private static class MultipleInputOperatorFactory implements StreamOperatorFactory<String> {
 
 Review comment:
   This is not using `SimpleOperatorFactory`, isn't it?

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382674662
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InputSelection.java
 ##########
 @@ -110,8 +101,39 @@ public int fairSelectNextIndexOutOf2(int availableInputsMask, int lastReadInputI
 		throw new UnsupportedOperationException("Only two inputs are supported.");
 	}
 
-	private static boolean isALLMaskOf2(long inputMask) {
-		return (3 & inputMask) == 3;
+	/**
+	 * Fairly select one of the available inputs for reading.
+	 *
+	 * @param availableInputsMask The mask of all available inputs.
+	 * @param lastReadInputIndex The index of last read input.
+	 * @return the index of the input for reading or -1, and -1 indicates no input is selected (
+	 *         {@code inputMask} is empty or the inputs in {@code inputMask} are unavailable).
+	 */
+	public int fairSelectNextIndex(int availableInputsMask, int lastReadInputIndex) {
+		int selectionMask = (int) inputMask;
+		int combineMask = availableInputsMask & selectionMask;
 
 Review comment:
   Good question, I had to double check that. `availableInputsMask` could only be set to `-1` to indicate all 32 inputs are available. I will document that. `MultipleInputSelectionHandler` is adhering to this contract.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149077809 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149160727 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149077809) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195) 
   * ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149160727) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot commented on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 68393ef701e7b462638a26e89df167fa4a36ec14 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150438127) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551) 
   * 352eaf94ac42b2428e2f2f90081e7a023136d9fd UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380699709
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/LinkedBufferStorage.java
 ##########
 @@ -56,17 +57,27 @@ public boolean isFull() {
 	@Override
 	public void rollOver() {
 		mainStorage.rollOver();
-		linkedStorage.rollOver();
+		for (BufferStorage linked : linkedStorage) {
+			linked.rollOver();
+		}
 	}
 
 	@Override
 	public long getPendingBytes() {
-		return mainStorage.getPendingBytes() + linkedStorage.getPendingBytes();
+		long pendingBytes = mainStorage.getPendingBytes();
+		for (BufferStorage linked : linkedStorage) {
+			pendingBytes += linked.getPendingBytes();
 
 Review comment:
   nit: mark the class with `@NotThreadSafe`?

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380691220
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
 ##########
 @@ -63,40 +63,57 @@ public static CheckpointedInputGate createCheckpointedInputGate(
 	public static CheckpointedInputGate[] createCheckpointedInputGatePair(
 			AbstractInvokable toNotifyOnCheckpoint,
 			CheckpointingMode checkpointMode,
-			InputGate inputGate1,
-			InputGate inputGate2,
 			Configuration taskManagerConfig,
 			TaskIOMetricGroup taskIOMetricGroup,
-			String taskName) {
+			String taskName,
+			InputGate ...inputGates) {
 
 		int pageSize = ConfigurationParserUtils.getPageSize(taskManagerConfig);
 
-		BufferStorage mainBufferStorage1 = createBufferStorage(
-			checkpointMode, pageSize, taskManagerConfig, taskName);
-		BufferStorage mainBufferStorage2 = createBufferStorage(
-			checkpointMode, pageSize, taskManagerConfig, taskName);
-		checkState(mainBufferStorage1.getMaxBufferedBytes() == mainBufferStorage2.getMaxBufferedBytes());
+		BufferStorage[] mainBufferStorages = new BufferStorage[inputGates.length];
+		for (int i = 0; i < inputGates.length; i++) {
+			mainBufferStorages[i] = createBufferStorage(
+				checkpointMode, pageSize, taskManagerConfig, taskName);
+		}
+
+		BufferStorage[] linkedBufferStorages = new BufferStorage[inputGates.length];
 
-		BufferStorage linkedBufferStorage1 = new LinkedBufferStorage(
-			mainBufferStorage1,
-			mainBufferStorage2,
-			mainBufferStorage1.getMaxBufferedBytes());
-		BufferStorage linkedBufferStorage2 = new LinkedBufferStorage(
-			mainBufferStorage2,
-			mainBufferStorage1,
-			mainBufferStorage1.getMaxBufferedBytes());
+		for (int i = 0; i < inputGates.length; i++) {
+			linkedBufferStorages[i] = new LinkedBufferStorage(
+				mainBufferStorages[i],
+				mainBufferStorages[i].getMaxBufferedBytes(),
+				copyBufferStoragesExceptOf(i, mainBufferStorages));
+		}
 
 		CheckpointBarrierHandler barrierHandler = createCheckpointBarrierHandler(
 			checkpointMode,
-			inputGate1.getNumberOfInputChannels() + inputGate2.getNumberOfInputChannels(),
+			Arrays.stream(inputGates).mapToInt(InputGate::getNumberOfInputChannels).sum(),
 			taskName,
 			toNotifyOnCheckpoint);
 		registerCheckpointMetrics(taskIOMetricGroup, barrierHandler);
 
-		return new CheckpointedInputGate[] {
-			new CheckpointedInputGate(inputGate1, linkedBufferStorage1, barrierHandler),
-			new CheckpointedInputGate(inputGate2, linkedBufferStorage2, barrierHandler, inputGate1.getNumberOfInputChannels())
-		};
+		CheckpointedInputGate[] checkpointedInputGates = new CheckpointedInputGate[inputGates.length];
+
+		int channelIndexOffset = 0;
+		for (int i = 0; i < inputGates.length; i++) {
+			checkpointedInputGates[i] = new CheckpointedInputGate(inputGates[i], linkedBufferStorages[i], barrierHandler, channelIndexOffset);
+			channelIndexOffset += inputGates[i].getNumberOfInputChannels();
+		}
+
+		return checkpointedInputGates;
+	}
+
+	private static BufferStorage[] copyBufferStoragesExceptOf(
+			int skipStorage,
+			BufferStorage[] bufferStorages) {
+		BufferStorage[] copy = new BufferStorage[bufferStorages.length - 1];
+		int copyTo = 0;
+		for (int copyFrom = 0; copyFrom < bufferStorages.length; copyFrom++) {
+			if (copyFrom != skipStorage) {
+				copy[copyTo++] = bufferStorages[copyFrom];
+			}
+		}
+		return copy;
 
 Review comment:
   Can we improve it with two `arrayCopy` calls:
   ```
   System.arraycopy(bufferStorages, 0, copy, 0, skipStorage);
   System.arraycopy(bufferStorages, skipStorage + 1, copy, skipStorage, bufferStorages.length - skipStorage - 1);
   ```
   ?
   Also, variable names don't say much to me (just `src`, `dst` and `i` would be better).

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384339354
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
 ##########
 @@ -637,19 +638,24 @@ public StreamNode getStreamNode(Integer vertexID) {
 		return streamNodes.keySet();
 	}
 
+	@VisibleForTesting
 	public List<StreamEdge> getStreamEdges(int sourceId, int targetId) {
-
 		List<StreamEdge> result = new ArrayList<>();
 		for (StreamEdge edge : getStreamNode(sourceId).getOutEdges()) {
 			if (edge.getTargetId() == targetId) {
 				result.add(edge);
 			}
 		}
+		return result;
+	}
 
+	@VisibleForTesting
+	@Deprecated
+	public List<StreamEdge> getStreamEdgesOrThrow(int sourceId, int targetId) {
 
 Review comment:
   Yes, we should. As I wrote, the old method shouldn't exists. At least here I renamed it so it is not so confusing :/
   
   But I didn't want to spend even more time cleaning up unrelated old code (because I was cleaning up/refactoring already so much).

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149077809 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149160727 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:e9472785fa25680b297986397e3c732875d4ccff Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149163748 TriggerType:PUSH TriggerID:e9472785fa25680b297986397e3c732875d4ccff
   Hash:e9472785fa25680b297986397e3c732875d4ccff Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219 TriggerType:PUSH TriggerID:e9472785fa25680b297986397e3c732875d4ccff
   Hash:82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149077809) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195) 
   * ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149160727) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217) 
   * e9472785fa25680b297986397e3c732875d4ccff Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149163748) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219) 
   * 82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r384115214
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
 ##########
 @@ -735,6 +739,56 @@ public StreamGraph generate() {
 		return Collections.singleton(transform.getId());
 	}
 
+	private <OUT> Collection<Integer> transformMultipleInputTransform(MultipleInputTransformation<OUT> transform) {
+		List<Collection<Integer>> allInputIds = new ArrayList<>();
+
+		for (Transformation<?> input : transform.getInputs()) {
+			allInputIds.add(transform(input));
+		}
+
+		// the recursive call might have already transformed this
+		if (alreadyTransformed.containsKey(transform)) {
+			return alreadyTransformed.get(transform);
+		}
+
 
 Review comment:
   Or maybe because of mutability of `Transformation`?
   In either case, I wouldn't spend time on it because the only impact I see is this confusion.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383186216
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
 
 Review comment:
   Hmmm, indeed we do. I've noticed this behaviour when writing a unit test and was wondering where does it come from, but in the end, it made me realise that neither the production code nor the tests should relay on such behaviour, so I think this is just fine. 

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149269902) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149269902) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244) 
   * ac7b323867f647d50b170c23df6c6052b2c08d61 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150614271",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150672723",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150856839",
       "triggerID" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5701",
       "triggerID" : "b85332b20e2765101e28eca8ac45584ab57c77a0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b85332b20e2765101e28eca8ac45584ab57c77a0 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150856839) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5701) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383860044
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java
 ##########
 @@ -183,9 +183,20 @@ public void runMailboxLoop() throws Exception {
 
 		final MailboxController defaultActionContext = new MailboxController(this);
 
-		while (processMail(localMailbox)) {
+		while (runMailboxStep(localMailbox, defaultActionContext)) {
+		}
+	}
+
+	public boolean runMailboxStep() throws Exception {
 
 Review comment:
   I don't mind. It's non-public API to begin with.

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383189009
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/MinWatermarkGauge.java
 ##########
 @@ -20,21 +20,23 @@
 
 import org.apache.flink.metrics.Gauge;
 
 Review comment:
   Again not a hotfix imho.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383417322
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java
 ##########
 @@ -42,8 +42,11 @@ private MetricNames() {
 	public static final String IO_NUM_BUFFERS_OUT_RATE = IO_NUM_BUFFERS_OUT + SUFFIX_RATE;
 
 	public static final String IO_CURRENT_INPUT_WATERMARK = "currentInputWatermark";
+	@Deprecated
 	public static final String IO_CURRENT_INPUT_1_WATERMARK = "currentInput1Watermark";
+	@Deprecated
 	public static final String IO_CURRENT_INPUT_2_WATERMARK = "currentInput2Watermark";
+	public static final String IO_CURRENT_INPUT_WATERMARK_PATERN = "currentInput%dWatermark";
 
 Review comment:
   As previously, even if we re ordered it as prefix, patterns carry more information and are more concrete/well defined. For example, if you have prefix, does it mean that there is no suffix or other infix? Pattern mitigates this ambiguity.

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r385112918
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/MultipleConnectedStreams.java
 ##########
 @@ -0,0 +1,47 @@
+/*
+ * 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.datastream;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.transformations.MultipleInputTransformation;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This is a very basic and rough stub for a class connecting multiple input {@link DataStream}s
+ * into one, using {@link MultipleInputStreamOperator}.
+ */
+@Experimental
+public class MultipleConnectedStreams {
+
+	protected final StreamExecutionEnvironment environment;
+
+	public MultipleConnectedStreams(StreamExecutionEnvironment env) {
+		this.environment = requireNonNull(env);
+	}
+
+	public StreamExecutionEnvironment getExecutionEnvironment() {
+		return environment;
+	}
+
+	public <OUT> SingleOutputStreamOperator<OUT> doTransform(MultipleInputTransformation<OUT> transform) {
 
 Review comment:
   Should be `transform`. `doTransform` would be the private implementation.

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

[GitHub] [flink] pnowojski merged pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski merged pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098
 
 
   

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383381055
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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.transformations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This Transformation represents the application of a
+ * {@link org.apache.flink.streaming.api.operators.MultipleInputStreamOperator}
+ * to input {@code Transformations}. The result is again only one stream.
+ *
+ * @param <OUT> The type of the elements that result from this {@code MultipleInputTransformation}
+ */
+@Internal
+public class MultipleInputTransformation<OUT> extends PhysicalTransformation<OUT> {
+
+	private final ArrayList<Transformation<?>> inputs = new ArrayList<>();
+
+	private final StreamOperatorFactory<OUT> operatorFactory;
+
+	private final ArrayList<KeySelector<?, ?>> stateKeySelectors = new ArrayList<>();
+
+	private TypeInformation<?> stateKeyType;
+
+	public MultipleInputTransformation(
+			String name,
+			StreamOperatorFactory<OUT> operatorFactory,
+			TypeInformation<OUT> outputType,
+			int parallelism) {
+		super(name, outputType, parallelism);
+		this.operatorFactory = operatorFactory;
+	}
+
+	public List<Transformation<?>> getInputs() {
+		return inputs;
+	}
+
+	/**
+	 * Returns the {@code TypeInformation} for the elements from the inputs.
+	 */
+	public List<TypeInformation<?>> getInputTypes() {
+		return inputs.stream()
+			.map(Transformation::getOutputType)
+			.collect(Collectors.toList());
+	}
+
+	/**
+	 * Returns the {@code StreamOperatorFactory} of this Transformation.
+	 */
+	public StreamOperatorFactory<OUT> getOperatorFactory() {
+		return operatorFactory;
+	}
+
+	public void addInput(Transformation<?> input) {
+		checkState(
+			stateKeySelectors.isEmpty(),
+			"Trying to add non-keyed input to keyed transformation.");
+		inputs.add(input);
+	}
+
+	public void addInput(Transformation<?> input, KeySelector<?, ?> keySelector) {
+		checkState(
+			stateKeySelectors.size() == inputs.size(),
+			"Trying to add keyed input to non-keyed transformation");
+		inputs.add(input);
+		stateKeySelectors.add(keySelector);
+	}
+
+	public List<KeySelector<?, ?>> getStateKeySelectors() {
+		return stateKeySelectors;
+	}
+
+	public void setStateKeyType(TypeInformation<?> stateKeyType) {
+		this.stateKeyType = stateKeyType;
+	}
+
+	public TypeInformation<?> getStateKeyType() {
+		return stateKeyType;
+	}
+
+	@Override
+	public Collection<Transformation<?>> getTransitivePredecessors() {
+		List<Transformation<?>> result = Lists.newArrayList();
 
 Review comment:
   Bah, I don't like going back and forth between collections and streams. Especially in flatMap. But ok ;)

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380678105
 
 

 ##########
 File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
 ##########
 @@ -312,6 +316,40 @@ public void testOutputTypeConfigurationWithTwoInputTransformation() throws Excep
 		assertEquals(BasicTypeInfo.INT_TYPE_INFO, outputTypeConfigurableOperation.getTypeInformation());
 	}
 
+	@Test
+	public void testMultipleInputTransformation() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		//TODO: referring to getNewNodeId() is hacky, but I'm not sure if there is a better idea?
+		int startingNewNodeId = Transformation.getNewNodeId();
+
+		DataStream<Integer> source1 = env.fromElements(1, 10);
+		DataStream<Long> source2 = env.fromElements(2L, 11L);
+		DataStream<String> source3 = env.fromElements("42", "44");
+
+		MultipleInputTransformation<String> transform = new MultipleInputTransformation<String>(
+			"My Operator",
+			new MultipleInputOperatorFactory(),
+			BasicTypeInfo.STRING_TYPE_INFO,
+			3);
+
+		transform.addInput(source1.getTransformation());
+		transform.addInput(source2.getTransformation());
+		transform.addInput(source3.getTransformation());
+
+		env.addOperator(transform);
+		StreamGraph streamGraph = env.getStreamGraph();
+		assertEquals(4, streamGraph.getStreamNodes().size());
+
+		int id = startingNewNodeId;
+		assertEquals(1, streamGraph.getStreamEdges(id + 1, id + 4).size());
+		assertEquals(1, streamGraph.getStreamEdges(id + 2, id + 4).size());
+		assertEquals(1, streamGraph.getStreamEdges(id + 3, id + 4).size());
+		assertEquals(1, streamGraph.getStreamEdges(id + 1).size());
+		assertEquals(1, streamGraph.getStreamEdges(id + 2).size());
+		assertEquals(1, streamGraph.getStreamEdges(id + 3).size());
+		assertEquals(0, streamGraph.getStreamEdges(id + 4).size());
 
 Review comment:
   Why can't we use `Transformation#getId` here? E.g.
   ```
   assertEquals(1, streamGraph.getStreamEdges(transform.getId(), source1.getId()).size());
   ```

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383198531
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java
 ##########
 @@ -183,9 +183,20 @@ public void runMailboxLoop() throws Exception {
 
 		final MailboxController defaultActionContext = new MailboxController(this);
 
-		while (processMail(localMailbox)) {
+		while (runMailboxStep(localMailbox, defaultActionContext)) {
+		}
+	}
+
+	public boolean runMailboxStep() throws Exception {
 
 Review comment:
   Visible for testing.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382667265
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/LinkedBufferStorage.java
 ##########
 @@ -56,17 +57,27 @@ public boolean isFull() {
 	@Override
 	public void rollOver() {
 		mainStorage.rollOver();
-		linkedStorage.rollOver();
+		for (BufferStorage linked : linkedStorage) {
+			linked.rollOver();
+		}
 	}
 
 	@Override
 	public long getPendingBytes() {
-		return mainStorage.getPendingBytes() + linkedStorage.getPendingBytes();
+		long pendingBytes = mainStorage.getPendingBytes();
+		for (BufferStorage linked : linkedStorage) {
+			pendingBytes += linked.getPendingBytes();
 
 Review comment:
   Usually we treat most of the classes `@NotThreadSafe` by default and all/almost all of the classes in this package are not thread safe. 

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383392154
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
 ##########
 @@ -735,6 +739,56 @@ public StreamGraph generate() {
 		return Collections.singleton(transform.getId());
 	}
 
+	private <OUT> Collection<Integer> transformMultipleInputTransform(MultipleInputTransformation<OUT> transform) {
+		List<Collection<Integer>> allInputIds = new ArrayList<>();
+
+		for (Transformation<?> input : transform.getInputs()) {
+			allInputIds.add(transform(input));
+		}
+
+		// the recursive call might have already transformed this
+		if (alreadyTransformed.containsKey(transform)) {
+			return alreadyTransformed.get(transform);
+		}
+
 
 Review comment:
   I'm not sure. I was mostly copying the logic from `transformTwoInputTransform` (and also the same pattern is in `transformOneInputTransform` as well).
   
   However it looks like this might be because of cyclic dependencies. 
   
   Do you think this is a bug in all of those places?

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383178835
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
 ##########
 @@ -73,8 +73,8 @@
 	private static final String ITERATION_ID = "iterationId";
 	private static final String OUTPUT_SELECTOR_WRAPPER = "outputSelectorWrapper";
 	private static final String BUFFER_TIMEOUT = "bufferTimeout";
-	private static final String TYPE_SERIALIZER_IN_1 = "typeSerializer_in_1";
-	private static final String TYPE_SERIALIZER_IN_2 = "typeSerializer_in_2";
+	private static final String TYPE_SERIALIZERS_IN_COUNT = "typeSerializer_in_count";
+	private static final String TYPE_SERIALIZERS_IN_PATTERN = "typeSerializer_in_%d";
 
 Review comment:
   PREFIX would be good enough, see TYPE_SERIALIZER_SIDEOUT_PREFIX.

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383169987
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Input.java
 ##########
 @@ -0,0 +1,32 @@
+/*
+ * 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;
+
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+/**
+ * {@link Input} interface used in {@link MultipleInputStreamOperator}.
+ */
+public interface Input<IN> {
+	/**
+	 * Processes one element that arrived on the first input of this two-input operator.
 
 Review comment:
   n-input?

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "311d3e9843bd601a8de8bee78c2ecd34222d19d6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "311d3e9843bd601a8de8bee78c2ecd34222d19d6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149077809) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195) 
   * ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149160727) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217) 
   * e9472785fa25680b297986397e3c732875d4ccff Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149163748) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219) 
   * 82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149269902) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383414666
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java
 ##########
 @@ -183,9 +183,20 @@ public void runMailboxLoop() throws Exception {
 
 		final MailboxController defaultActionContext = new MailboxController(this);
 
-		while (processMail(localMailbox)) {
+		while (runMailboxStep(localMailbox, defaultActionContext)) {
+		}
+	}
+
+	public boolean runMailboxStep() throws Exception {
 
 Review comment:
   I think this could be part of the public api for this class.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 68393ef701e7b462638a26e89df167fa4a36ec14 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150438127) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ac7b323867f647d50b170c23df6c6052b2c08d61 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150350672) 
   * 68393ef701e7b462638a26e89df167fa4a36ec14 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383177132
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
 ##########
 @@ -637,19 +638,24 @@ public StreamNode getStreamNode(Integer vertexID) {
 		return streamNodes.keySet();
 	}
 
+	@VisibleForTesting
 	public List<StreamEdge> getStreamEdges(int sourceId, int targetId) {
-
 		List<StreamEdge> result = new ArrayList<>();
 		for (StreamEdge edge : getStreamNode(sourceId).getOutEdges()) {
 			if (edge.getTargetId() == targetId) {
 				result.add(edge);
 			}
 		}
+		return result;
+	}
 
+	@VisibleForTesting
+	@Deprecated
+	public List<StreamEdge> getStreamEdgesOrThrow(int sourceId, int targetId) {
 
 Review comment:
   Not sure what this whole change is about. Better commit message? Why is the new method deprecated already?
   
   Usually when I read `orThrow` I except some customizable error; see `Optional#orElseThrow(Supplier<X> exceptionSupplier)`.
   
   I'd probably just add this methods to the test as 
   ```
   public static List<StreamEdge> getNonEmptyStreamEdges(Environment env, int sourceId, int targetId) { ... }
   ```
   It just adds convenience for writing test.  `@VisibleForTesting` should be about giving access to something that is inaccessible.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 68393ef701e7b462638a26e89df167fa4a36ec14 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150438127) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380776082
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
+
+	private boolean isPrepared;
+
+	public StreamMultipleInputProcessor(
+			CheckpointedInputGate[] checkpointedInputGates,
+			TypeSerializer<?>[] inputSerializers,
+			IOManager ioManager,
+			StreamStatusMaintainer streamStatusMaintainer,
+			MultipleInputStreamOperator<?> streamOperator,
+			MultipleInputSelectionHandler inputSelectionHandler,
+			WatermarkGauge[] inputWatermarkGauges,
+			OperatorChain<?, ?> operatorChain,
+			Counter numRecordsIn) {
+
+		this.inputSelectionHandler = checkNotNull(inputSelectionHandler);
+
+		List<Input> inputs = streamOperator.getInputs();
+		int operatorsCount = inputs.size();
+
+		this.inputProcessors = new InputProcessor[operatorsCount];
+		this.streamStatuses = new StreamStatus[operatorsCount];
+		this.numRecordsIn = numRecordsIn;
+
+		for (int i = 0; i < operatorsCount; i++) {
+			StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>(
+				inputs.get(i),
+				streamStatusMaintainer,
+				inputWatermarkGauges[i],
+				i);
+
+			inputProcessors[i] = new InputProcessor(
+				dataOutput,
+				new StreamTaskNetworkInput<>(
+					checkpointedInputGates[i],
+					inputSerializers[i],
+					ioManager,
+					new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), dataOutput),
 
 Review comment:
   Should it be `i` instead of `0`?

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "311d3e9843bd601a8de8bee78c2ecd34222d19d6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "311d3e9843bd601a8de8bee78c2ecd34222d19d6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * 82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149269902) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380557788
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
 ##########
 @@ -235,20 +234,16 @@ public void addOutputSelector(OutputSelector<?> outputSelector) {
 		this.outputSelectors.add(outputSelector);
 	}
 
-	public TypeSerializer<?> getTypeSerializerIn1() {
-		return typeSerializerIn1;
+	public void setSerializersIn(TypeSerializer<?> ...typeSerializersIn) {
 
 Review comment:
   With this signature, it's possible to call without any arguments.
   If it's not a legal state then it makes sense to check arguments (or better ensure at compile time).
   
   Also in other classes, e.g. `MinWatermarkGauge`.

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383396049
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
 ##########
 @@ -235,20 +234,16 @@ public void addOutputSelector(OutputSelector<?> outputSelector) {
 		this.outputSelectors.add(outputSelector);
 	}
 
-	public TypeSerializer<?> getTypeSerializerIn1() {
-		return typeSerializerIn1;
+	public void setSerializersIn(TypeSerializer<?> ...typeSerializersIn) {
 
 Review comment:
   It's not used in the sources. I think indeed the job graph generator could crash if this is empty. Also currently that could happen only with incorrectly constructed/implemented `MultipleInputOperator`, so I think it's a good idea to just disallow it. 

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380729971
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InputSelection.java
 ##########
 @@ -110,8 +101,39 @@ public int fairSelectNextIndexOutOf2(int availableInputsMask, int lastReadInputI
 		throw new UnsupportedOperationException("Only two inputs are supported.");
 	}
 
-	private static boolean isALLMaskOf2(long inputMask) {
-		return (3 & inputMask) == 3;
+	/**
+	 * Fairly select one of the available inputs for reading.
+	 *
+	 * @param availableInputsMask The mask of all available inputs.
+	 * @param lastReadInputIndex The index of last read input.
+	 * @return the index of the input for reading or -1, and -1 indicates no input is selected (
+	 *         {@code inputMask} is empty or the inputs in {@code inputMask} are unavailable).
+	 */
+	public int fairSelectNextIndex(int availableInputsMask, int lastReadInputIndex) {
+		int selectionMask = (int) inputMask;
+		int combineMask = availableInputsMask & selectionMask;
+
+		if (combineMask == 0) {
+			return -1;
+		}
+
+		int nextReadInputIndex = fairSelectFromRightBits(combineMask, lastReadInputIndex + 1);
+		if (nextReadInputIndex >= 0) {
+			return nextReadInputIndex;
+		}
+		return fairSelectFromRightBits(combineMask, 0);
+	}
+
+	private int fairSelectFromRightBits(int combineMask, int nextReadInputIndex) {
+		int rightBits = combineMask >> nextReadInputIndex;
+		while (rightBits > 0) {
+			if (rightBits % 2 == 1) {
 
 Review comment:
   Here, we check if the rightmost bit is set, right?
   In that case, masking would be faster and more readable:
   `rightBits & 1 == 1`

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

[GitHub] [flink] AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r383173609
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java
 ##########
 @@ -0,0 +1,125 @@
+/*
+ * 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.transformations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This Transformation represents the application of a
+ * {@link org.apache.flink.streaming.api.operators.MultipleInputStreamOperator}
+ * to input {@code Transformations}. The result is again only one stream.
+ *
+ * @param <OUT> The type of the elements that result from this {@code MultipleInputTransformation}
+ */
+@Internal
+public class MultipleInputTransformation<OUT> extends PhysicalTransformation<OUT> {
+
+	private final ArrayList<Transformation<?>> inputs = new ArrayList<>();
+
+	private final StreamOperatorFactory<OUT> operatorFactory;
+
+	private final ArrayList<KeySelector<?, ?>> stateKeySelectors = new ArrayList<>();
+
+	private TypeInformation<?> stateKeyType;
+
+	public MultipleInputTransformation(
+			String name,
+			StreamOperatorFactory<OUT> operatorFactory,
+			TypeInformation<OUT> outputType,
+			int parallelism) {
+		super(name, outputType, parallelism);
+		this.operatorFactory = operatorFactory;
+	}
+
+	public List<Transformation<?>> getInputs() {
+		return inputs;
+	}
+
+	/**
+	 * Returns the {@code TypeInformation} for the elements from the inputs.
+	 */
+	public List<TypeInformation<?>> getInputTypes() {
+		return inputs.stream()
+			.map(Transformation::getOutputType)
+			.collect(Collectors.toList());
+	}
+
+	/**
+	 * Returns the {@code StreamOperatorFactory} of this Transformation.
+	 */
+	public StreamOperatorFactory<OUT> getOperatorFactory() {
+		return operatorFactory;
+	}
+
+	public void addInput(Transformation<?> input) {
+		checkState(
+			stateKeySelectors.isEmpty(),
+			"Trying to add non-keyed input to keyed transformation.");
+		inputs.add(input);
+	}
+
+	public void addInput(Transformation<?> input, KeySelector<?, ?> keySelector) {
+		checkState(
+			stateKeySelectors.size() == inputs.size(),
+			"Trying to add keyed input to non-keyed transformation");
+		inputs.add(input);
+		stateKeySelectors.add(keySelector);
+	}
+
+	public List<KeySelector<?, ?>> getStateKeySelectors() {
+		return stateKeySelectors;
+	}
+
+	public void setStateKeyType(TypeInformation<?> stateKeyType) {
+		this.stateKeyType = stateKeyType;
+	}
+
+	public TypeInformation<?> getStateKeyType() {
+		return stateKeyType;
+	}
+
+	@Override
+	public Collection<Transformation<?>> getTransitivePredecessors() {
+		List<Transformation<?>> result = Lists.newArrayList();
 
 Review comment:
   Any reason to not use streams here?

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149077809 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149160727 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149077809) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195) 
   * ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149160727) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148994879",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187",
       "triggerID" : "df7f0151d94bc7705c87baf855ae3d8d57f7e463",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149078552",
       "triggerID" : "e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149077809",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195",
       "triggerID" : "8355093ce1ed0dab9985d9f522f3bcd97c66d016",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149160727",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217",
       "triggerID" : "ce0a2e4c48f33e7826d6f469c54f3ba342c9552d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149163748",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e9472785fa25680b297986397e3c732875d4ccff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219",
       "triggerID" : "e9472785fa25680b297986397e3c732875d4ccff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149269902",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5244",
       "triggerID" : "82937c8a0ec226cc90b29ac0f67f9abbb33c9e4f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150350672",
       "triggerID" : "ac7b323867f647d50b170c23df6c6052b2c08d61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150438127",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5551",
       "triggerID" : "68393ef701e7b462638a26e89df167fa4a36ec14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150614271",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5606",
       "triggerID" : "352eaf94ac42b2428e2f2f90081e7a023136d9fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150672723",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646",
       "triggerID" : "8e56fa6affa8e58ed6ad9233345c7c5b697de4ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8e56fa6affa8e58ed6ad9233345c7c5b697de4ea Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150672723) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5646) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149077809 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149077809) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195) 
   * ce0a2e4c48f33e7826d6f469c54f3ba342c9552d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382658325
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
 ##########
 @@ -581,6 +611,13 @@ public void setTwoInputStateKey(Integer vertexID, KeySelector<?, ?> keySelector1
 		node.setStateKeySerializer(keySerializer);
 	}
 
+	public void setMultipleInputStateKey(
+			int id,
+			List<KeySelector<?, ?>> stateKeySelectors,
+			TypeSerializer<?> keySerializer) {
+		throw new UnsupportedOperationException();
+	}
 
 Review comment:
   I dropped the method as a result of a [previous comment](https://github.com/apache/flink/pull/11098#discussion_r380549321)

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

[GitHub] [flink] pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r382680589
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InputSelection.java
 ##########
 @@ -110,8 +101,39 @@ public int fairSelectNextIndexOutOf2(int availableInputsMask, int lastReadInputI
 		throw new UnsupportedOperationException("Only two inputs are supported.");
 	}
 
-	private static boolean isALLMaskOf2(long inputMask) {
-		return (3 & inputMask) == 3;
+	/**
+	 * Fairly select one of the available inputs for reading.
+	 *
+	 * @param availableInputsMask The mask of all available inputs.
+	 * @param lastReadInputIndex The index of last read input.
+	 * @return the index of the input for reading or -1, and -1 indicates no input is selected (
+	 *         {@code inputMask} is empty or the inputs in {@code inputMask} are unavailable).
+	 */
+	public int fairSelectNextIndex(int availableInputsMask, int lastReadInputIndex) {
+		int selectionMask = (int) inputMask;
+		int combineMask = availableInputsMask & selectionMask;
+
+		if (combineMask == 0) {
+			return -1;
+		}
+
+		int nextReadInputIndex = fairSelectFromRightBits(combineMask, lastReadInputIndex + 1);
+		if (nextReadInputIndex >= 0) {
+			return nextReadInputIndex;
+		}
+		return fairSelectFromRightBits(combineMask, 0);
+	}
+
+	private int fairSelectFromRightBits(int combineMask, int nextReadInputIndex) {
 
 Review comment:
   I'm not sure if this alternative is more readable and I don't think it will be faster (compiler should be able to optimise it anyway AND branch prediction wouldn't work for one or two iterations anyway), but ok.

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149077809 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149160727 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:e9472785fa25680b297986397e3c732875d4ccff Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149163748 TriggerType:PUSH TriggerID:e9472785fa25680b297986397e3c732875d4ccff
   Hash:e9472785fa25680b297986397e3c732875d4ccff Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219 TriggerType:PUSH TriggerID:e9472785fa25680b297986397e3c732875d4ccff
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149077809) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195) 
   * ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149160727) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217) 
   * e9472785fa25680b297986397e3c732875d4ccff Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149163748) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5219) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:311d3e9843bd601a8de8bee78c2ecd34222d19d6 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:311d3e9843bd601a8de8bee78c2ecd34222d19d6
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149078552 TriggerType:PUSH TriggerID:e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149077809 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:8355093ce1ed0dab9985d9f522f3bcd97c66d016 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195 TriggerType:PUSH TriggerID:8355093ce1ed0dab9985d9f522f3bcd97c66d016
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149160727 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217 TriggerType:PUSH TriggerID:ce0a2e4c48f33e7826d6f469c54f3ba342c9552d
   Hash:e9472785fa25680b297986397e3c732875d4ccff Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:e9472785fa25680b297986397e3c732875d4ccff
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   * 311d3e9843bd601a8de8bee78c2ecd34222d19d6 UNKNOWN
   * e72b2a38b7ec718b44a14f98897dbaf22f9d0d0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149078552) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5196) 
   * 8355093ce1ed0dab9985d9f522f3bcd97c66d016 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149077809) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5195) 
   * ce0a2e4c48f33e7826d6f469c54f3ba342c9552d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149160727) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5217) 
   * e9472785fa25680b297986397e3c732875d4ccff UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#issuecomment-586328210
 
 
   <!--
   Meta data
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148994879 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   Hash:df7f0151d94bc7705c87baf855ae3d8d57f7e463 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187 TriggerType:PUSH TriggerID:df7f0151d94bc7705c87baf855ae3d8d57f7e463
   -->
   ## CI report:
   
   * df7f0151d94bc7705c87baf855ae3d8d57f7e463 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148994879) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5187) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

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

[GitHub] [flink] rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #11098: [FLINK-16060][task] Implement working StreamMultipleInputProcessor
URL: https://github.com/apache/flink/pull/11098#discussion_r380772843
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java
 ##########
 @@ -0,0 +1,309 @@
+/*
+ * 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.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Input processor for {@link MultipleInputStreamOperator}.
+ */
+@Internal
+public final class StreamMultipleInputProcessor implements StreamInputProcessor {
+
+	private final MultipleInputSelectionHandler inputSelectionHandler;
+
+	private final InputProcessor<?>[] inputProcessors;
+
+	private final OperatorChain<?, ?> operatorChain;
+
+	/**
+	 * Stream status for the two inputs. We need to keep track for determining when
+	 * to forward stream status changes downstream.
+	 */
+	private final StreamStatus[] streamStatuses;
+
+	private final Counter numRecordsIn;
+
+	/** Always try to read from the first input. */
+	private int lastReadInputIndex = 1;
 
 Review comment:
   Do we start effectively with 2?
   There is `lastReadInputIndex + 1` in `InputSelection`

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