Izeren commented on code in PR #28268:
URL: https://github.com/apache/flink/pull/28268#discussion_r3325618815


##########
flink-filesystems/flink-s3-fs-native/src/main/java/org/apache/flink/fs/s3native/writer/NativeS3RecoverableFsDataOutputStream.java:
##########
@@ -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 seedTailFile,

Review Comment:
   !nit, not sure that `seedTailFile` explains me the purpose. Could be 
incompletePartFile, incompleteSubPartFile or something along the lines. 



##########
flink-filesystems/flink-s3-fs-native/src/main/java/org/apache/flink/fs/s3native/writer/NativeS3RecoverableFsDataOutputStream.java:
##########
@@ -106,7 +130,32 @@ public NativeS3RecoverableFsDataOutputStream(
         this.currentPartSize = 0;
         this.closed = false;
 
-        createNewTempFile();
+        if (seedTailFile != null) {
+            adoptSeedTailFile(seedTailFile, seedTailLength);
+        } else {
+            createNewTempFile();
+        }
+    }
+
+    private void adoptSeedTailFile(File seedFile, long expectedLength) throws 
IOException {

Review Comment:
   Do we re-use `currentTempFile` . If not I'd rather return file path from 
this method (and from `createTempFile`) to avoid risks of messing up 
initialization of the filePath



##########
flink-filesystems/flink-s3-fs-native/src/main/java/org/apache/flink/fs/s3native/writer/NativeS3RecoverableWriter.java:
##########
@@ -85,14 +88,85 @@ public RecoverableFsDataOutputStream.Committer 
recoverForCommit(CommitRecoverabl
     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());
+
+        File seedTail = null;
+        long seedTailLength = 0L;
+        if (s3recoverable.incompleteObjectName() != null) {
+            seedTail = downloadIncompleteTail(s3recoverable);
+            seedTailLength = s3recoverable.incompleteObjectLength();
+        }
+
+        try {
+            LOG.debug(
+                    "Resuming stream - key: {}, uploadId: {}, parts: {}, 
bytesInParts: {}, seedTail: {} ({} bytes)",
+                    s3recoverable.getObjectName(),
+                    s3recoverable.uploadId(),
+                    s3recoverable.parts().size(),
+                    s3recoverable.numBytesInParts(),
+                    s3recoverable.incompleteObjectName(),
+                    seedTailLength);
+            return new NativeS3RecoverableFsDataOutputStream(
+                    s3AccessHelper,
+                    s3recoverable.getObjectName(),
+                    s3recoverable.uploadId(),
+                    localTmpDir,
+                    userDefinedMinPartSize,
+                    s3recoverable.parts(),
+                    s3recoverable.numBytesInParts(),
+                    seedTail,
+                    seedTailLength);
+        } catch (Throwable t) {
+            // The downloaded seed file is owned by recover() until the 
constructor takes
+            // ownership. If construction fails, drop the local file so we 
don't leak it.
+            if (seedTail != null) {
+                try {
+                    Files.deleteIfExists(seedTail.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 {
+        File tmpDir = new File(localTmpDir);

Review Comment:
   !nit, final?



##########
flink-filesystems/flink-s3-fs-native/src/test/java/org/apache/flink/fs/s3native/writer/InMemoryNativeS3Operations.java:
##########
@@ -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.UUID;
+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 
{
+
+    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();
+
+    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-" + UUID.randomUUID());

Review Comment:
   Do we need some control over: `UUID.randomUUID()` in tests for determinism?



##########
flink-filesystems/flink-s3-fs-native/src/test/java/org/apache/flink/fs/s3native/writer/InMemoryNativeS3Operations.java:
##########
@@ -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.UUID;
+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 
{

Review Comment:
   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



##########
flink-filesystems/flink-s3-fs-native/src/main/java/org/apache/flink/fs/s3native/writer/NativeS3RecoverableWriter.java:
##########
@@ -85,14 +88,85 @@ public RecoverableFsDataOutputStream.Committer 
recoverForCommit(CommitRecoverabl
     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());
+
+        File seedTail = null;
+        long seedTailLength = 0L;
+        if (s3recoverable.incompleteObjectName() != null) {
+            seedTail = downloadIncompleteTail(s3recoverable);
+            seedTailLength = s3recoverable.incompleteObjectLength();
+        }
+
+        try {
+            LOG.debug(
+                    "Resuming stream - key: {}, uploadId: {}, parts: {}, 
bytesInParts: {}, seedTail: {} ({} bytes)",
+                    s3recoverable.getObjectName(),
+                    s3recoverable.uploadId(),
+                    s3recoverable.parts().size(),
+                    s3recoverable.numBytesInParts(),
+                    s3recoverable.incompleteObjectName(),
+                    seedTailLength);
+            return new NativeS3RecoverableFsDataOutputStream(
+                    s3AccessHelper,
+                    s3recoverable.getObjectName(),
+                    s3recoverable.uploadId(),
+                    localTmpDir,
+                    userDefinedMinPartSize,
+                    s3recoverable.parts(),
+                    s3recoverable.numBytesInParts(),
+                    seedTail,
+                    seedTailLength);
+        } catch (Throwable t) {
+            // The downloaded seed file is owned by recover() until the 
constructor takes
+            // ownership. If construction fails, drop the local file so we 
don't leak it.
+            if (seedTail != null) {
+                try {
+                    Files.deleteIfExists(seedTail.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 {
+        File tmpDir = new File(localTmpDir);
+        if (!tmpDir.exists() && !tmpDir.mkdirs()) {
+            throw new IOException("Cannot create local tmp dir: " + 
localTmpDir);
+        }
+        File target = new File(tmpDir, "s3-resume-" + UUID.randomUUID());

Review Comment:
   !int, final?



##########
flink-filesystems/flink-s3-fs-native/src/main/java/org/apache/flink/fs/s3native/writer/NativeS3RecoverableWriter.java:
##########
@@ -85,14 +88,85 @@ public RecoverableFsDataOutputStream.Committer 
recoverForCommit(CommitRecoverabl
     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());
+
+        File seedTail = null;
+        long seedTailLength = 0L;
+        if (s3recoverable.incompleteObjectName() != null) {
+            seedTail = downloadIncompleteTail(s3recoverable);
+            seedTailLength = s3recoverable.incompleteObjectLength();
+        }
+
+        try {
+            LOG.debug(
+                    "Resuming stream - key: {}, uploadId: {}, parts: {}, 
bytesInParts: {}, seedTail: {} ({} bytes)",
+                    s3recoverable.getObjectName(),
+                    s3recoverable.uploadId(),
+                    s3recoverable.parts().size(),
+                    s3recoverable.numBytesInParts(),
+                    s3recoverable.incompleteObjectName(),
+                    seedTailLength);
+            return new NativeS3RecoverableFsDataOutputStream(
+                    s3AccessHelper,
+                    s3recoverable.getObjectName(),
+                    s3recoverable.uploadId(),
+                    localTmpDir,
+                    userDefinedMinPartSize,
+                    s3recoverable.parts(),
+                    s3recoverable.numBytesInParts(),
+                    seedTail,
+                    seedTailLength);
+        } catch (Throwable t) {
+            // The downloaded seed file is owned by recover() until the 
constructor takes
+            // ownership. If construction fails, drop the local file so we 
don't leak it.
+            if (seedTail != null) {
+                try {
+                    Files.deleteIfExists(seedTail.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 {
+        File tmpDir = new File(localTmpDir);
+        if (!tmpDir.exists() && !tmpDir.mkdirs()) {
+            throw new IOException("Cannot create local tmp dir: " + 
localTmpDir);
+        }
+        File target = new File(tmpDir, "s3-resume-" + UUID.randomUUID());
+        try {
+            long downloaded =
+                    
s3AccessHelper.getObject(s3recoverable.incompleteObjectName(), target);
+            if (downloaded != s3recoverable.incompleteObjectLength()) {
+                throw new IOException(
+                        "Incomplete-tail object "

Review Comment:
   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)



##########
flink-filesystems/flink-s3-fs-native/src/main/java/org/apache/flink/fs/s3native/writer/NativeS3RecoverableFsDataOutputStream.java:
##########
@@ -106,7 +130,32 @@ public NativeS3RecoverableFsDataOutputStream(
         this.currentPartSize = 0;
         this.closed = false;
 
-        createNewTempFile();
+        if (seedTailFile != null) {
+            adoptSeedTailFile(seedTailFile, seedTailLength);

Review Comment:
   !nit, similarly, I would use naming closer to `init`/`restore`/`resume` 
rather than `adopt` 



##########
flink-filesystems/flink-s3-fs-native/src/test/java/org/apache/flink/fs/s3native/writer/NativeS3RecoverableWriterRecoveryTest.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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 org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.RecoverableFsDataOutputStream;
+import org.apache.flink.core.fs.RecoverableWriter;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link NativeS3RecoverableWriter#recover}. */
+class NativeS3RecoverableWriterRecoveryTest {
+
+    private static final String BUCKET = 
InMemoryNativeS3Operations.DEFAULT_BUCKET;
+    private static final String KEY = "out.txt";
+    private static final long MIN_PART_SIZE = 10L;
+
+    @TempDir java.nio.file.Path tmp;
+
+    @Test
+    void persistThenRecoverPreservesTailBytes() throws Exception {
+        InMemoryNativeS3Operations s3 = new InMemoryNativeS3Operations();
+        NativeS3RecoverableWriter writer1 =
+                NativeS3RecoverableWriter.writer(
+                        s3, tmp.toString(), MIN_PART_SIZE, /* maxConcurrent */ 
1);

Review Comment:
   What is the impact of concurrency on these incomplete subparts? How many of 
incomplete subparts can exist per file path at the same time?



##########
flink-filesystems/flink-s3-fs-native/src/test/java/org/apache/flink/fs/s3native/writer/InMemoryNativeS3Operations.java:
##########
@@ -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.UUID;
+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 
{

Review Comment:
   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. 



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to