@@ -94,7 +94,7 @@ object CheckpointFileManager extends Logging {
94
94
*/
95
95
sealed trait RenameHelperMethods { self => CheckpointFileManager
96
96
/** Create a file with overwrite. */
97
- def create (path : Path ): FSDataOutputStream
97
+ def createTempFile (path : Path ): FSDataOutputStream
98
98
99
99
/**
100
100
* Rename a file.
@@ -107,7 +107,7 @@ object CheckpointFileManager extends Logging {
107
107
* implementation must not overwrite if the file alraedy exists and
108
108
* must throw `FileAlreadyExistsException` in that case.
109
109
*/
110
- def rename (srcPath : Path , dstPath : Path , overwriteIfPossible : Boolean ): Unit
110
+ def renameTempFile (srcPath : Path , dstPath : Path , overwriteIfPossible : Boolean ): Unit
111
111
}
112
112
113
113
/**
@@ -131,7 +131,7 @@ object CheckpointFileManager extends Logging {
131
131
finalPath : Path ,
132
132
tempPath : Path ,
133
133
overwriteIfPossible : Boolean )
134
- extends CancellableFSDataOutputStream (fm.create (tempPath)) {
134
+ extends CancellableFSDataOutputStream (fm.createTempFile (tempPath)) {
135
135
136
136
def this (fm : CheckpointFileManager with RenameHelperMethods , path : Path , overwrite : Boolean ) = {
137
137
this (fm, path, generateTempPath(path), overwrite)
@@ -143,8 +143,15 @@ object CheckpointFileManager extends Logging {
143
143
override def close (): Unit = synchronized {
144
144
try {
145
145
if (terminated) return
146
- super .close()
147
- fm.rename(tempPath, finalPath, overwriteIfPossible)
146
+ underlyingStream.close()
147
+ try {
148
+ fm.renameTempFile(tempPath, finalPath, overwriteIfPossible)
149
+ } catch {
150
+ case fe : FileAlreadyExistsException =>
151
+ logWarning(
152
+ s " Failed to rename temp file $tempPath to $finalPath because file exists " , fe)
153
+ if (! overwriteIfPossible) throw fe
154
+ }
148
155
logInfo(s " Renamed temp file $tempPath to $finalPath" )
149
156
} finally {
150
157
terminated = true
@@ -208,9 +215,6 @@ class FileSystemBasedCheckpointFileManager(path: Path, hadoopConf: Configuration
208
215
209
216
protected val fs = path.getFileSystem(hadoopConf)
210
217
211
- fs.setVerifyChecksum(false )
212
- fs.setWriteChecksum(false )
213
-
214
218
override def list (path : Path , filter : PathFilter ): Array [FileStatus ] = {
215
219
fs.listStatus(path, filter)
216
220
}
@@ -219,7 +223,7 @@ class FileSystemBasedCheckpointFileManager(path: Path, hadoopConf: Configuration
219
223
fs.mkdirs(path, FsPermission .getDirDefault)
220
224
}
221
225
222
- override def create (path : Path ): FSDataOutputStream = {
226
+ override def createTempFile (path : Path ): FSDataOutputStream = {
223
227
fs.create(path, true )
224
228
}
225
229
@@ -236,33 +240,29 @@ class FileSystemBasedCheckpointFileManager(path: Path, hadoopConf: Configuration
236
240
fs.exists(path)
237
241
}
238
242
239
- override def rename (srcPath : Path , dstPath : Path , overwriteIfPossible : Boolean ): Unit = {
243
+ override def renameTempFile (srcPath : Path , dstPath : Path , overwriteIfPossible : Boolean ): Unit = {
240
244
if (! overwriteIfPossible && fs.exists(dstPath)) {
241
245
throw new FileAlreadyExistsException (
242
246
s " Failed to rename $srcPath to $dstPath as destination already exists " )
243
247
}
244
248
245
- try {
246
- if (! fs.rename(srcPath, dstPath) && ! overwriteIfPossible) {
247
- if (fs.exists(dstPath)) {
248
- // Some implementations of FileSystem may not throw FileAlreadyExistsException but
249
- // only return false if file already exists. Explicitly throw the error.
250
- // Note that this is definitely not atomic, so this is only a best-effort attempt
251
- // to throw the most appropriate exception when rename returned false.
249
+ if (! fs.rename(srcPath, dstPath)) {
250
+ // If overwriteIfPossible = false, then we want to find out why the rename failed and
251
+ // try to throw the right error.
252
+ if (fs.exists(dstPath)) {
253
+ // Some implementations of FileSystem may only return false instead of throwing
254
+ // FileAlreadyExistsException. In that case, explicitly throw the error the error
255
+ // if overwriteIfPossible = false. Note that this is definitely not atomic.
256
+ // This is only a best-effort attempt to identify the situation when rename returned
257
+ // false.
258
+ if (! overwriteIfPossible) {
252
259
throw new FileAlreadyExistsException (s " $dstPath already exists " )
253
- } else {
254
- val msg = s " Failed to rename temp file $srcPath to $dstPath as rename returned false "
255
- logWarning(msg)
256
- throw new IOException (msg)
257
260
}
261
+ } else {
262
+ val msg = s " Failed to rename temp file $srcPath to $dstPath as rename returned false "
263
+ logWarning(msg)
264
+ throw new IOException (msg)
258
265
}
259
- } catch {
260
- case fe : FileAlreadyExistsException =>
261
- // Some implementation of FileSystem can directly throw FileAlreadyExistsException if file
262
- // already exists. Ignore the error if overwriteIfPossible = true as it is expected to be
263
- // best effort.
264
- logWarning(s " Failed to rename temp file $srcPath to $dstPath because file exists " , fe)
265
- if (! overwriteIfPossible) throw fe
266
266
}
267
267
}
268
268
@@ -303,9 +303,11 @@ class FileContextBasedCheckpointFileManager(path: Path, hadoopConf: Configuratio
303
303
fc.mkdir(path, FsPermission .getDirDefault, true )
304
304
}
305
305
306
- override def create (path : Path ): FSDataOutputStream = {
306
+ override def createTempFile (path : Path ): FSDataOutputStream = {
307
307
import CreateFlag ._
308
- fc.create(path, EnumSet .of(CREATE , OVERWRITE ))
308
+ import Options ._
309
+ fc.create(
310
+ path, EnumSet .of(CREATE , OVERWRITE ), CreateOpts .checksumParam(ChecksumOpt .createDisabled()))
309
311
}
310
312
311
313
override def createAtomic (
@@ -321,7 +323,7 @@ class FileContextBasedCheckpointFileManager(path: Path, hadoopConf: Configuratio
321
323
fc.util.exists(path)
322
324
}
323
325
324
- override def rename (srcPath : Path , dstPath : Path , overwriteIfPossible : Boolean ): Unit = {
326
+ override def renameTempFile (srcPath : Path , dstPath : Path , overwriteIfPossible : Boolean ): Unit = {
325
327
import Options .Rename ._
326
328
fc.rename(srcPath, dstPath, if (overwriteIfPossible) OVERWRITE else NONE )
327
329
}
0 commit comments