Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nullable;
import javax.annotation.concurrent.NotThreadSafe;

import java.io.BufferedOutputStream;
Expand Down Expand Up @@ -95,6 +96,29 @@ public NativeS3RecoverableFsDataOutputStream(
List<PartETag> existingParts,
long numBytesInParts)
throws IOException {
this(
s3AccessHelper,
key,
uploadId,
localTmpDir,
minPartSize,
existingParts,
numBytesInParts,
null,
0L);
}

public NativeS3RecoverableFsDataOutputStream(
NativeS3ObjectOperations s3AccessHelper,
String key,
String uploadId,
String localTmpDir,
long minPartSize,
List<PartETag> existingParts,
long numBytesInParts,
@Nullable File incompleteTailFile,
long incompleteTailLength)
throws IOException {
this.s3AccessHelper = s3AccessHelper;
this.key = key;
this.uploadId = uploadId;
Expand All @@ -106,7 +130,32 @@ public NativeS3RecoverableFsDataOutputStream(
this.currentPartSize = 0;
this.closed = false;

createNewTempFile();
if (incompleteTailFile != null) {
resumeFromIncompleteTail(incompleteTailFile, incompleteTailLength);
} else {
createNewTempFile();
}
}

private void resumeFromIncompleteTail(File tailFile, long expectedLength) throws IOException {
if (!tailFile.exists()) {
throw new IOException("Incomplete-tail file does not exist: " + tailFile);
}
final long actualLength = tailFile.length();
if (actualLength != expectedLength) {
throw new IOException(
"Incomplete-tail file "
+ tailFile
+ " has unexpected length: expected "
+ expectedLength
+ " got "
+ actualLength);
}
currentTempFile = tailFile;
// Append mode so subsequent writes land after the recovered bytes.
currentFileStream = new FileOutputStream(currentTempFile, true);
currentOutputStream = new BufferedOutputStream(currentFileStream, BUFFER_SIZE);
currentPartSize = expectedLength;
}

private void createNewTempFile() throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicBoolean;

import static org.apache.flink.util.Preconditions.checkNotNull;
Expand Down Expand Up @@ -84,15 +87,93 @@ public RecoverableFsDataOutputStream.Committer recoverForCommit(CommitRecoverabl
@Override
public RecoverableFsDataOutputStream recover(ResumeRecoverable recoverable) throws IOException {
checkNotClosed();
NativeS3Recoverable s3recoverable = castToNativeS3Recoverable(recoverable);
return new NativeS3RecoverableFsDataOutputStream(
s3AccessHelper,
s3recoverable.getObjectName(),
s3recoverable.uploadId(),
localTmpDir,
userDefinedMinPartSize,
s3recoverable.parts(),
s3recoverable.numBytesInParts());
final NativeS3Recoverable s3recoverable = castToNativeS3Recoverable(recoverable);

File incompleteTail = null;
long incompleteTailLength = 0L;
if (s3recoverable.incompleteObjectName() != null) {
incompleteTail = downloadIncompleteTail(s3recoverable);
incompleteTailLength = s3recoverable.incompleteObjectLength();
}

try {
LOG.debug(
"Resuming stream - key: {}, uploadId: {}, parts: {}, bytesInParts: {}, incompleteTail: {} ({} bytes)",
s3recoverable.getObjectName(),
s3recoverable.uploadId(),
s3recoverable.parts().size(),
s3recoverable.numBytesInParts(),
s3recoverable.incompleteObjectName(),
incompleteTailLength);
return new NativeS3RecoverableFsDataOutputStream(
s3AccessHelper,
s3recoverable.getObjectName(),
s3recoverable.uploadId(),
localTmpDir,
userDefinedMinPartSize,
s3recoverable.parts(),
s3recoverable.numBytesInParts(),
incompleteTail,
incompleteTailLength);
} catch (Throwable t) {
// The downloaded tail file is owned by recover() until the constructor takes
// ownership. If construction fails, drop the local file so we don't leak it.
if (incompleteTail != null) {
try {
Files.deleteIfExists(incompleteTail.toPath());
} catch (IOException cleanup) {
t.addSuppressed(cleanup);
}
}
throw t;
}
}

/**
* Downloads the side object holding the previously-persisted sub-part-size tail into a fresh
* file under {@link #localTmpDir}. The side object itself is left in place so that a repeated
* recovery from the same checkpoint remains correct; cleanup is the responsibility of {@link
* #cleanupRecoverableState(ResumeRecoverable)} which Flink invokes when the checkpoint is
* retired.
*/
private File downloadIncompleteTail(NativeS3Recoverable s3recoverable) throws IOException {
final File tmpDir = new File(localTmpDir);
if (!tmpDir.exists() && !tmpDir.mkdirs()) {
throw new IOException("Cannot create local tmp dir: " + localTmpDir);
}
final File target = new File(tmpDir, "s3-resume-" + UUID.randomUUID());
try {
final long downloaded =
s3AccessHelper.getObject(s3recoverable.incompleteObjectName(), target);
if (downloaded != s3recoverable.incompleteObjectLength()) {
throw new IOException(
"Incomplete-tail object "
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This exception doesn't tell what are the implications. Does it mean that state is corrupted and can't be recovered unless object on S3 is restored? If so, would be useful to explain it. Would help both oncall engineer and to classify such errors correctly (retriable/non-retriable)

+ s3recoverable.incompleteObjectName()
+ " has unexpected length (expected "
+ s3recoverable.incompleteObjectLength()
+ " bytes, got "
+ downloaded
+ " bytes). The side object holding the in-flight tail "
+ "has been truncated, overwritten, or replaced out-of-band "
+ "since the checkpoint was taken. Recovery cannot proceed: "
+ "the writer state is inconsistent with the checkpoint and "
+ "this failure is NOT retriable from the same checkpoint. "
+ "Either restore the side object to its original length or "
+ "roll back to an earlier checkpoint that does not reference it.");
}
return target;
} catch (IOException e) {
try {
Files.deleteIfExists(target.toPath());
} catch (IOException cleanup) {
LOG.warn(
"Failed to delete partial download {} after error: {}",
target,
cleanup.getMessage());
e.addSuppressed(cleanup);
}
throw e;
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,165 @@
/*
* 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.fs.s3native.writer;

import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;

/**
* In-memory implementation for {@link NativeS3ObjectOperations}.
*
* <p>Backs every reachable S3 operation with hash maps so writer/committer logic can be exercised
* without an S3 endpoint (no MinIO/Testcontainers required). The parent's {@code S3Client} / {@code
* S3TransferManager} constructor arguments are passed as {@code null} because no overridden method
* dereferences them.
*
* <p><b>State exposure:</b> the storage maps are exposed as public final fields so tests can
* inspect them, corrupt them, or simulate object loss directly:
*
* <ul>
* <li>{@link #storedObjects} — keys written via {@link #putObject(String, File)} (e.g. the
* incomplete-tail side objects persisted by {@link NativeS3RecoverableFsDataOutputStream}).
* <li>{@link #committedObjects} — keys finalized via {@link #commitMultiPartUpload}.
* <li>{@link #openMultipartUploads} — uploadId → partNumber → bytes for in-flight MPUs; entries
* are removed on commit or abort.
* </ul>
*
* <p>{@link #getObject} reads from <em>both</em> {@link #storedObjects} and {@link
* #committedObjects} so tests can fetch a committed object the same way real S3 would serve it.
*
* <p><b>Thread safety:</b> not thread-safe. Use a single thread per instance, matching the
* single-thread invariant of the production {@link NativeS3RecoverableFsDataOutputStream}.
*/
public final class InMemoryNativeS3Operations extends NativeS3ObjectOperations {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I think it would it be better to have NativeS3ObjectOperations as interface otherwise you still would need to bring in all sdk based implementation dependencies.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Agreed. An interface would make the test seam cleaner and keep SDK types off the test classpath. The reason I didn't do it here is scope:

NativeS3ObjectOperations returns SDK types directly (CompletedPart, UploadPartResponse, PartETag-like records), so introducing an interface means either

(a) extracting Flink-owned DTOs to replace those return types across NativeS3RecoverableFsDataOutputStream / NativeS3Committer / NativeS3RecoverableWriter, or

(b) leaving the SDK types in the interface signatures , which doesn't actually remove the dependency.

Both options are meaningful refactors that I'd rather not bundle into a data-loss bugfix. For now the test subclass passes null for the SDK ctor args and overrides every method it touches, so no SDK client is constructed at test time (the SDK is only on the compile classpath, which it already is for main code). I'll file a follow-up to do the interface extraction properly — happy to take it on right after this lands. WDYT?

Checked https://github.com/localstack/localstack. It is not maintained anymore.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Yes, makes sense to decouple as a separate PR

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

If this is meant to be used as test harness for FileSystem testing (as replacement of localStack). Arguably it is good to have tests for it too


public static final String DEFAULT_BUCKET = "test-bucket";

/** Keys written via {@link #putObject(String, File)}. */
public final Map<String, byte[]> storedObjects = new HashMap<>();

/** Keys finalized via {@link #commitMultiPartUpload}. */
public final Map<String, byte[]> committedObjects = new HashMap<>();

/** uploadId → partNumber → uploaded bytes for in-flight MPUs. */
public final Map<String, Map<Integer, byte[]>> openMultipartUploads = new HashMap<>();

private final String bucketName;
private final AtomicInteger uploadIdSeq = new AtomicInteger();
private final AtomicInteger putObjectSeq = new AtomicInteger();

public InMemoryNativeS3Operations() {
this(DEFAULT_BUCKET);
}

public InMemoryNativeS3Operations(String bucketName) {
super(/* s3Client */ null, /* transferManager */ null, bucketName, /* useAsync */ false);
this.bucketName = bucketName;
}

@Override
public String startMultiPartUpload(String key) {
String uploadId = "U" + uploadIdSeq.incrementAndGet();
openMultipartUploads.put(uploadId, new HashMap<>());
return uploadId;
}

@Override
public UploadPartResult uploadPart(
String key, String uploadId, int partNumber, File file, long length)
throws IOException {
Map<Integer, byte[]> parts = openMultipartUploads.get(uploadId);
if (parts == null) {
throw new IOException("unknown uploadId: " + uploadId);
}
byte[] data = Files.readAllBytes(file.toPath());
if (data.length != length) {
throw new IOException(
"part length mismatch: expected " + length + ", got " + data.length);
}
parts.put(partNumber, data);
return new UploadPartResult(partNumber, "etag-" + uploadId + "-" + partNumber);
}

@Override
public PutObjectResult putObject(String key, File file) throws IOException {
storedObjects.put(key, Files.readAllBytes(file.toPath()));
return new PutObjectResult("etag-put-" + putObjectSeq.incrementAndGet());
}

@Override
public long getObject(String key, File targetLocation) throws IOException {
byte[] data = storedObjects.get(key);
if (data == null) {
data = committedObjects.get(key);
}
if (data == null) {
throw new IOException("not found: " + key);
}
Files.write(targetLocation.toPath(), data);
return data.length;
}

@Override
public CompleteMultipartUploadResult commitMultiPartUpload(
String key, String uploadId, List<UploadPartResult> parts, long length)
throws IOException {
Map<Integer, byte[]> uploaded = openMultipartUploads.remove(uploadId);
if (uploaded == null) {
throw new IOException("unknown uploadId: " + uploadId);
}
List<Integer> ordered = new ArrayList<>(parts.size());
for (UploadPartResult p : parts) {
ordered.add(p.getPartNumber());
}
Collections.sort(ordered);
ByteArrayOutputStream merged = new ByteArrayOutputStream();
for (int n : ordered) {
byte[] partData = uploaded.get(n);
if (partData == null) {
throw new IOException("missing part " + n + " for uploadId " + uploadId);
}
merged.write(partData);
}
byte[] finalBytes = merged.toByteArray();
if (finalBytes.length != length) {
throw new IOException(
"committed length mismatch: expected " + length + ", got " + finalBytes.length);
}
committedObjects.put(key, finalBytes);
return new CompleteMultipartUploadResult(bucketName, key, "final-etag-" + uploadId, null);
}

@Override
public void abortMultiPartUpload(String key, String uploadId) {
openMultipartUploads.remove(uploadId);
}

@Override
public boolean deleteObject(String key) {
return storedObjects.remove(key) != null || committedObjects.remove(key) != null;
}
}
Loading