zhipeng93 commented on a change in pull request #30:
URL: https://github.com/apache/flink-ml/pull/30#discussion_r753775516



##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/common/datastream/AllReduceUtils.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.ml.common.datastream;
+
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Collector;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Applies all-reduce on a DataStream where each partition contains only one 
double array.
+ *
+ * <p>AllReduce is a communication primitive widely used in MPI. In this 
implementation, all workers
+ * do reduce on a partition of the whole data and they all get the final 
reduce result. In detail,
+ * we split each double array into chunks of fixed size buffer (4KB by 
default) and let each subtask
+ * handle several chunks.
+ *
+ * <p>There're mainly three stages:
+ * <li>1. All workers send their partial data to other workers for reduce.
+ * <li>2. All workers do reduce on all data it received and then send partial 
results to others.
+ * <li>3. All workers merge partial results into final result.
+ */
+public class AllReduceUtils {
+
+    private static final int TRANSFER_BUFFER_SIZE = 1024 * 4;
+
+    /**
+     * Applies allReduce on the input data stream. The input data stream is 
supposed to contain one
+     * double array in each partition. The result data stream has the same 
parallelism as the input,
+     * where each partition contains one double array that sums all of the 
double arrays in the
+     * input data stream.
+     *
+     * <p>Note that we throw exception when one of the following two cases 
happen:
+     * <li>1. There exists one partition that contains more than one double 
array.
+     * <li>2. The length of double array is not consistent among all 
partitions.
+     *
+     * @param input The input data stream.
+     * @return The result data stream.
+     */
+    public static DataStream<double[]> allReduce(DataStream<double[]> input) {
+        // chunkId, totalElements, partitionedArray
+        DataStream<Tuple3<Integer, Integer, double[]>> allReduceSend =
+                input.flatMap(new AllReduceSend()).name("all-reduce-send");
+
+        // taskId, chunkId, totalElements, partitionedArray
+        DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSum =
+                allReduceSend
+                        .partitionCustom(
+                                (chunkId, numPartitions) -> chunkId % 
numPartitions, x -> x.f0)
+                        .transform(
+                                "all-reduce-sum",
+                                new TupleTypeInfo<>(
+                                        BasicTypeInfo.INT_TYPE_INFO,
+                                        BasicTypeInfo.INT_TYPE_INFO,
+                                        BasicTypeInfo.INT_TYPE_INFO,
+                                        
PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO),
+                                new AllReduceSum())
+                        .name("all-reduce-sum");
+
+        return allReduceSum
+                .partitionCustom((taskIdx, numPartitions) -> taskIdx % 
numPartitions, x -> x.f0)
+                .transform(
+                        "all-reduce-recv", TypeInformation.of(double[].class), 
new AllReduceRecv())
+                .name("all-reduce-recv");
+    }
+
+    /**
+     * Splits each double array into multiple chunks and send each chunk to 
the corresponding
+     * partition.
+     */
+    private static class AllReduceSend
+            extends RichFlatMapFunction<double[], Tuple3<Integer, Integer, 
double[]>> {
+        boolean hasReceivedOneRecord = false;
+        double[] transBuf = new double[TRANSFER_BUFFER_SIZE];
+
+        @Override
+        public void flatMap(
+                double[] receivedElement, Collector<Tuple3<Integer, Integer, 
double[]>> out) {
+            if (hasReceivedOneRecord) {
+                throw new RuntimeException("The input cannot contain more than 
one double array.");
+            }
+            hasReceivedOneRecord = true;
+            int numTasks = getRuntimeContext().getNumberOfParallelSubtasks();
+
+            for (int taskId = 0; taskId < numTasks; taskId++) {
+                int startChunkId = getStartChunkId(taskId, numTasks, 
receivedElement.length);
+                int numChunksToHandle =
+                        getNumChunksByTaskId(taskId, numTasks, 
receivedElement.length);
+                for (int chunk = 0; chunk < numChunksToHandle; chunk++) {
+                    int bufStart = (startChunkId + chunk) * 
TRANSFER_BUFFER_SIZE;
+                    System.arraycopy(
+                            receivedElement,
+                            bufStart,
+                            transBuf,
+                            0,
+                            getLengthOfChunk(startChunkId + chunk, 
receivedElement.length));
+                    out.collect(Tuple3.of(startChunkId + chunk, 
receivedElement.length, transBuf));
+                }
+            }
+        }
+    }
+
+    /** Aggregates partitioned array from other workers. */
+    private static class AllReduceSum
+            extends AbstractStreamOperator<Tuple4<Integer, Integer, Integer, 
double[]>>
+            implements OneInputStreamOperator<
+                            Tuple3<Integer, Integer, double[]>,
+                            Tuple4<Integer, Integer, Integer, double[]>>,
+                    BoundedOneInput {
+
+        /**
+         * A map that aggregates the received partial arrays. The key is 
chunkId, the value is
+         * (chunkId, totalLength, aggPartitionedArray).
+         */
+        Map<Integer, Tuple3<Integer, Integer, double[]>> sum = new HashMap<>();
+
+        @Override
+        public void endInput() {
+            int numTasks = getRuntimeContext().getNumberOfParallelSubtasks();
+            for (Tuple3<Integer, Integer, double[]> ele : sum.values()) {
+                for (int taskId = 0; taskId < numTasks; taskId++) {
+                    int chunkId = ele.f0;
+                    int totalLength = ele.f1;
+                    double[] aggregatedSum = ele.f2;
+                    output.collect(
+                            new StreamRecord<>(
+                                    Tuple4.of(taskId, chunkId, totalLength, 
aggregatedSum)));
+                }
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple3<Integer, Integer, 
double[]>> streamRecord) {
+            Tuple3<Integer, Integer, double[]> record = 
streamRecord.getValue();
+            int chunkId = record.f0;
+            int totalLength = record.f1;
+            double[] partialSum = record.f2;
+            if (sum.containsKey(chunkId)) {
+                if (sum.get(chunkId).f1 != totalLength) {
+                    throw new RuntimeException("The input double array must 
have same length.");
+                }
+                double[] cur = sum.get(chunkId).f2;
+                for (int i = 0; i < cur.length; i++) {
+                    cur[i] += partialSum[i];
+                }
+            } else {
+                sum.put(chunkId, Tuple3.of(chunkId, totalLength, partialSum));
+            }
+        }
+    }
+
+    /** Organizes the received chunks into a big array. */
+    private static class AllReduceRecv extends AbstractStreamOperator<double[]>
+            implements OneInputStreamOperator<
+                            Tuple4<Integer, Integer, Integer, double[]>, 
double[]>,
+                    BoundedOneInput {
+
+        /**
+         * used to store the reduced results. Note that if we use {@link
+         * 
org.apache.flink.iteration.IterationConfig.OperatorLifeCycle}.ALL_ROUND, the 
memory can
+         * be maintained across iterations and will not incur frequently GC.
+         */
+        double[] received;
+
+        @Override
+        public void endInput() {
+            if (null != received) {
+                output.collect(new StreamRecord<>(received));
+            }
+        }
+
+        @Override
+        public void processElement(
+                StreamRecord<Tuple4<Integer, Integer, Integer, double[]>> 
streamRecord) {
+            Tuple4<Integer, Integer, Integer, double[]> ele = 
streamRecord.getValue();
+            int chunkId = ele.f1;
+            int totalLength = ele.f2;
+            double[] partialSum = ele.f3;
+            if (null == received) {
+                received = new double[totalLength];

Review comment:
       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.

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

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


Reply via email to