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 2021/10/20 06:53:24 UTC

[GitHub] [flink-ml] guoweiM commented on a change in pull request #6: [FLINK-2][iteration] Add broadcast output to broadcast events to all the downstream tasks

guoweiM commented on a change in pull request #6:
URL: https://github.com/apache/flink-ml/pull/6#discussion_r732456097



##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/broadcast/ChainingBroadcastOutput.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.iteration.broadcast;
+
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.OutputTag;
+
+/** The broadcast output corresponding to a chained output. */
+public class ChainingBroadcastOutput<OUT> implements BroadcastOutput<OUT> {
+    private final Output<StreamRecord<OUT>> rawOutput;
+    private final OutputTag outputTag;
+
+    ChainingBroadcastOutput(Output<StreamRecord<OUT>> rawOutput, OutputTag outputTag) {
+        this.rawOutput = rawOutput;
+        this.outputTag = outputTag;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void broadcastEmit(StreamRecord<OUT> record) {

Review comment:
       If I understand correctly, broadcast means that no matter what the tag is, the data should be broadcast down. If this is the case, I understand that rawOutput.collect should be fine here?

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/broadcast/BroadcastOutputFactory.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.iteration.broadcast;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.OutputTag;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Factor that creates the corresponding {@link BroadcastOutput} from the {@link Output}. */

Review comment:
       Factory

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/broadcast/ChainingBroadcastOutput.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.iteration.broadcast;
+
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.OutputTag;
+
+/** The broadcast output corresponding to a chained output. */

Review comment:
       Here I suggest that you can refer to specific classes, such as {@linkto ChainingOutput}

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/broadcast/BroadcastOutputFactory.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.iteration.broadcast;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.OutputTag;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Factor that creates the corresponding {@link BroadcastOutput} from the {@link Output}. */
+public class BroadcastOutputFactory {
+
+    /**
+     * Creates the wrapper broadcast ooutput from {@code output}.

Review comment:
       ooutput -> output

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/broadcast/OutputReflectionContext.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.iteration.broadcast;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.iteration.utils.ReflectionUtils;
+import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.OutputTag;
+
+import java.lang.reflect.Field;
+
+/** The reflection utilities to parse the output and create the broadcast output. */
+public class OutputReflectionContext {
+    private final Class<?> broadcastingOutputClass;
+    private final Field broadcastingOutputsField;
+
+    private final Class<?> chainingOutputClass;
+    private final Field chainingOutputTagField;
+
+    private final Field recordWriterField;
+    private final Field recordWriterSerializationDelegateField;
+    private final Field serializationDelegateSerializerField;
+
+    public OutputReflectionContext() {
+        try {
+            this.broadcastingOutputClass =
+                    Class.forName(
+                            "org.apache.flink.streaming.runtime.tasks.BroadcastingOutputCollector");
+            this.broadcastingOutputsField =
+                    ReflectionUtils.getClassField(broadcastingOutputClass, "outputs");
+
+            this.chainingOutputClass =
+                    Class.forName("org.apache.flink.streaming.runtime.tasks.ChainingOutput");
+            this.chainingOutputTagField =
+                    ReflectionUtils.getClassField(chainingOutputClass, "outputTag");
+
+            this.recordWriterField =
+                    ReflectionUtils.getClassField(RecordWriterOutput.class, "recordWriter");
+            this.recordWriterSerializationDelegateField =
+                    ReflectionUtils.getClassField(
+                            RecordWriterOutput.class, "serializationDelegate");
+            this.serializationDelegateSerializerField =
+                    ReflectionUtils.getClassField(SerializationDelegate.class, "serializer");
+        } catch (Exception e) {
+            throw new RuntimeException("Failed to initialize the OutputReflectionContext", e);
+        }
+    }
+
+    public boolean isBroadcastingOutput(Output<?> bareOutput) {

Review comment:
       bareOutput ---> rawOutput

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/broadcast/CountingBroadcastOutput.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.iteration.broadcast;
+
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The intermediate broadcast output that wrappers a list of internal outputs. It will broadcast the
+ * records to all the internal outputs and increment the counter.
+ */
+public class CountingBroadcastOutput<OUT> implements BroadcastOutput<OUT> {
+
+    private final Counter numRecordsOut;
+    private final List<BroadcastOutput<OUT>> internalOutputs;
+
+    public CountingBroadcastOutput(
+            Counter numRecordsOut, List<BroadcastOutput<OUT>> internalOutputs) {
+        this.numRecordsOut = numRecordsOut;
+        this.internalOutputs = internalOutputs;
+    }
+
+    @Override
+    public void broadcastEmit(StreamRecord<OUT> record) throws IOException {
+        numRecordsOut.inc();

Review comment:
       Why does the broadcasted record only increment once?




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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org