lindong28 commented on a change in pull request #30: URL: https://github.com/apache/flink-ml/pull/30#discussion_r752304927
########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/allreduce/AllReduceUtils.java ########## @@ -0,0 +1,314 @@ +/* + * 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.allreduce; + +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.Preconditions; + +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 pieces of fixed size buffer (4KB by default) and let each subtask + * handle several pieces. + * + * <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; Review comment: Why do we choose this size? Is there any case where this number should be adjusted? If so, should we make this configurable by the caller? ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/allreduce/AllReduceUtils.java ########## @@ -0,0 +1,314 @@ +/* + * 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.allreduce; + +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.Preconditions; + +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 pieces of fixed size buffer (4KB by default) and let each subtask + * handle several pieces. + * + * <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) { + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSend = + input.transform( + "all-reduce-send", + new TupleTypeInfo<>( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO), + new AllReduceSend()) + .name("all-reduce-send"); + + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSum = + allReduceSend + .partitionCustom((key, numPartitions) -> key % numPartitions, x -> x.f1) + .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((key, numPartitions) -> key % numPartitions, x -> x.f0) + .transform( + "all-reduce-recv", TypeInformation.of(double[].class), new AllReduceRecv()) + .name("all-reduce-recv"); + } + + /** + * Splits each double array into multiple pieces and send each piece to the corresponding + * partition. + */ + private static class AllReduceSend + extends AbstractStreamOperator<Tuple4<Integer, Integer, Integer, double[]>> + implements OneInputStreamOperator< + double[], Tuple4<Integer, Integer, Integer, double[]>>, + BoundedOneInput { + + double[] receivedElement; + + double[] transBuf = new double[TRANSFER_BUFFER_SIZE]; + + @Override + public void endInput() { + int numTasks = getRuntimeContext().getNumberOfParallelSubtasks(); + + for (int taskId = 0; taskId < numTasks; taskId++) { + int startPieceId = + DistributedInfo.getStartPieceId(taskId, numTasks, receivedElement.length); + int numPiecesToHandle = + DistributedInfo.getNumPiecesByTaskId( + taskId, numTasks, receivedElement.length); + for (int piece = 0; piece < numPiecesToHandle; piece++) { + int bufStart = (startPieceId + piece) * TRANSFER_BUFFER_SIZE; + System.arraycopy( + receivedElement, + bufStart, + transBuf, + 0, + DistributedInfo.getLengthOfPiece( + startPieceId + piece, receivedElement.length)); + output.collect( + new StreamRecord<>( + Tuple4.of( + taskId, Review comment: It looks like the first field of this tuple (i.e. taskId) is never used. Can we remove it? ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/allreduce/AllReduceUtils.java ########## @@ -0,0 +1,314 @@ +/* + * 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.allreduce; + +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.Preconditions; + +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 pieces of fixed size buffer (4KB by default) and let each subtask + * handle several pieces. + * + * <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 { Review comment: Can we move `AllReduceUtils` to the package `org.apache.flink.ml.common.datastream` instead of adding a new package here? ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/allreduce/AllReduceUtils.java ########## @@ -0,0 +1,314 @@ +/* + * 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.allreduce; + +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.Preconditions; + +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 pieces of fixed size buffer (4KB by default) and let each subtask + * handle several pieces. + * + * <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) { + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSend = + input.transform( + "all-reduce-send", + new TupleTypeInfo<>( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO), + new AllReduceSend()) + .name("all-reduce-send"); + + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSum = + allReduceSend + .partitionCustom((key, numPartitions) -> key % numPartitions, x -> x.f1) + .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((key, numPartitions) -> key % numPartitions, x -> x.f0) + .transform( + "all-reduce-recv", TypeInformation.of(double[].class), new AllReduceRecv()) + .name("all-reduce-recv"); + } + + /** + * Splits each double array into multiple pieces and send each piece to the corresponding + * partition. + */ + private static class AllReduceSend + extends AbstractStreamOperator<Tuple4<Integer, Integer, Integer, double[]>> + implements OneInputStreamOperator< + double[], Tuple4<Integer, Integer, Integer, double[]>>, + BoundedOneInput { + + double[] receivedElement; + + double[] transBuf = new double[TRANSFER_BUFFER_SIZE]; + + @Override + public void endInput() { + int numTasks = getRuntimeContext().getNumberOfParallelSubtasks(); + + for (int taskId = 0; taskId < numTasks; taskId++) { + int startPieceId = + DistributedInfo.getStartPieceId(taskId, numTasks, receivedElement.length); + int numPiecesToHandle = + DistributedInfo.getNumPiecesByTaskId( + taskId, numTasks, receivedElement.length); + for (int piece = 0; piece < numPiecesToHandle; piece++) { + int bufStart = (startPieceId + piece) * TRANSFER_BUFFER_SIZE; + System.arraycopy( + receivedElement, + bufStart, + transBuf, + 0, + DistributedInfo.getLengthOfPiece( + startPieceId + piece, receivedElement.length)); + output.collect( + new StreamRecord<>( + Tuple4.of( + taskId, + startPieceId + piece, + receivedElement.length, + transBuf))); + } + } + } + + @Override + public void processElement(StreamRecord<double[]> streamRecord) { + Preconditions.checkState( + null == receivedElement, + "The input cannot contain more than one double array."); + receivedElement = streamRecord.getValue(); + } + } + + /** Aggregates partitioned array from other workers. */ + private static class AllReduceSum + extends AbstractStreamOperator<Tuple4<Integer, Integer, Integer, double[]>> + implements OneInputStreamOperator< + Tuple4<Integer, Integer, Integer, double[]>, + Tuple4<Integer, Integer, Integer, double[]>>, + BoundedOneInput { + + /** + * A map that aggregates the received partial arrays. The key is pieceId, the value is + * (pieceId, 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 pieceId = ele.f0; + int totalLength = ele.f1; + double[] aggregatedSum = ele.f2; + output.collect( + new StreamRecord<>( + Tuple4.of(taskId, pieceId, totalLength, aggregatedSum))); + } + } + } + + @Override + public void processElement( + StreamRecord<Tuple4<Integer, Integer, Integer, double[]>> streamRecord) { + Tuple4<Integer, Integer, Integer, double[]> record = streamRecord.getValue(); + int pieceId = record.f1; + int totalLength = record.f2; + double[] partialSum = record.f3; + if (sum.containsKey(pieceId)) { + Preconditions.checkState( + sum.get(pieceId).f1 == totalLength, + "The input double array must have same length."); + double[] cur = sum.get(pieceId).f2; + for (int i = 0; i < cur.length; i++) { + cur[i] += partialSum[i]; + } + } else { + sum.put(pieceId, Tuple3.of(pieceId, totalLength, partialSum)); + } + } + } + + /** Organizes the received pieces 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 pieceId = ele.f1; + int totalLength = ele.f2; + double[] partialSum = ele.f3; + if (null == received) { + received = new double[totalLength]; + } + System.arraycopy( + partialSum, + 0, + received, + pieceId * TRANSFER_BUFFER_SIZE, + DistributedInfo.getLengthOfPiece(pieceId, received.length)); + } + } + + /** Util class to split a big array into pieces. */ + private static class DistributedInfo { Review comment: Can we remove this static class and move its static methods to `AllReduceUtils`? ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/allreduce/AllReduceUtils.java ########## @@ -0,0 +1,314 @@ +/* + * 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.allreduce; + +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.Preconditions; + +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 pieces of fixed size buffer (4KB by default) and let each subtask + * handle several pieces. + * + * <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) { + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSend = + input.transform( + "all-reduce-send", + new TupleTypeInfo<>( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO), + new AllReduceSend()) + .name("all-reduce-send"); + + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSum = + allReduceSend + .partitionCustom((key, numPartitions) -> key % numPartitions, x -> x.f1) + .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((key, numPartitions) -> key % numPartitions, x -> x.f0) + .transform( + "all-reduce-recv", TypeInformation.of(double[].class), new AllReduceRecv()) + .name("all-reduce-recv"); + } + + /** + * Splits each double array into multiple pieces and send each piece to the corresponding + * partition. + */ + private static class AllReduceSend + extends AbstractStreamOperator<Tuple4<Integer, Integer, Integer, double[]>> + implements OneInputStreamOperator< + double[], Tuple4<Integer, Integer, Integer, double[]>>, + BoundedOneInput { + + double[] receivedElement; + + double[] transBuf = new double[TRANSFER_BUFFER_SIZE]; + + @Override + public void endInput() { + int numTasks = getRuntimeContext().getNumberOfParallelSubtasks(); + + for (int taskId = 0; taskId < numTasks; taskId++) { + int startPieceId = + DistributedInfo.getStartPieceId(taskId, numTasks, receivedElement.length); + int numPiecesToHandle = + DistributedInfo.getNumPiecesByTaskId( + taskId, numTasks, receivedElement.length); + for (int piece = 0; piece < numPiecesToHandle; piece++) { + int bufStart = (startPieceId + piece) * TRANSFER_BUFFER_SIZE; + System.arraycopy( + receivedElement, + bufStart, + transBuf, + 0, + DistributedInfo.getLengthOfPiece( + startPieceId + piece, receivedElement.length)); + output.collect( + new StreamRecord<>( + Tuple4.of( + taskId, + startPieceId + piece, + receivedElement.length, + transBuf))); + } + } + } + + @Override + public void processElement(StreamRecord<double[]> streamRecord) { + Preconditions.checkState( Review comment: `Preconditions.checkState` throws `IllegalStateException`, which means there is bug in this class/method. But the check will fail due to caller-provided input, right? ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/allreduce/AllReduceUtils.java ########## @@ -0,0 +1,314 @@ +/* + * 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.allreduce; + +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.Preconditions; + +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 pieces of fixed size buffer (4KB by default) and let each subtask + * handle several pieces. + * + * <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) { + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSend = + input.transform( + "all-reduce-send", + new TupleTypeInfo<>( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO), + new AllReduceSend()) + .name("all-reduce-send"); + + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSum = + allReduceSend + .partitionCustom((key, numPartitions) -> key % numPartitions, x -> x.f1) Review comment: Can we replace `key` with a more meaningful name, e.g. pieceIdx? Can we rename `piece` to `chunk` since `chunk` appears to be more widely in Flink? ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/allreduce/AllReduceUtils.java ########## @@ -0,0 +1,314 @@ +/* + * 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.allreduce; + +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.Preconditions; + +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 pieces of fixed size buffer (4KB by default) and let each subtask + * handle several pieces. + * + * <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) { + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSend = + input.transform( + "all-reduce-send", + new TupleTypeInfo<>( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO), + new AllReduceSend()) + .name("all-reduce-send"); + + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSum = + allReduceSend + .partitionCustom((key, numPartitions) -> key % numPartitions, x -> x.f1) + .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((key, numPartitions) -> key % numPartitions, x -> x.f0) + .transform( + "all-reduce-recv", TypeInformation.of(double[].class), new AllReduceRecv()) + .name("all-reduce-recv"); + } + + /** + * Splits each double array into multiple pieces and send each piece to the corresponding + * partition. + */ + private static class AllReduceSend + extends AbstractStreamOperator<Tuple4<Integer, Integer, Integer, double[]>> + implements OneInputStreamOperator< + double[], Tuple4<Integer, Integer, Integer, double[]>>, + BoundedOneInput { + + double[] receivedElement; + + double[] transBuf = new double[TRANSFER_BUFFER_SIZE]; + + @Override + public void endInput() { + int numTasks = getRuntimeContext().getNumberOfParallelSubtasks(); + + for (int taskId = 0; taskId < numTasks; taskId++) { + int startPieceId = + DistributedInfo.getStartPieceId(taskId, numTasks, receivedElement.length); + int numPiecesToHandle = + DistributedInfo.getNumPiecesByTaskId( + taskId, numTasks, receivedElement.length); + for (int piece = 0; piece < numPiecesToHandle; piece++) { + int bufStart = (startPieceId + piece) * TRANSFER_BUFFER_SIZE; + System.arraycopy( + receivedElement, + bufStart, + transBuf, + 0, + DistributedInfo.getLengthOfPiece( + startPieceId + piece, receivedElement.length)); + output.collect( + new StreamRecord<>( + Tuple4.of( + taskId, + startPieceId + piece, + receivedElement.length, + transBuf))); + } + } + } + + @Override + public void processElement(StreamRecord<double[]> streamRecord) { + Preconditions.checkState( + null == receivedElement, + "The input cannot contain more than one double array."); + receivedElement = streamRecord.getValue(); + } + } + + /** Aggregates partitioned array from other workers. */ + private static class AllReduceSum + extends AbstractStreamOperator<Tuple4<Integer, Integer, Integer, double[]>> + implements OneInputStreamOperator< + Tuple4<Integer, Integer, Integer, double[]>, + Tuple4<Integer, Integer, Integer, double[]>>, + BoundedOneInput { + + /** + * A map that aggregates the received partial arrays. The key is pieceId, the value is + * (pieceId, 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 pieceId = ele.f0; + int totalLength = ele.f1; + double[] aggregatedSum = ele.f2; + output.collect( + new StreamRecord<>( + Tuple4.of(taskId, pieceId, totalLength, aggregatedSum))); + } + } + } + + @Override + public void processElement( + StreamRecord<Tuple4<Integer, Integer, Integer, double[]>> streamRecord) { + Tuple4<Integer, Integer, Integer, double[]> record = streamRecord.getValue(); + int pieceId = record.f1; + int totalLength = record.f2; + double[] partialSum = record.f3; + if (sum.containsKey(pieceId)) { + Preconditions.checkState( + sum.get(pieceId).f1 == totalLength, + "The input double array must have same length."); + double[] cur = sum.get(pieceId).f2; + for (int i = 0; i < cur.length; i++) { + cur[i] += partialSum[i]; + } + } else { + sum.put(pieceId, Tuple3.of(pieceId, totalLength, partialSum)); + } + } + } + + /** Organizes the received pieces 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 pieceId = ele.f1; + int totalLength = ele.f2; + double[] partialSum = ele.f3; + if (null == received) { + received = new double[totalLength]; Review comment: Instead of repeatedly allocating and deallocating this double array in every round, could we use AllRound mode to re-use the same double array? Note that this array is supposed to be very large in the target AllReduce use-case. ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/allreduce/AllReduceUtils.java ########## @@ -0,0 +1,314 @@ +/* + * 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.allreduce; + +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.Preconditions; + +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 pieces of fixed size buffer (4KB by default) and let each subtask + * handle several pieces. + * + * <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) { + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSend = + input.transform( + "all-reduce-send", + new TupleTypeInfo<>( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO), + new AllReduceSend()) + .name("all-reduce-send"); + + // taskId, pieceId, totalElements, partitionedArray + DataStream<Tuple4<Integer, Integer, Integer, double[]>> allReduceSum = + allReduceSend + .partitionCustom((key, numPartitions) -> key % numPartitions, x -> x.f1) + .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((key, numPartitions) -> key % numPartitions, x -> x.f0) + .transform( + "all-reduce-recv", TypeInformation.of(double[].class), new AllReduceRecv()) + .name("all-reduce-recv"); + } + + /** + * Splits each double array into multiple pieces and send each piece to the corresponding + * partition. + */ + private static class AllReduceSend Review comment: Would it be simpler for AllReduceSend to be a `MapFunction`? -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
