StephanEwen commented on a change in pull request #7313: 
[FLINK-11116][fs-connector] Removed randomness from HDFS and Local fs writers.
URL: https://github.com/apache/flink/pull/7313#discussion_r249832214
 
 

 ##########
 File path: 
flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableWriter.java
 ##########
 @@ -121,24 +127,14 @@ public boolean supportsResume() {
                return true;
        }
 
-       @VisibleForTesting
-       static org.apache.hadoop.fs.Path generateStagingTempFilePath(
-                       org.apache.hadoop.fs.FileSystem fs,
-                       org.apache.hadoop.fs.Path targetFile) throws 
IOException {
-
+       private static org.apache.hadoop.fs.Path 
generateStagingTempFilePath(org.apache.hadoop.fs.Path targetFile) {
                checkArgument(targetFile.isAbsolute(), "targetFile must be 
absolute");
 
                final org.apache.hadoop.fs.Path parent = targetFile.getParent();
                final String name = targetFile.getName();
 
                checkArgument(parent != null, "targetFile must not be the root 
directory");
 
-               while (true) {
-                       org.apache.hadoop.fs.Path candidate = new 
org.apache.hadoop.fs.Path(
-                                       parent, "." + name + ".inprogress." + 
UUID.randomUUID().toString());
-                       if (!fs.exists(candidate)) {
-                               return candidate;
-                       }
-               }
+               return new org.apache.hadoop.fs.Path(parent, "." + name + 
".inprogress");
 
 Review comment:
   I added this as a simple way to avoid collisions, without relying on the 
semantics of the code that invokes this.
   
   I would try and think through what happens after a failure/recovery when an 
old task (running on a network-partitioned TM that has not yet cancelled) is 
still writing to the same file as a recovered task. Can that happen? If yes, 
what is the behavior then? Do we reduce the guarantees in that case? The 
partitioned TM should not be able to commit a checkpoint (to not roll over to a 
proper file), but can it mess up / overwrite the staging contents? My initial 
thought is that it probably could - not 100% sure though how HDFS leases would 
behave then, could they prevent that?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to