aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala
blob: 8bb9efc46cc583fa0a3ecc1ad66cd72dfca44243 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
/*
 * 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._
import java.nio.ByteBuffer

import org.apache.spark.SparkEnv
import org.apache.spark.storage._

/**
 * Create and maintain the shuffle blocks' mapping between logic block and physical file location.
 * Data of shuffle blocks from the same map task are stored in a single consolidated data file.
 * The offsets of the data blocks in the data file are stored in a separate index file.
 *
 * We use the name of the shuffle data's shuffleBlockId with reduce ID set to 0 and add ".data"
 * as the filename postfix for data file, and ".index" as the filename postfix for index file.
 *
 */
private[spark]
class IndexShuffleBlockManager extends ShuffleBlockManager {

  private lazy val blockManager = SparkEnv.get.blockManager

  /**
   * Mapping to a single shuffleBlockId with reduce ID 0.
   * */
  def consolidateId(shuffleId: Int, mapId: Int): ShuffleBlockId = {
    ShuffleBlockId(shuffleId, mapId, 0)
  }

  def getDataFile(shuffleId: Int, mapId: Int): File = {
    blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, 0))
  }

  private def getIndexFile(shuffleId: Int, mapId: Int): File = {
    blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, 0))
  }

  /**
   * Remove data file and index file that contain the output data from one map.
   * */
  def removeDataByMap(shuffleId: Int, mapId: Int): Unit = {
    var file = getDataFile(shuffleId, mapId)
    if (file.exists()) {
      file.delete()
    }

    file = getIndexFile(shuffleId, mapId)
    if (file.exists()) {
      file.delete()
    }
  }

  /**
   * Write an index file with the offsets of each block, plus a final offset at the end for the
   * end of the output file. This will be used by getBlockLocation to figure out where each block
   * begins and ends.
   * */
  def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]) = {
    val indexFile = getIndexFile(shuffleId, mapId)
    val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile)))
    try {
      // We take in lengths of each block, need to convert it to offsets.
      var offset = 0L
      out.writeLong(offset)

      for (length <- lengths) {
        offset += length
        out.writeLong(offset)
      }
    } finally {
      out.close()
    }
  }

  /**
   * Get the location of a block in a map output file. Uses the index file we create for it.
   * */
  private def getBlockLocation(blockId: ShuffleBlockId): FileSegment = {
    // The block is actually going to be a range of a single map output file for this map, so
    // find out the consolidated file, then the offset within that from our index
    val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId)

    val in = new DataInputStream(new FileInputStream(indexFile))
    try {
      in.skip(blockId.reduceId * 8)
      val offset = in.readLong()
      val nextOffset = in.readLong()
      new FileSegment(getDataFile(blockId.shuffleId, blockId.mapId), offset, nextOffset - offset)
    } finally {
      in.close()
    }
  }

  override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = {
    val segment = getBlockLocation(blockId)
    blockManager.diskStore.getBytes(segment)
  }

  override def getBlockData(blockId: ShuffleBlockId): Either[FileSegment, ByteBuffer] = {
    Left(getBlockLocation(blockId.asInstanceOf[ShuffleBlockId]))
  }

  override def stop() = {}
}