@@ -102,6 +102,44 @@ object Checkpoint extends Logging {
102
102
Seq .empty
103
103
}
104
104
}
105
+
106
+ /** Serialize the checkpoint, or throw any exception that occurs */
107
+ def serialize (checkpoint : Checkpoint , conf : SparkConf ): Array [Byte ] = {
108
+ val compressionCodec = CompressionCodec .createCodec(conf)
109
+ val bos = new ByteArrayOutputStream ()
110
+ val zos = compressionCodec.compressedOutputStream(bos)
111
+ val oos = new ObjectOutputStream (zos)
112
+ Utils .tryWithSafeFinally {
113
+ oos.writeObject(checkpoint)
114
+ } {
115
+ oos.close()
116
+ }
117
+ bos.toByteArray
118
+ }
119
+
120
+ /** Deserialize a checkpoint from the input stream, or throw any exception that occurs */
121
+ def deserialize (inputStream : InputStream , conf : SparkConf ): Checkpoint = {
122
+ val compressionCodec = CompressionCodec .createCodec(conf)
123
+ var ois : ObjectInputStreamWithLoader = null
124
+ Utils .tryWithSafeFinally {
125
+
126
+ // ObjectInputStream uses the last defined user-defined class loader in the stack
127
+ // to find classes, which maybe the wrong class loader. Hence, a inherited version
128
+ // of ObjectInputStream is used to explicitly use the current thread's default class
129
+ // loader to find and load classes. This is a well know Java issue and has popped up
130
+ // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627)
131
+ val zis = compressionCodec.compressedInputStream(inputStream)
132
+ ois = new ObjectInputStreamWithLoader (zis,
133
+ Thread .currentThread().getContextClassLoader)
134
+ val cp = ois.readObject.asInstanceOf [Checkpoint ]
135
+ cp.validate()
136
+ cp
137
+ } {
138
+ if (ois != null ) {
139
+ ois.close()
140
+ }
141
+ }
142
+ }
105
143
}
106
144
107
145
@@ -189,17 +227,10 @@ class CheckpointWriter(
189
227
}
190
228
191
229
def write (checkpoint : Checkpoint , clearCheckpointDataLater : Boolean ) {
192
- val bos = new ByteArrayOutputStream ()
193
- val zos = compressionCodec.compressedOutputStream(bos)
194
- val oos = new ObjectOutputStream (zos)
195
- Utils .tryWithSafeFinally {
196
- oos.writeObject(checkpoint)
197
- } {
198
- oos.close()
199
- }
200
230
try {
231
+ val bytes = Checkpoint .serialize(checkpoint, conf)
201
232
executor.execute(new CheckpointWriteHandler (
202
- checkpoint.checkpointTime, bos.toByteArray , clearCheckpointDataLater))
233
+ checkpoint.checkpointTime, bytes , clearCheckpointDataLater))
203
234
logDebug(" Submitted checkpoint of time " + checkpoint.checkpointTime + " writer queue" )
204
235
} catch {
205
236
case rej : RejectedExecutionException =>
@@ -264,25 +295,8 @@ object CheckpointReader extends Logging {
264
295
checkpointFiles.foreach(file => {
265
296
logInfo(" Attempting to load checkpoint from file " + file)
266
297
try {
267
- var ois : ObjectInputStreamWithLoader = null
268
- var cp : Checkpoint = null
269
- Utils .tryWithSafeFinally {
270
- val fis = fs.open(file)
271
- // ObjectInputStream uses the last defined user-defined class loader in the stack
272
- // to find classes, which maybe the wrong class loader. Hence, a inherited version
273
- // of ObjectInputStream is used to explicitly use the current thread's default class
274
- // loader to find and load classes. This is a well know Java issue and has popped up
275
- // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627)
276
- val zis = compressionCodec.compressedInputStream(fis)
277
- ois = new ObjectInputStreamWithLoader (zis,
278
- Thread .currentThread().getContextClassLoader)
279
- cp = ois.readObject.asInstanceOf [Checkpoint ]
280
- } {
281
- if (ois != null ) {
282
- ois.close()
283
- }
284
- }
285
- cp.validate()
298
+ val fis = fs.open(file)
299
+ val cp = Checkpoint .deserialize(fis, conf)
286
300
logInfo(" Checkpoint successfully loaded from file " + file)
287
301
logInfo(" Checkpoint was generated at time " + cp.checkpointTime)
288
302
return Some (cp)
0 commit comments