@@ -30,6 +30,7 @@ import org.scalatest.concurrent.Eventually.{eventually, interval, timeout}
30
30
31
31
import org .apache .spark .{LocalSparkContext , SparkConf , SparkContext , SparkFunSuite , TestUtils }
32
32
import org .apache .spark .LocalSparkContext .withSpark
33
+ import org .apache .spark .deploy .SparkHadoopUtil
33
34
import org .apache .spark .internal .config ._
34
35
import org .apache .spark .io .CompressionCodec
35
36
import org .apache .spark .launcher .SparkLauncher .{EXECUTOR_MEMORY , SPARK_MASTER }
@@ -67,8 +68,10 @@ class FallbackStorageSuite extends SparkFunSuite with LocalSparkContext {
67
68
.set(STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED , true )
68
69
.set(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH ,
69
70
Files .createTempDirectory(" tmp" ).toFile.getAbsolutePath + " /" )
71
+ val hadoopConf = SparkHadoopUtil .get.newConfiguration(conf)
72
+ val rpcEndpointRef = new FallbackStorageRpcEndpointRef (conf, hadoopConf)
70
73
val fallbackStorage = new FallbackStorage (conf)
71
- val bmm = new BlockManagerMaster (new NoopRpcEndpointRef (conf) , null , conf, false )
74
+ val bmm = new BlockManagerMaster (rpcEndpointRef , null , conf, false )
72
75
73
76
val bm = mock(classOf [BlockManager ])
74
77
val dbm = new DiskBlockManager (conf, deleteFilesOnStop = false , isDriver = false )
@@ -118,8 +121,10 @@ class FallbackStorageSuite extends SparkFunSuite with LocalSparkContext {
118
121
.set(STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED , true )
119
122
.set(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH ,
120
123
" file://" + Files .createTempDirectory(" tmp" ).toFile.getAbsolutePath + " /" )
124
+ val hadoopConf = SparkHadoopUtil .get.newConfiguration(conf)
125
+ val rpcEndpointRef = new FallbackStorageRpcEndpointRef (conf, hadoopConf)
121
126
val fallbackStorage = new FallbackStorage (conf)
122
- val bmm = new BlockManagerMaster (new NoopRpcEndpointRef (conf) , null , conf, false )
127
+ val bmm = new BlockManagerMaster (rpcEndpointRef , null , conf, false )
123
128
124
129
val bm = mock(classOf [BlockManager ])
125
130
val dbm = new DiskBlockManager (conf, deleteFilesOnStop = false , isDriver = false )
@@ -153,7 +158,7 @@ class FallbackStorageSuite extends SparkFunSuite with LocalSparkContext {
153
158
assert(readResult.nioByteBuffer().array().sameElements(content))
154
159
}
155
160
156
- test(" SPARK-34142: fallback storage API - cleanUp" ) {
161
+ test(" SPARK-34142: fallback storage API - cleanUp app " ) {
157
162
withTempDir { dir =>
158
163
Seq (true , false ).foreach { cleanUp =>
159
164
val appId = s " test $cleanUp"
@@ -165,8 +170,38 @@ class FallbackStorageSuite extends SparkFunSuite with LocalSparkContext {
165
170
val location = new File (dir, appId)
166
171
assert(location.mkdir())
167
172
assert(location.exists())
173
+ val shuffle = new File (location, " 1" )
174
+ assert(shuffle.mkdir())
175
+ assert(shuffle.exists())
168
176
FallbackStorage .cleanUp(conf, new Configuration ())
169
177
assert(location.exists() != cleanUp)
178
+ assert(shuffle.exists() != cleanUp)
179
+ }
180
+ }
181
+ }
182
+
183
+ test(" SPARK-34142: fallback storage API - cleanUp shuffle" ) {
184
+ withTempDir { dir =>
185
+ Seq (true , false ).foreach { cleanUp =>
186
+ val appId = s " test $cleanUp"
187
+ val conf = new SparkConf (false )
188
+ .set(" spark.app.id" , appId)
189
+ .set(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH , dir.getAbsolutePath + " /" )
190
+ .set(STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP , cleanUp)
191
+
192
+ val location = new File (dir, appId)
193
+ assert(location.mkdir())
194
+ assert(location.exists())
195
+ val shuffle1 = new File (location, " 1" )
196
+ assert(shuffle1.mkdir())
197
+ assert(shuffle1.exists())
198
+ val shuffle2 = new File (location, " 2" )
199
+ assert(shuffle2.mkdir())
200
+ assert(shuffle2.exists())
201
+ FallbackStorage .cleanUp(conf, new Configuration (), Some (1 ))
202
+ assert(location.exists())
203
+ assert(shuffle1.exists() != cleanUp)
204
+ assert(shuffle2.exists())
170
205
}
171
206
}
172
207
}
@@ -177,6 +212,8 @@ class FallbackStorageSuite extends SparkFunSuite with LocalSparkContext {
177
212
.set(STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED , true )
178
213
.set(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH ,
179
214
Files .createTempDirectory(" tmp" ).toFile.getAbsolutePath + " /" )
215
+ val hadoopConf = SparkHadoopUtil .get.newConfiguration(conf)
216
+ val rpcEndpointRef = new FallbackStorageRpcEndpointRef (conf, hadoopConf)
180
217
181
218
val ids = Set ((1 , 1L , 1 ))
182
219
val bm = mock(classOf [BlockManager ])
@@ -202,7 +239,7 @@ class FallbackStorageSuite extends SparkFunSuite with LocalSparkContext {
202
239
203
240
when(bm.getPeers(mc.any()))
204
241
.thenReturn(Seq (FallbackStorage .FALLBACK_BLOCK_MANAGER_ID ))
205
- val bmm = new BlockManagerMaster (new NoopRpcEndpointRef (conf) , null , conf, false )
242
+ val bmm = new BlockManagerMaster (rpcEndpointRef , null , conf, false )
206
243
when(bm.master).thenReturn(bmm)
207
244
val blockTransferService = mock(classOf [BlockTransferService ])
208
245
when(blockTransferService.uploadBlockSync(mc.any(), mc.any(), mc.any(), mc.any(), mc.any(),
0 commit comments