viirya commented on a change in pull request #30492:
URL: https://github.com/apache/spark/pull/30492#discussion_r531171533



##########
File path: core/src/main/scala/org/apache/spark/storage/FallbackStorage.scala
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.storage
+
+import java.io.DataInputStream
+import java.nio.ByteBuffer
+
+import scala.concurrent.Future
+import scala.reflect.ClassTag
+
+import org.apache.hadoop.fs.{FileSystem, Path}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import 
org.apache.spark.internal.config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH
+import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer}
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcTimeout}
+import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleBlockInfo}
+import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID
+import org.apache.spark.util.Utils
+
+/**
+ * A fallback storage used by storage decommissioners.
+ */
+private[storage] class FallbackStorage(conf: SparkConf) extends Logging {
+  require(conf.get(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined)
+
+  private val fallbackPath = new 
Path(conf.get(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).get)
+  private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf)
+  private val fallbackFileSystem = FileSystem.get(fallbackPath.toUri, 
hadoopConf)
+
+  // Visible for testing
+  def copy(
+      shuffleBlockInfo: ShuffleBlockInfo,
+      bm: BlockManager): Unit = {
+    val shuffleId = shuffleBlockInfo.shuffleId
+    val mapId = shuffleBlockInfo.mapId
+
+    bm.migratableResolver match {
+      case r: IndexShuffleBlockResolver =>
+        val indexFile = r.getIndexFile(shuffleId, mapId)
+
+        if (indexFile.exists()) {
+          fallbackFileSystem.copyFromLocalFile(
+            new Path(indexFile.getAbsolutePath),
+            new Path(fallbackPath, s"$shuffleId/${indexFile.getName}"))
+
+          val dataFile = r.getDataFile(shuffleId, mapId)
+          if (dataFile.exists()) {
+            fallbackFileSystem.copyFromLocalFile(

Review comment:
       The API `copyFromLocalFile` doesn't mention, but when the remote file 
exists, does it overwrite or throw exception?




----------------------------------------------------------------
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:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to