HeartSaVioR commented on code in PR #37474:
URL: https://github.com/apache/spark/pull/37474#discussion_r950751083


##########
hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.scala:
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.internal.io.cloud
+
+import java.nio.file.FileAlreadyExistsException
+import java.util.EnumSet
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs._
+
+import org.apache.spark.internal.Logging
+import 
org.apache.spark.sql.execution.streaming.AbstractFileContextBasedCheckpointFileManager
+import 
org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream
+
+class AbortableStreamBasedCheckpointFileManager(path: Path, hadoopConf: 
Configuration)

Review Comment:
   Would be nice to have a classdoc describing the rationalization (briefly 
explain what this does and how it helps), and the requirement of path. End 
users wouldn't know about capability we define in Hadoop Filesystem - so better 
to just list up schemes directly (e.g. s3a).



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala:
##########
@@ -58,50 +77,40 @@ abstract class CheckpointFileManagerTests extends 
SparkFunSuite with SQLHelper {
       // Create atomic without overwrite
       var path = new Path(s"$dir/file")
       assert(!fm.exists(path))
-      fm.createAtomic(path, overwriteIfPossible = false).cancel()
+      fm.createAtomic(path, overwriteIfPossible = 
false).writeContent(1).cancel()
       assert(!fm.exists(path))
-      fm.createAtomic(path, overwriteIfPossible = false).close()
+      fm.createAtomic(path, overwriteIfPossible = 
false).writeContent(2).close()
       assert(fm.exists(path))
+      assert(fm.open(path).readContent() == 2)
       quietly {
         intercept[IOException] {
           // should throw exception since file exists and overwrite is false
-          fm.createAtomic(path, overwriteIfPossible = false).close()
+          fm.createAtomic(path, overwriteIfPossible = 
false).writeContent(3).close()
         }
       }
+      assert(fm.open(path).readContent() == 2)
 
       // Create atomic with overwrite if possible
       path = new Path(s"$dir/file2")
       assert(!fm.exists(path))
-      fm.createAtomic(path, overwriteIfPossible = true).cancel()
+      fm.createAtomic(path, overwriteIfPossible = 
true).writeContent(4).cancel()
       assert(!fm.exists(path))
-      fm.createAtomic(path, overwriteIfPossible = true).close()
+      fm.createAtomic(path, overwriteIfPossible = true).writeContent(5).close()
       assert(fm.exists(path))
-      fm.createAtomic(path, overwriteIfPossible = true).close()  // should not 
throw exception
-
-      // crc file should not be leaked when origin file doesn't exist.
-      // The implementation of Hadoop filesystem may filter out checksum file, 
so
-      // listing files from local filesystem.
-      val fileNames = new File(path.getParent.toString).listFiles().toSeq
-        .filter(p => p.isFile).map(p => p.getName)
-      val crcFiles = fileNames.filter(n => n.startsWith(".") && 
n.endsWith(".crc"))
-      val originFileNamesForExistingCrcFiles = crcFiles.map { name =>
-        // remove first "." and last ".crc"
-        name.substring(1, name.length - 4)
-      }
-
-      // Check all origin files exist for all crc files.
-      
assert(originFileNamesForExistingCrcFiles.toSet.subsetOf(fileNames.toSet),
-        s"Some of origin files for crc files don't exist - crc files: 
$crcFiles / " +
-          s"expected origin files: $originFileNamesForExistingCrcFiles / 
actual files: $fileNames")
+      assert(fm.open(path).readContent() == 5)
+      // should not throw exception
+      fm.createAtomic(path, overwriteIfPossible = true).writeContent(6).close()
+      assert(fm.open(path).readContent() == 6)
 
+      checkLeakingCrcFiles(dir)
       // Open and delete
       fm.open(path).close()
       fm.delete(path)
       assert(!fm.exists(path))
       intercept[IOException] {
         fm.open(path)
       }
-      fm.delete(path) // should not throw exception
+      fm.delete(dir) // should not throw exception

Review Comment:
   Isn't `path` correct here as we test against file rather than directory? 



##########
hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.scala:
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.internal.io.cloud
+
+import java.nio.file.FileAlreadyExistsException
+import java.util.EnumSet
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs._
+
+import org.apache.spark.internal.Logging
+import 
org.apache.spark.sql.execution.streaming.AbstractFileContextBasedCheckpointFileManager
+import 
org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream
+
+class AbortableStreamBasedCheckpointFileManager(path: Path, hadoopConf: 
Configuration)

Review Comment:
   Also good to describe the limitation as well. Assuming the path is S3, if my 
understanding is correct, concurrent puts against same file will all succeed. 
Although we check the file with exists before closing the stream (put), there 
is a race condition which could mess up the checkpoint, so end users are 
encouraged to take serious care to prevent multiple queries to access same 
checkpoint.
   
   Default checkpoint file manager with file system supporting atomic rename 
would not suffer from concurrency issue.
   
   cc. @steveloughran in case if you got a chance to correct me if I'm 
mistaken. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to