@@ -149,61 +149,16 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex
149
149
sc.parallelize(1 to 10 ).count()
150
150
}
151
151
152
- test( " caching " ) {
152
+ private def testCaching ( storageLevel : StorageLevel ) : Unit = {
153
153
sc = new SparkContext (clusterUrl, " test" )
154
- val data = sc.parallelize(1 to 1000 , 10 ).cache()
155
- assert(data.count() === 1000 )
156
- assert(data.count() === 1000 )
157
- assert(data.count() === 1000 )
158
- }
159
-
160
- test(" caching on disk" ) {
161
- sc = new SparkContext (clusterUrl, " test" )
162
- val data = sc.parallelize(1 to 1000 , 10 ).persist(StorageLevel .DISK_ONLY )
163
- assert(data.count() === 1000 )
164
- assert(data.count() === 1000 )
165
- assert(data.count() === 1000 )
166
- }
167
-
168
- test(" caching in memory, replicated" ) {
169
- sc = new SparkContext (clusterUrl, " test" )
170
- val data = sc.parallelize(1 to 1000 , 10 ).persist(StorageLevel .MEMORY_ONLY_2 )
171
- assert(data.count() === 1000 )
172
- assert(data.count() === 1000 )
173
- assert(data.count() === 1000 )
174
- }
175
-
176
- test(" caching in memory, serialized, replicated" ) {
177
- sc = new SparkContext (clusterUrl, " test" )
178
- val data = sc.parallelize(1 to 1000 , 10 ).persist(StorageLevel .MEMORY_ONLY_SER_2 )
179
- assert(data.count() === 1000 )
180
- assert(data.count() === 1000 )
181
- assert(data.count() === 1000 )
182
- }
183
-
184
- test(" caching on disk, replicated" ) {
185
- sc = new SparkContext (clusterUrl, " test" )
186
- val data = sc.parallelize(1 to 1000 , 10 ).persist(StorageLevel .DISK_ONLY_2 )
187
- assert(data.count() === 1000 )
188
- assert(data.count() === 1000 )
189
- assert(data.count() === 1000 )
190
- }
191
-
192
- test(" caching in memory and disk, replicated" ) {
193
- sc = new SparkContext (clusterUrl, " test" )
194
- val data = sc.parallelize(1 to 1000 , 10 ).persist(StorageLevel .MEMORY_AND_DISK_2 )
195
- assert(data.count() === 1000 )
196
- assert(data.count() === 1000 )
197
- assert(data.count() === 1000 )
198
- }
199
-
200
- test(" caching in memory and disk, serialized, replicated" ) {
201
- sc = new SparkContext (clusterUrl, " test" )
202
- val data = sc.parallelize(1 to 1000 , 10 ).persist(StorageLevel .MEMORY_AND_DISK_SER_2 )
203
-
204
- assert(data.count() === 1000 )
205
- assert(data.count() === 1000 )
206
- assert(data.count() === 1000 )
154
+ sc.jobProgressListener.waitUntilExecutorsUp(2 , 30000 )
155
+ val data = sc.parallelize(1 to 1000 , 10 )
156
+ val cachedData = data.persist(storageLevel)
157
+ assert(cachedData.count === 1000 )
158
+ assert(sc.getExecutorStorageStatus.map(_.rddBlocksById(cachedData.id).size).sum ===
159
+ storageLevel.replication * data.getNumPartitions)
160
+ assert(cachedData.count === 1000 )
161
+ assert(cachedData.count === 1000 )
207
162
208
163
// Get all the locations of the first partition and try to fetch the partitions
209
164
// from those locations.
@@ -221,6 +176,20 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex
221
176
}
222
177
}
223
178
179
+ Seq (
180
+ " caching" -> StorageLevel .MEMORY_ONLY ,
181
+ " caching on disk" -> StorageLevel .DISK_ONLY ,
182
+ " caching in memory, replicated" -> StorageLevel .MEMORY_ONLY_2 ,
183
+ " caching in memory, serialized, replicated" -> StorageLevel .MEMORY_ONLY_SER_2 ,
184
+ " caching on disk, replicated" -> StorageLevel .DISK_ONLY_2 ,
185
+ " caching in memory and disk, replicated" -> StorageLevel .MEMORY_AND_DISK_2 ,
186
+ " caching in memory and disk, serialized, replicated" -> StorageLevel .MEMORY_AND_DISK_SER_2
187
+ ).foreach { case (testName, storageLevel) =>
188
+ test(testName) {
189
+ testCaching(storageLevel)
190
+ }
191
+ }
192
+
224
193
test(" compute without caching when no partitions fit in memory" ) {
225
194
val size = 10000
226
195
val conf = new SparkConf ()
0 commit comments