Ngone51 commented on a change in pull request #30312: URL: https://github.com/apache/spark/pull/30312#discussion_r534271909
########## File path: core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala ########## @@ -0,0 +1,469 @@ +/* + * 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.spark.shuffle + +import java.io.File +import java.net.ConnectException +import java.nio.ByteBuffer +import java.util.concurrent.ExecutorService + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} + +import com.google.common.base.Throwables + +import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv} +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.BlockFetchingListener +import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler +import org.apache.spark.network.util.TransportConf +import org.apache.spark.shuffle.ShuffleBlockPusher._ +import org.apache.spark.storage.{BlockId, BlockManagerId, ShufflePushBlockId} +import org.apache.spark.util.{ThreadUtils, Utils} + +/** + * Used for pushing shuffle blocks to remote shuffle services when push shuffle is enabled. + * When push shuffle is enabled, it is created after the shuffle writer finishes writing the shuffle + * file and initiates the block push process. + * + * @param conf spark configuration + */ +@Since("3.1.0") +private[spark] class ShuffleBlockPusher( + conf: SparkConf) extends Logging { + private[this] var maxBlockSizeToPush = 0L + private[this] var maxBlockBatchSize = 0L + private[this] var maxBytesInFlight = 0L + private[this] var maxReqsInFlight = 0 + private[this] var maxBlocksInFlightPerAddress = 0 + private[this] var bytesInFlight = 0L + private[this] var reqsInFlight = 0 + private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]() + private[this] val deferredPushRequests = new HashMap[BlockManagerId, Queue[PushRequest]]() + private[this] val pushRequests = new Queue[PushRequest] + private[this] val errorHandler = createErrorHandler() + // VisibleForTesting + private[shuffle] val unreachableBlockMgrs = new HashSet[BlockManagerId]() + private[this] var stopPushing = false + + initialize() + + private def initialize(): Unit = { + maxBlockSizeToPush = conf.get(SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 + maxBlockBatchSize = conf.get(SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH) * 1024 * 1024 + maxBytesInFlight = conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 + maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) + maxBlocksInFlightPerAddress = conf.get(REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS) Review comment: Why not init them at the time we declare them? ---------------------------------------------------------------- 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] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
