|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.storage |
| 19 | + |
| 20 | +import java.io.DataInputStream |
| 21 | +import java.nio.ByteBuffer |
| 22 | + |
| 23 | +import scala.concurrent.Future |
| 24 | +import scala.reflect.ClassTag |
| 25 | + |
| 26 | +import org.apache.hadoop.fs.{FileSystem, Path} |
| 27 | + |
| 28 | +import org.apache.spark.SparkConf |
| 29 | +import org.apache.spark.deploy.SparkHadoopUtil |
| 30 | +import org.apache.spark.internal.Logging |
| 31 | +import org.apache.spark.internal.config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH |
| 32 | +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} |
| 33 | +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcTimeout} |
| 34 | +import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleBlockInfo} |
| 35 | +import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID |
| 36 | +import org.apache.spark.util.Utils |
| 37 | + |
| 38 | +/** |
| 39 | + * A fallback storage used by storage decommissioners. |
| 40 | + */ |
| 41 | +private[storage] class FallbackStorage(conf: SparkConf) extends Logging { |
| 42 | + require(conf.contains("spark.app.id")) |
| 43 | + require(conf.get(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) |
| 44 | + |
| 45 | + private val fallbackPath = new Path(conf.get(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).get) |
| 46 | + private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) |
| 47 | + private val fallbackFileSystem = FileSystem.get(fallbackPath.toUri, hadoopConf) |
| 48 | + private val appId = conf.getAppId |
| 49 | + |
| 50 | + // Visible for testing |
| 51 | + def copy( |
| 52 | + shuffleBlockInfo: ShuffleBlockInfo, |
| 53 | + bm: BlockManager): Unit = { |
| 54 | + val shuffleId = shuffleBlockInfo.shuffleId |
| 55 | + val mapId = shuffleBlockInfo.mapId |
| 56 | + |
| 57 | + bm.migratableResolver match { |
| 58 | + case r: IndexShuffleBlockResolver => |
| 59 | + val indexFile = r.getIndexFile(shuffleId, mapId) |
| 60 | + |
| 61 | + if (indexFile.exists()) { |
| 62 | + fallbackFileSystem.copyFromLocalFile( |
| 63 | + new Path(indexFile.getAbsolutePath), |
| 64 | + new Path(fallbackPath, s"$appId/$shuffleId/${indexFile.getName}")) |
| 65 | + |
| 66 | + val dataFile = r.getDataFile(shuffleId, mapId) |
| 67 | + if (dataFile.exists()) { |
| 68 | + fallbackFileSystem.copyFromLocalFile( |
| 69 | + new Path(dataFile.getAbsolutePath), |
| 70 | + new Path(fallbackPath, s"$appId/$shuffleId/${dataFile.getName}")) |
| 71 | + } |
| 72 | + |
| 73 | + // Report block statuses |
| 74 | + val reduceId = NOOP_REDUCE_ID |
| 75 | + val indexBlockId = ShuffleIndexBlockId(shuffleId, mapId, reduceId) |
| 76 | + FallbackStorage.reportBlockStatus(bm, indexBlockId, indexFile.length) |
| 77 | + if (dataFile.exists) { |
| 78 | + val dataBlockId = ShuffleDataBlockId(shuffleId, mapId, reduceId) |
| 79 | + FallbackStorage.reportBlockStatus(bm, dataBlockId, dataFile.length) |
| 80 | + } |
| 81 | + } |
| 82 | + case r => |
| 83 | + logWarning(s"Unsupported Resolver: ${r.getClass.getName}") |
| 84 | + } |
| 85 | + } |
| 86 | + |
| 87 | + def exists(shuffleId: Int, filename: String): Boolean = { |
| 88 | + fallbackFileSystem.exists(new Path(fallbackPath, s"$appId/$shuffleId/$filename")) |
| 89 | + } |
| 90 | +} |
| 91 | + |
| 92 | +class NoopRpcEndpointRef(conf: SparkConf) extends RpcEndpointRef(conf) { |
| 93 | + import scala.concurrent.ExecutionContext.Implicits.global |
| 94 | + override def address: RpcAddress = null |
| 95 | + override def name: String = "fallback" |
| 96 | + override def send(message: Any): Unit = {} |
| 97 | + override def ask[T: ClassTag](message: Any, timeout: RpcTimeout): Future[T] = { |
| 98 | + Future{true.asInstanceOf[T]} |
| 99 | + } |
| 100 | +} |
| 101 | + |
| 102 | +object FallbackStorage extends Logging { |
| 103 | + /** We use one block manager id as a place holder. */ |
| 104 | + val FALLBACK_BLOCK_MANAGER_ID: BlockManagerId = BlockManagerId("fallback", "remote", 7337) |
| 105 | + |
| 106 | + def getFallbackStorage(conf: SparkConf): Option[FallbackStorage] = { |
| 107 | + if (conf.get(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) { |
| 108 | + Some(new FallbackStorage(conf)) |
| 109 | + } else { |
| 110 | + None |
| 111 | + } |
| 112 | + } |
| 113 | + |
| 114 | + /** Register the fallback block manager and its RPC endpoint. */ |
| 115 | + def registerBlockManagerIfNeeded(master: BlockManagerMaster, conf: SparkConf): Unit = { |
| 116 | + if (conf.get(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined) { |
| 117 | + master.registerBlockManager( |
| 118 | + FALLBACK_BLOCK_MANAGER_ID, Array.empty[String], 0, 0, new NoopRpcEndpointRef(conf)) |
| 119 | + } |
| 120 | + } |
| 121 | + |
| 122 | + /** Report block status to block manager master and map output tracker master. */ |
| 123 | + private def reportBlockStatus(blockManager: BlockManager, blockId: BlockId, dataLength: Long) = { |
| 124 | + assert(blockManager.master != null) |
| 125 | + blockManager.master.updateBlockInfo( |
| 126 | + FALLBACK_BLOCK_MANAGER_ID, blockId, StorageLevel.DISK_ONLY, memSize = 0, dataLength) |
| 127 | + } |
| 128 | + |
| 129 | + /** |
| 130 | + * Read a ManagedBuffer. |
| 131 | + */ |
| 132 | + def read(conf: SparkConf, blockId: BlockId): ManagedBuffer = { |
| 133 | + logInfo(s"Read $blockId") |
| 134 | + val fallbackPath = new Path(conf.get(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).get) |
| 135 | + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) |
| 136 | + val fallbackFileSystem = FileSystem.get(fallbackPath.toUri, hadoopConf) |
| 137 | + val appId = conf.getAppId |
| 138 | + |
| 139 | + val (shuffleId, mapId, startReduceId, endReduceId) = blockId match { |
| 140 | + case id: ShuffleBlockId => |
| 141 | + (id.shuffleId, id.mapId, id.reduceId, id.reduceId + 1) |
| 142 | + case batchId: ShuffleBlockBatchId => |
| 143 | + (batchId.shuffleId, batchId.mapId, batchId.startReduceId, batchId.endReduceId) |
| 144 | + case _ => |
| 145 | + throw new IllegalArgumentException("unexpected shuffle block id format: " + blockId) |
| 146 | + } |
| 147 | + |
| 148 | + val name = ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID).name |
| 149 | + val indexFile = new Path(fallbackPath, s"$appId/$shuffleId/$name") |
| 150 | + val start = startReduceId * 8L |
| 151 | + val end = endReduceId * 8L |
| 152 | + Utils.tryWithResource(fallbackFileSystem.open(indexFile)) { inputStream => |
| 153 | + Utils.tryWithResource(new DataInputStream(inputStream)) { index => |
| 154 | + index.skip(start) |
| 155 | + val offset = index.readLong() |
| 156 | + index.skip(end - (start + 8L)) |
| 157 | + val nextOffset = index.readLong() |
| 158 | + val name = ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID).name |
| 159 | + val dataFile = new Path(fallbackPath, s"$appId/$shuffleId/$name") |
| 160 | + val f = fallbackFileSystem.open(dataFile) |
| 161 | + val size = nextOffset - 1 - offset |
| 162 | + logDebug(s"To byte array $size") |
| 163 | + val array = new Array[Byte](size.toInt) |
| 164 | + val startTimeNs = System.nanoTime() |
| 165 | + f.seek(offset) |
| 166 | + f.read(array) |
| 167 | + logDebug(s"Took ${(System.nanoTime() - startTimeNs) / (1000 * 1000)}ms") |
| 168 | + f.close() |
| 169 | + new NioManagedBuffer(ByteBuffer.wrap(array)) |
| 170 | + } |
| 171 | + } |
| 172 | + } |
| 173 | +} |
| 174 | + |
0 commit comments