dawidwys commented on a change in pull request #14216:
URL: https://github.com/apache/flink/pull/14216#discussion_r530362543



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/BroadcastConnectedStream.java
##########
@@ -57,8 +58,8 @@
 public class BroadcastConnectedStream<IN1, IN2> {
 
        private final StreamExecutionEnvironment environment;
-       private final DataStream<IN1> inputStream1;
-       private final BroadcastStream<IN2> inputStream2;
+       private final DataStream<IN1> nonBroadcastStream;

Review comment:
       I really appreciate the renaming!

##########
File path: 
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BroadcastStateITCase.java
##########
@@ -149,6 +153,45 @@ public long extractTimestamp(String element, long 
previousElementTimestamp) {
                env.execute();
        }
 
+       @Test
+       public void testBroadcastBatchTranslationThrowsException() throws 
Exception {
+               final MapStateDescriptor<Long, Long> utterDescriptor = new 
MapStateDescriptor<>(
+                               "broadcast-state", 
BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+               );
+
+               final List<Long> input = new ArrayList<>();
+               input.add(1L);
+               input.add(2L);
+               input.add(3L);
+
+               final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+               env.setRuntimeMode(RuntimeExecutionMode.BATCH);
+
+               final DataStream<Long> srcOne = env.fromCollection(input);
+               final DataStream<Long> srcTwo = env.fromCollection(input);
+               final BroadcastStream<Long> broadcast = 
srcTwo.broadcast(utterDescriptor);
+
+               srcOne.connect(broadcast).process(
+                               new BroadcastProcessFunction<Long, Long, 
Long>() {
+                                       @Override
+                                       public void processElement(Long value, 
ReadOnlyContext ctx, Collector<Long> out) {
+
+                                       }
+
+                                       @Override
+                                       public void 
processBroadcastElement(Long value, Context ctx, Collector<Long> out) {
+
+                                       }
+                               });
+
+               try {
+                       env.execute();
+                       fail("Execution should have failed during job graph 
creation.");

Review comment:
       How about we use the `ExpectedException` rule instead? I find it more 
concise. 
   
   ```
   @Rule
   public ExpectedException thrown = ExpectedException.none();
   
   thrown.expect(UnsupportedOperationException.class);
   thrown.expectMessage("The Broadcast State Pattern is not support in BATCH 
execution mode.");
   ```

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/BroadcastStateTransformation.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.datastream.BroadcastStream;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This is the transformation for the Broadcast State pattern. In a nutshell, 
this transformation
+ * allows to take a broadcasted (non-keyed) stream, connect it with another 
keyed or non-keyed
+ * stream, and apply a function on the resulting connected stream. This 
function will have access
+ * to all the elements that belong to the non-keyed, broadcasted side, as this 
is kept in Flink's
+ * state.
+ *
+ * <p>For more information see the
+ * <a 
href="https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/state/broadcast_state.html";>
+ *     Broadcast State Pattern documentation page</a>.
+ *
+ * @param <IN1> The type of the elements in the non-broadcasted input.
+ * @param <IN2> The type of the elements in the broadcasted input.
+ * @param <OUT> The type of the elements that result from this transformation.
+ */
+@Internal
+public class BroadcastStateTransformation<IN1, IN2, OUT> extends 
PhysicalTransformation<OUT> {
+
+       private final DataStream<IN1> inputStream;

Review comment:
       How about keeping `Transformations` here.
   
   The way it is now creates an architectural cyclic dependency: DataStream is 
an abstraction on top of Transformation, but BroadcastStateTransformation keeps 
DataStream as a field.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/BroadcastStateTransformation.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.datastream.BroadcastStream;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This is the transformation for the Broadcast State pattern. In a nutshell, 
this transformation
+ * allows to take a broadcasted (non-keyed) stream, connect it with another 
keyed or non-keyed
+ * stream, and apply a function on the resulting connected stream. This 
function will have access
+ * to all the elements that belong to the non-keyed, broadcasted side, as this 
is kept in Flink's
+ * state.
+ *
+ * <p>For more information see the
+ * <a 
href="https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/state/broadcast_state.html";>
+ *     Broadcast State Pattern documentation page</a>.
+ *
+ * @param <IN1> The type of the elements in the non-broadcasted input.
+ * @param <IN2> The type of the elements in the broadcasted input.
+ * @param <OUT> The type of the elements that result from this transformation.
+ */
+@Internal
+public class BroadcastStateTransformation<IN1, IN2, OUT> extends 
PhysicalTransformation<OUT> {
+
+       private final DataStream<IN1> inputStream;
+
+       private final BroadcastStream<IN2> broadcastStream;
+
+       private final StreamOperatorFactory<OUT> operatorFactory;
+
+       private final TypeInformation<?> stateKeyType;
+
+       private final KeySelector<IN1, ?> keySelector;
+
+       public BroadcastStateTransformation(
+                       final String name,
+                       final DataStream<IN1> inputStream,
+                       final BroadcastStream<IN2> broadcastStream,
+                       final StreamOperatorFactory<OUT> operatorFactory,
+                       final TypeInformation<OUT> outTypeInfo,
+                       final int parallelism) {
+               this(
+                               name,
+                               checkNotNull(inputStream),
+                               broadcastStream,
+                               operatorFactory,
+                               null,
+                               null,
+                               outTypeInfo,
+                               parallelism);
+       }
+
+       public BroadcastStateTransformation(
+                       final String name,
+                       final KeyedStream<IN1, ?> inputStream,
+                       final BroadcastStream<IN2> broadcastStream,
+                       final StreamOperatorFactory<OUT> operatorFactory,
+                       final TypeInformation<OUT> outTypeInfo,
+                       final int parallelism) {
+               this(
+                               name,
+                               checkNotNull(inputStream),
+                               broadcastStream,
+                               operatorFactory,
+                               inputStream.getKeyType(),
+                               inputStream.getKeySelector(),
+                               outTypeInfo,
+                               parallelism);
+       }
+
+       private BroadcastStateTransformation(
+                       final String name,
+                       final DataStream<IN1> inputStream,
+                       final BroadcastStream<IN2> broadcastStream,
+                       final StreamOperatorFactory<OUT> operatorFactory,
+                       final TypeInformation<?> keyType,
+                       final KeySelector<IN1, ?> keySelector,

Review comment:
       add `@Nullable` annotations

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/BroadcastStateTransformationTranslator.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.translators;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.streaming.api.graph.SimpleTransformationTranslator;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.apache.flink.streaming.api.graph.TransformationTranslator;
+import 
org.apache.flink.streaming.api.transformations.BroadcastStateTransformation;
+
+import java.util.Collection;
+import java.util.Collections;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@link TransformationTranslator} for the {@link 
BroadcastStateTransformation}.
+ *
+ * @param <IN1> The type of the elements in the non-broadcasted input of the 
{@link BroadcastStateTransformation}.
+ * @param <IN2> The type of the elements in the broadcasted input of the 
{@link BroadcastStateTransformation}.
+ * @param <OUT> The type of the elements that result from the {@link 
BroadcastStateTransformation}.
+ */
+@Internal
+public class BroadcastStateTransformationTranslator<IN1, IN2, OUT>
+               extends SimpleTransformationTranslator<OUT, 
BroadcastStateTransformation<IN1, IN2, OUT>>  {
+
+       @Override
+       protected Collection<Integer> translateForBatchInternal(
+                       final BroadcastStateTransformation<IN1, IN2, OUT> 
transformation,
+                       final Context context) {
+               throw new UnsupportedOperationException("The Broadcast State 
Pattern is not support in BATCH execution mode.");
+       }
+
+       @Override
+       protected Collection<Integer> translateForStreamingInternal(
+                       final BroadcastStateTransformation<IN1, IN2, OUT> 
transformation,

Review comment:
       Could we unify this between `BroadcastStateTransformationTranslator` and 
`TwoInputTransformationTranslator`? The entire method is identical in the two 
classes (minus checking for `keySelector` in a single input, but the version 
from `TwoInputTransformationTranslator` should work as well.)
   
   We did a similar thing with `AbstractOneInputTransformationTranslator`.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/BroadcastStateTransformation.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.datastream.BroadcastStream;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This is the transformation for the Broadcast State pattern. In a nutshell, 
this transformation
+ * allows to take a broadcasted (non-keyed) stream, connect it with another 
keyed or non-keyed
+ * stream, and apply a function on the resulting connected stream. This 
function will have access
+ * to all the elements that belong to the non-keyed, broadcasted side, as this 
is kept in Flink's
+ * state.
+ *
+ * <p>For more information see the
+ * <a 
href="https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/state/broadcast_state.html";>
+ *     Broadcast State Pattern documentation page</a>.
+ *
+ * @param <IN1> The type of the elements in the non-broadcasted input.
+ * @param <IN2> The type of the elements in the broadcasted input.
+ * @param <OUT> The type of the elements that result from this transformation.
+ */
+@Internal
+public class BroadcastStateTransformation<IN1, IN2, OUT> extends 
PhysicalTransformation<OUT> {
+
+       private final DataStream<IN1> inputStream;
+
+       private final BroadcastStream<IN2> broadcastStream;
+
+       private final StreamOperatorFactory<OUT> operatorFactory;
+
+       private final TypeInformation<?> stateKeyType;
+
+       private final KeySelector<IN1, ?> keySelector;
+
+       public BroadcastStateTransformation(
+                       final String name,
+                       final DataStream<IN1> inputStream,
+                       final BroadcastStream<IN2> broadcastStream,
+                       final StreamOperatorFactory<OUT> operatorFactory,
+                       final TypeInformation<OUT> outTypeInfo,
+                       final int parallelism) {
+               this(
+                               name,
+                               checkNotNull(inputStream),
+                               broadcastStream,
+                               operatorFactory,
+                               null,
+                               null,
+                               outTypeInfo,
+                               parallelism);
+       }
+
+       public BroadcastStateTransformation(

Review comment:
       I am not the biggest fan of having multiple constructors. Having a 
single entry point to a class makes it easier to reason about its fields state. 
Could we change the ctor to static methods? Or even just expose the actual ctor 
and move the keyselector extraction to the `KeyedStream`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to