forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-35217] Add missing fsync to #closeForCommit in some subclasses…
… of RecoverableFsDataOutputStream. (apache#24722)
- Loading branch information
1 parent
8a75f8f
commit 80af4d5
Showing
9 changed files
with
578 additions
and
24 deletions.
There are no files selected for viewing
59 changes: 59 additions & 0 deletions
59
...main/java/org/apache/flink/core/fs/CommitterFromPersistRecoverableFsDataOutputStream.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
97 changes: 97 additions & 0 deletions
97
...c/test/java/org/apache/flink/core/fs/local/AbstractRecoverableFsDataOutputStreamTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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); | ||
} | ||
} |
Oops, something went wrong.