Skip to content

Conversation

@adoroszlai
Copy link
Contributor

What changes were proposed in this pull request?

AtomicFileOutputStream should reject overwriting existing temp file.

https://issues.apache.org/jira/browse/RATIS-2260

How was this patch tested?

CI:
https://github.com/adoroszlai/ratis/actions/runs/13900762173

@adoroszlai adoroszlai self-assigned this Mar 17, 2025
Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@adoroszlai , thanks a lot for working on this! Please see the comment inlined.


public AtomicFileOutputStream(File outFile, File tmpFile) throws IOException {
super(FileUtils.newOutputStreamForceAtClose(tmpFile, StandardOpenOption.CREATE, StandardOpenOption.WRITE));
super(FileUtils.newOutputStreamForceAtClose(tmpFile, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's pass a createNew parameter. Usually, we don't care about overwriting tmp file since it is just tmp. On the other hand, we don't want to throw an exception when server starting because of a leftover tmp file. The leftover tmp file may be due to killing the previous run.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we allow the file to exist, at least we should truncate it. Otherwise leftover content may be included in the final file.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You are right -- we need TRUNCATE_EXISTING.

  0: 0123456789
  0: aaaaaa6789
  0: bbb
  static void main(String[] args) throws Exception {
    final File f = new File("a.txt");
    try(OutputStream out = newOutputStreamForceAtClose(f, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE)) {
      out.write("0123456789".getBytes(StandardCharsets.UTF_8));
    }
    dump(f);
    try(OutputStream out = newOutputStreamForceAtClose(f, StandardOpenOption.CREATE, StandardOpenOption.WRITE)) {
      out.write("aaaaaa".getBytes(StandardCharsets.UTF_8));
    }
    dump(f);
    try(OutputStream out = newOutputStreamForceAtClose(f, StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.TRUNCATE_EXISTING)) {
      out.write("bbb".getBytes(StandardCharsets.UTF_8));
    }
    dump(f);
  }

  static void dump(File f) throws Exception {
    try(BufferedReader in = new BufferedReader(new InputStreamReader(
        newInputStream(f.toPath()), StandardCharsets.UTF_8))) {
      String line;
      for(int i = 0; (line = in.readLine()) != null; i++) {
        System.out.printf("%3d: %s%n", i, line);
      }
    }
  }

@adoroszlai adoroszlai changed the title RATIS-2260. AtomicFileOutputStream should verify temp file does not exist RATIS-2260. AtomicFileOutputStream should truncate temp file Mar 17, 2025
Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 the change looks good.

@szetszwo szetszwo merged commit 0bc5aad into apache:master Mar 17, 2025
14 checks passed
@adoroszlai adoroszlai deleted the RATIS-2260 branch March 17, 2025 18:05
@adoroszlai
Copy link
Contributor Author

Thanks @szetszwo for reviewing and merging this.

szetszwo pushed a commit to szetszwo/ratis that referenced this pull request Mar 30, 2025
szetszwo pushed a commit to szetszwo/ratis that referenced this pull request May 19, 2025
szetszwo pushed a commit to szetszwo/ratis that referenced this pull request May 23, 2025
szetszwo pushed a commit to szetszwo/ratis that referenced this pull request May 23, 2025
szetszwo pushed a commit to szetszwo/ratis that referenced this pull request May 23, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants