@@ -62,7 +62,11 @@ private[spark] class IndexShuffleBlockResolver(
62
62
private val remoteShuffleMaxDisk : Option [Long ] =
63
63
conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_MAX_DISK_SIZE )
64
64
65
- def getDataFile (shuffleId : Int , mapId : Long ): File = getDataFile(shuffleId, mapId, None )
65
+ def getDataFile (shuffleId : Int , mapId : Long ): File =
66
+ getDataFile(shuffleId, mapId, None , true )
67
+
68
+ def getDataFile (shuffleId : Int , mapId : Long , needCreate : Boolean ): File =
69
+ getDataFile(shuffleId, mapId, None , needCreate)
66
70
67
71
/**
68
72
* Get the shuffle files that are stored locally. Used for block migrations.
@@ -95,12 +99,16 @@ private[spark] class IndexShuffleBlockResolver(
95
99
* When the dirs parameter is None then use the disk manager's local directories. Otherwise,
96
100
* read from the specified directories.
97
101
*/
98
- def getDataFile (shuffleId : Int , mapId : Long , dirs : Option [Array [String ]]): File = {
102
+ def getDataFile (
103
+ shuffleId : Int ,
104
+ mapId : Long ,
105
+ dirs : Option [Array [String ]],
106
+ needCreate : Boolean ): File = {
99
107
val blockId = ShuffleDataBlockId (shuffleId, mapId, NOOP_REDUCE_ID )
100
108
dirs
101
109
.map(d =>
102
110
new File (ExecutorDiskUtils .getFilePath(d, blockManager.subDirsPerLocalDir, blockId.name)))
103
- .getOrElse(blockManager.diskBlockManager.getFile(blockId))
111
+ .getOrElse(blockManager.diskBlockManager.getFile(blockId, needCreate ))
104
112
}
105
113
106
114
/**
@@ -112,12 +120,13 @@ private[spark] class IndexShuffleBlockResolver(
112
120
def getIndexFile (
113
121
shuffleId : Int ,
114
122
mapId : Long ,
115
- dirs : Option [Array [String ]] = None ): File = {
123
+ dirs : Option [Array [String ]] = None ,
124
+ needCreate : Boolean = true ): File = {
116
125
val blockId = ShuffleIndexBlockId (shuffleId, mapId, NOOP_REDUCE_ID )
117
126
dirs
118
127
.map(d =>
119
128
new File (ExecutorDiskUtils .getFilePath(d, blockManager.subDirsPerLocalDir, blockId.name)))
120
- .getOrElse(blockManager.diskBlockManager.getFile(blockId))
129
+ .getOrElse(blockManager.diskBlockManager.getFile(blockId, needCreate ))
121
130
}
122
131
123
132
private def getMergedBlockDataFile (
@@ -154,17 +163,18 @@ private[spark] class IndexShuffleBlockResolver(
154
163
* Remove data file and index file that contain the output data from one map.
155
164
*/
156
165
def removeDataByMap (shuffleId : Int , mapId : Long ): Unit = {
157
- var file = getDataFile(shuffleId, mapId)
166
+ var file = getDataFile(shuffleId, mapId, needCreate = false )
158
167
if (file.exists() && ! file.delete()) {
159
168
logWarning(s " Error deleting data ${file.getPath()}" )
160
169
}
161
170
162
- file = getIndexFile(shuffleId, mapId)
171
+ file = getIndexFile(shuffleId, mapId, needCreate = false )
163
172
if (file.exists() && ! file.delete()) {
164
173
logWarning(s " Error deleting index ${file.getPath()}" )
165
174
}
166
175
167
- file = getChecksumFile(shuffleId, mapId, conf.get(config.SHUFFLE_CHECKSUM_ALGORITHM ))
176
+ file = getChecksumFile(shuffleId, mapId, conf.get(config.SHUFFLE_CHECKSUM_ALGORITHM ),
177
+ needCreate = false )
168
178
if (file.exists() && ! file.delete()) {
169
179
logWarning(s " Error deleting checksum ${file.getPath()}" )
170
180
}
@@ -549,13 +559,14 @@ private[spark] class IndexShuffleBlockResolver(
549
559
shuffleId : Int ,
550
560
mapId : Long ,
551
561
algorithm : String ,
552
- dirs : Option [Array [String ]] = None ): File = {
562
+ dirs : Option [Array [String ]] = None ,
563
+ needCreate : Boolean = true ): File = {
553
564
val blockId = ShuffleChecksumBlockId (shuffleId, mapId, NOOP_REDUCE_ID )
554
565
val fileName = ShuffleChecksumHelper .getChecksumFileName(blockId.name, algorithm)
555
566
dirs
556
567
.map(d =>
557
568
new File (ExecutorDiskUtils .getFilePath(d, blockManager.subDirsPerLocalDir, fileName)))
558
- .getOrElse(blockManager.diskBlockManager.getFile(fileName))
569
+ .getOrElse(blockManager.diskBlockManager.getFile(fileName, needCreate ))
559
570
}
560
571
561
572
override def getBlockData (
@@ -594,7 +605,7 @@ private[spark] class IndexShuffleBlockResolver(
594
605
}
595
606
new FileSegmentManagedBuffer (
596
607
transportConf,
597
- getDataFile(shuffleId, mapId, dirs),
608
+ getDataFile(shuffleId, mapId, dirs, true ),
598
609
startOffset,
599
610
endOffset - startOffset)
600
611
} finally {
0 commit comments