cfmcgrady commented on code in PR #1830: URL: https://github.com/apache/incubator-celeborn/pull/1830#discussion_r1309571233
########## client-mr/mr/src/main/java/org/apache/hadoop/mapred/CelebornSortBasedPusher.java: ########## @@ -0,0 +1,338 @@ +/* + * 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.hadoop.mapred; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.hadoop.io.RawComparator; +import org.apache.hadoop.io.WritableUtils; +import org.apache.hadoop.io.serializer.Serializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.celeborn.client.ShuffleClient; +import org.apache.celeborn.common.CelebornConf; +import org.apache.celeborn.common.unsafe.Platform; +import org.apache.celeborn.common.util.Utils; + +public class CelebornSortBasedPusher<K, V> extends OutputStream { + final Logger logger = LoggerFactory.getLogger(CelebornSortBasedPusher.class); + private final int mapId; + private final int attempt; + private final int numMappers; + private final int numReducers; + private final ShuffleClient shuffleClient; + private final int maxIOBufferSize; + private final int spillIOBufferSize; + private final Serializer<K> kSer; + private final Serializer<V> vSer; + private final RawComparator<K> comparator; + private final AtomicReference<Exception> exception = new AtomicReference<>(); + private final Counters.Counter mapOutputByteCounter; + private final Counters.Counter mapOutputRecordCounter; + private final Map<Integer, List<SerializedKV>> currentSerializedKVs; + private int writePos; + private byte[] serializedKV; + private final int maxPushDataSize; + + public CelebornSortBasedPusher( + int numMappers, + int numReducers, + int mapId, + int attemptId, + Serializer<K> kSer, + Serializer<V> vSer, + int maxIOBufferSize, + int spillIOBufferSize, + RawComparator<K> comparator, + Counters.Counter mapOutputByteCounter, + Counters.Counter mapOutputRecordCounter, + ShuffleClient shuffleClient, + CelebornConf celebornConf) { + this.numMappers = numMappers; + this.numReducers = numReducers; + this.mapId = mapId; + this.attempt = attemptId; + this.kSer = kSer; + this.vSer = vSer; + this.maxIOBufferSize = maxIOBufferSize; + this.spillIOBufferSize = spillIOBufferSize; + this.mapOutputByteCounter = mapOutputByteCounter; + this.mapOutputRecordCounter = mapOutputRecordCounter; + this.comparator = comparator; + this.shuffleClient = shuffleClient; + currentSerializedKVs = new HashMap<>(); + serializedKV = new byte[maxIOBufferSize]; + maxPushDataSize = (int) celebornConf.clientMrMaxPushData(); Review Comment: add `checkValues` for the conf `celeborn.client.mr.pushData.max` to validate that the config value is below Int.MaxValue. -- 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]
