Skip to content

Commit

Permalink
[FLINK-35217] Add missing fsync to #closeForCommit in some subclasses…
Browse files Browse the repository at this point in the history
… of RecoverableFsDataOutputStream. (apache#24722)
  • Loading branch information
StefanRRichter committed Apr 30, 2024
1 parent 8a75f8f commit 80af4d5
Show file tree
Hide file tree
Showing 9 changed files with 578 additions and 24 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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.flink.core.fs;

import java.io.IOException;

/**
* {@link RecoverableFsDataOutputStream} with fixed implementation of {@link #closeForCommit()} that
* is based on using {@link #persist()} to ensure durability and creates the {@link
* org.apache.flink.core.fs.RecoverableFsDataOutputStream.Committer} from the corresponding {@link
* org.apache.flink.core.fs.RecoverableWriter.ResumeRecoverable}.
*
* @param <RESUME_RECOVERABLE> return type of #persist()
*/
public abstract class CommitterFromPersistRecoverableFsDataOutputStream<
RESUME_RECOVERABLE extends RecoverableWriter.ResumeRecoverable>
extends RecoverableFsDataOutputStream {

/** @see RecoverableFsDataOutputStream#persist() */
@Override
public abstract RESUME_RECOVERABLE persist() throws IOException;

/**
* @see RecoverableFsDataOutputStream#closeForCommit()
* @param recoverable a resume recoverable to create the committer from. Typically the parameter
* is the return value of {@link #persist()}.
* @return the committer created from recoverable.
*/
protected abstract Committer createCommitterFromResumeRecoverable(
RESUME_RECOVERABLE recoverable);

/**
* @see RecoverableFsDataOutputStream#closeForCommit()
* @implNote Calls persist to ensure durability of the written data and creates a committer
* object from the return value of {@link #persist()}.
*/
@Override
public final Committer closeForCommit() throws IOException {
Committer committer = createCommitterFromResumeRecoverable(persist());
close();
return committer;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,10 @@
package org.apache.flink.core.fs.local;

import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.core.fs.CommitterFromPersistRecoverableFsDataOutputStream;
import org.apache.flink.core.fs.RecoverableFsDataOutputStream;
import org.apache.flink.core.fs.RecoverableWriter.CommitRecoverable;
import org.apache.flink.core.fs.RecoverableWriter.ResumeRecoverable;

import java.io.File;
import java.io.FileNotFoundException;
Expand All @@ -40,7 +41,8 @@

/** A {@link RecoverableFsDataOutputStream} for the {@link LocalFileSystem}. */
@Internal
public class LocalRecoverableFsDataOutputStream extends RecoverableFsDataOutputStream {
public class LocalRecoverableFsDataOutputStream
extends CommitterFromPersistRecoverableFsDataOutputStream<LocalRecoverable> {

private final File targetFile;

Expand Down Expand Up @@ -78,6 +80,15 @@ public LocalRecoverableFsDataOutputStream(File targetFile, File tempFile) throws
this.fos = Channels.newOutputStream(fileChannel);
}

@VisibleForTesting
LocalRecoverableFsDataOutputStream(
File targetFile, File tempFile, FileChannel fileChannel, OutputStream fos) {
this.targetFile = checkNotNull(targetFile);
this.tempFile = checkNotNull(tempFile);
this.fileChannel = fileChannel;
this.fos = fos;
}

@Override
public void write(int b) throws IOException {
fos.write(b);
Expand All @@ -104,7 +115,7 @@ public long getPos() throws IOException {
}

@Override
public ResumeRecoverable persist() throws IOException {
public LocalRecoverable persist() throws IOException {
// we call both flush and sync in order to ensure persistence on mounted
// file systems, like NFS, EBS, EFS, ...
flush();
Expand All @@ -114,10 +125,8 @@ public ResumeRecoverable persist() throws IOException {
}

@Override
public Committer closeForCommit() throws IOException {
final long pos = getPos();
close();
return new LocalCommitter(new LocalRecoverable(targetFile, tempFile, pos));
protected Committer createCommitterFromResumeRecoverable(LocalRecoverable recoverable) {
return new LocalCommitter(recoverable);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
/*
* 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.flink.core.fs.local;

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.core.fs.RecoverableFsDataOutputStream;
import org.apache.flink.util.FileUtils;

import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;

import java.io.Closeable;
import java.io.IOException;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Random;

/** Base class for testing implementations of {@link RecoverableFsDataOutputStream}. */
public abstract class AbstractRecoverableFsDataOutputStreamTest {

public enum Event {
CLOSE,
FLUSH,
SYNC
}

@TempDir Path tmp;

/**
* Tests that #closeForCommit leads to a durable write to the temporary file and to target on
* commit.
*/
@Test
public void testDurableWriteOnCommit() throws IOException {
// Setup
final int seed = 4711;
final Random random = new Random(seed);
final byte[] buffer = new byte[4 * 4096];
final List<LocalRecoverableFsDataOutputStreamTest.Event> testLog = new ArrayList<>();
final Path target = tmp.resolve("target");
final Path temp = tmp.resolve("temp");

Tuple2<RecoverableFsDataOutputStream, Closeable> testInstance =
createTestInstance(target, temp, testLog);

// Create test object
final RecoverableFsDataOutputStream testOutStreamInstance = testInstance.f0;

// Write test data
random.nextBytes(buffer);
testOutStreamInstance.write(buffer);

// Test closeForCommit
Assertions.assertTrue(testLog.isEmpty());
RecoverableFsDataOutputStream.Committer committer = testOutStreamInstance.closeForCommit();
Assertions.assertEquals(getExpectedResult(), testLog);

testInstance.f1.close();
Assertions.assertArrayEquals(buffer, FileUtils.readAllBytes(temp));

// Test commit
Assertions.assertFalse(target.toFile().exists());
committer.commit();
Assertions.assertTrue(target.toFile().exists());
Assertions.assertArrayEquals(buffer, FileUtils.readAllBytes(target));
}

public abstract Tuple2<RecoverableFsDataOutputStream, Closeable> createTestInstance(
Path target, Path temp, List<LocalRecoverableFsDataOutputStreamTest.Event> testLog)
throws IOException;

public List<Event> getExpectedResult() {
return Arrays.asList(
LocalRecoverableFsDataOutputStreamTest.Event.FLUSH,
LocalRecoverableFsDataOutputStreamTest.Event.SYNC,
LocalRecoverableFsDataOutputStreamTest.Event.CLOSE);
}
}

0 comments on commit 80af4d5

Please sign in to comment.