crodier commented on a change in pull request #5344: Fix races in
LookupSnapshotTaker, CoordinatorPollingBasicAuthenticatorCacheManager
URL: https://github.com/apache/incubator-druid/pull/5344#discussion_r299955603
##########
File path: java-util/src/main/java/io/druid/java/util/common/FileUtils.java
##########
@@ -150,15 +156,76 @@ public void addFile(File file)
* }}</pre>
*
* @param file the file to map
+ *
* @return a {@link MappedByteBufferHandler}, wrapping a read-only buffer
reflecting {@code file}
- * @throws FileNotFoundException if the {@code file} does not exist
- * @throws IOException if an I/O error occurs
*
+ * @throws FileNotFoundException if the {@code file} does not exist
+ * @throws IOException if an I/O error occurs
* @see FileChannel#map(FileChannel.MapMode, long, long)
*/
public static MappedByteBufferHandler map(File file) throws IOException
{
MappedByteBuffer mappedByteBuffer = Files.map(file);
return new MappedByteBufferHandler(mappedByteBuffer);
}
+
+ /**
+ * Write to a file atomically, by first writing to a temporary file in the
same directory and then moving it to
+ * the target location. This function attempts to clean up its temporary
files when possible, but they may stick
+ * around (for example, if the JVM crashes partway through executing the
function). In any case, the target file
+ * should be unharmed.
+ *
+ * The OutputStream passed to the consumer is uncloseable; calling close on
it will do nothing. This is to ensure
+ * that the stream stays open so we can fsync it here before closing.
Hopefully, this doesn't cause any problems
+ * for callers.
+ *
+ * This method is not just thread-safe, but is also safe to use from
multiple processes on the same machine.
+ */
+ public static void writeAtomically(final File file, OutputStreamConsumer f)
throws IOException
+ {
+ writeAtomically(file, file.getParentFile(), f);
+ }
+
+ private static void writeAtomically(final File file, final File tmpDir,
OutputStreamConsumer f) throws IOException
+ {
+ final File tmpFile = new File(tmpDir, String.format(".%s.%s",
file.getName(), UUID.randomUUID()));
+
+ try {
+ try (final FileOutputStream out = new FileOutputStream(tmpFile)) {
+ // Pass f an uncloseable stream so we can fsync before closing.
+ f.accept(uncloseable(out));
+
+ // fsync to avoid write-then-rename-then-crash causing empty files on
some filesystems.
+ out.getChannel().force(true);
+ }
+
+ // No exception thrown; do the move.
+ java.nio.file.Files.move(
+ tmpFile.toPath(),
+ file.toPath(),
+ StandardCopyOption.ATOMIC_MOVE,
+ StandardCopyOption.REPLACE_EXISTING
+ );
+ }
+ finally {
+ tmpFile.delete();
+ }
+ }
+
+ private static OutputStream uncloseable(final OutputStream out) throws
IOException
+ {
+ return new FilterOutputStream(out)
Review comment:
In unit tests locally with 15mb files, using a BufferedOutputStream here,
takes 1 second vs. ~16 seconds. FilterOutputStream isn't buffered, is too slow
for practical use?
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]