pnowojski commented on a change in pull request #15322:
URL: https://github.com/apache/flink/pull/15322#discussion_r668878534



##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.changelog.fs.StateChangeUploader.UploadTask;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.ChangelogStateHandleStreamImpl;
+import org.apache.flink.runtime.state.changelog.SequenceNumber;
+import org.apache.flink.runtime.state.changelog.SequenceNumberRange;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.changelog.StateChangelogWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Filesystem-based {@link StateChangelogWriter} implementation. Assumes 
TM-owned state - so no
+ * re-uploads.
+ *
+ * <p>On {@link #append(int, byte[]) append}, it stores the changes locally in 
memory (without any
+ * thread synchronization); {@link SequenceNumber} is not changed.
+ *
+ * <p>However, if they exceed {@link #preEmptivePersistThresholdInBytes} then 
{@link
+ * #persistInternal(SequenceNumber) persist} is called.
+ *
+ * <p>On {@link #persist(SequenceNumber) persist}, accumulated changes are 
sent to the {@link
+ * StateChangeUploader} as an immutable {@link StateChangeUploader.UploadTask 
task}. An {@link
+ * FsStateChangelogWriter.UploadCompletionListener upload listener} is also 
registered. Upon
+ * notification it updates the Writer local state (for future persist calls) 
and completes the
+ * future returned to the original caller. The uploader notifies all listeners 
via a callback in a
+ * task.
+ *
+ * <p>If persist() is called for the same state change before the upload 
completion then the
+ * listener is added but not the upload task (which must already exist).
+ *
+ * <p>Invariants:
+ *
+ * <ol>
+ *   <li>Every change has at most one associated upload (retries are performed 
at a lower level)
+ *   <li>Every change is present in at most one collection: either uploaded OR 
notUploaded

Review comment:
       nit `{@link #uploaded}` and `notUploaded`? (easier to keep the java doc 
up to date)

##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.changelog.fs.StateChangeUploader.UploadTask;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.ChangelogStateHandleStreamImpl;
+import org.apache.flink.runtime.state.changelog.SequenceNumber;
+import org.apache.flink.runtime.state.changelog.SequenceNumberRange;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.changelog.StateChangelogWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Filesystem-based {@link StateChangelogWriter} implementation. Assumes 
TM-owned state - so no
+ * re-uploads.
+ *
+ * <p>On {@link #append(int, byte[]) append}, it stores the changes locally in 
memory (without any
+ * thread synchronization); {@link SequenceNumber} is not changed.
+ *
+ * <p>However, if they exceed {@link #preEmptivePersistThresholdInBytes} then 
{@link
+ * #persistInternal(SequenceNumber) persist} is called.
+ *
+ * <p>On {@link #persist(SequenceNumber) persist}, accumulated changes are 
sent to the {@link
+ * StateChangeUploader} as an immutable {@link StateChangeUploader.UploadTask 
task}. An {@link
+ * FsStateChangelogWriter.UploadCompletionListener upload listener} is also 
registered. Upon
+ * notification it updates the Writer local state (for future persist calls) 
and completes the
+ * future returned to the original caller. The uploader notifies all listeners 
via a callback in a
+ * task.
+ *
+ * <p>If persist() is called for the same state change before the upload 
completion then the
+ * listener is added but not the upload task (which must already exist).
+ *
+ * <p>Invariants:
+ *
+ * <ol>
+ *   <li>Every change has at most one associated upload (retries are performed 
at a lower level)
+ *   <li>Every change is present in at most one collection: either uploaded OR 
notUploaded
+ *   <li>Changes BEING uploaded are NOT referenced locally - they will be 
added to uploaded upon
+ *       completion
+ *   <li>Failed and truncated changes are NOT stored - only their respective 
highest sequence
+ *       numbers
+ * </ol>
+ */
+@NotThreadSafe
+class FsStateChangelogWriter implements 
StateChangelogWriter<ChangelogStateHandleStreamImpl> {
+    private static final Logger LOG = 
LoggerFactory.getLogger(FsStateChangelogWriter.class);
+    private static final SequenceNumber INITIAL_SQN = SequenceNumber.of(0L);
+
+    private final UUID logId;
+    private final KeyGroupRange keyGroupRange;
+    private final StateChangeUploader uploader;
+    private final long preEmptivePersistThresholdInBytes;
+
+    /** Lock to synchronize handling of upload completion with new upload 
requests. */
+    // todo: replace with mailbox executor (after FLINK-23204)
+    private final Object lock = new Object();
+
+    /** A list of listener per upload (~ per checkpoint plus pre-emptive 
uploads). */
+    @GuardedBy("lock")
+    private final List<UploadCompletionListener> uploadCompletionListeners = 
new ArrayList<>();
+
+    private final FsStateChangelogCleaner cleaner;
+
+    /** Current {@link SequenceNumber}. */
+    private SequenceNumber activeSequenceNumber = INITIAL_SQN;
+
+    /**
+     * {@link SequenceNumber} before which changes will NOT be requested, 
exclusive. Increased after
+     * materialization.
+     */
+    @GuardedBy("lock")
+    private SequenceNumber lowestSequenceNumber = INITIAL_SQN;
+
+    /** Active changes, that will all use {@link #activeSequenceNumber}. */
+    private List<StateChange> activeChangeSet = new ArrayList<>();
+
+    /** {@link #activeChangeSet} size in bytes. */
+    private long activeChangeSetSize;
+
+    /** Changes that are not yet uploaded (upload not requested). */
+    private final NavigableMap<SequenceNumber, StateChangeSet> notUploaded = 
new TreeMap<>();
+
+    /** Uploaded changes, ready for use in snapshots. */
+    @GuardedBy("lock")
+    private final NavigableMap<SequenceNumber, UploadResult> uploaded = new 
TreeMap<>();
+
+    /**
+     * Highest {@link SequenceNumber} for which upload has failed (won't be 
restarted), inclusive.
+     */
+    @Nullable
+    @GuardedBy("lock")
+    private Tuple2<SequenceNumber, Throwable> highestFailed;
+
+    @GuardedBy("lock")
+    private boolean closed;
+
+    FsStateChangelogWriter(
+            UUID logId,
+            KeyGroupRange keyGroupRange,
+            StateChangeUploader uploader,
+            long preEmptivePersistThresholdInBytes,
+            FsStateChangelogCleaner cleaner) {
+        this.logId = logId;
+        this.keyGroupRange = keyGroupRange;
+        this.uploader = uploader;
+        this.preEmptivePersistThresholdInBytes = 
preEmptivePersistThresholdInBytes;
+        this.cleaner = cleaner;
+    }
+
+    @Override
+    public void append(int keyGroup, byte[] value) throws IOException {
+        LOG.trace("append to {}: keyGroup={} {} bytes", logId, keyGroup, 
value.length);
+        checkState(!closed, "%s is closed", logId);
+        activeChangeSet.add(new StateChange(keyGroup, value));
+        activeChangeSetSize += value.length;
+        if (activeChangeSetSize >= preEmptivePersistThresholdInBytes) {
+            LOG.debug(
+                    "pre-emptively flush {}Mb of appended changes to the 
common store",
+                    activeChangeSetSize / 1024 / 1024);
+            persistInternal(notUploaded.isEmpty() ? activeSequenceNumber : 
notUploaded.firstKey());
+        }
+    }
+
+    @Override
+    public SequenceNumber initialSequenceNumber() {
+        return INITIAL_SQN;
+    }
+
+    @Override
+    public SequenceNumber lastAppendedSequenceNumber() {
+        LOG.trace("query {} sqn: {}", logId, activeSequenceNumber);
+        SequenceNumber tmp = activeSequenceNumber;
+        rollover();

Review comment:
       I remember we have been already discussing this `rollover()` in a method 
that sounds like a getter, but I forgot the rationale.
   1. Can you remember me what's the reason behind it here?
   2. At the very least maybe add a comment explaining it?
   3. Maybe we can refactor the code to be more self-documenting without the 
need for the comment? Rename the method? Split it into two?

##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeUploader.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.changelog.SequenceNumber;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.flink.changelog.fs.FsStateChangelogOptions.BASE_PATH;
+import static 
org.apache.flink.changelog.fs.FsStateChangelogOptions.COMPRESSION_ENABLED;
+import static 
org.apache.flink.changelog.fs.FsStateChangelogOptions.UPLOAD_BUFFER_SIZE;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+// todo: consider using CheckpointStreamFactory / CheckpointStorageWorkerView
+//     Considerations:
+//     0. need for checkpointId in the current API to resolve the location
+//       option a: pass checkpointId (race condition?)
+//       option b: pass location (race condition?)
+//       option c: add FsCheckpointStorageAccess.createSharedStateStream
+//     1. different settings for materialized/changelog (e.g. timeouts)
+//     2. re-use closeAndGetHandle
+//     3. re-use in-memory handles (.metadata)
+//     4. handle in-memory handles duplication

Review comment:
       Besides a `todo`, could you explain the purpose of this interface?

##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.changelog.fs.StateChangeUploader.UploadTask;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.ChangelogStateHandleStreamImpl;
+import org.apache.flink.runtime.state.changelog.SequenceNumber;
+import org.apache.flink.runtime.state.changelog.SequenceNumberRange;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.changelog.StateChangelogWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Filesystem-based {@link StateChangelogWriter} implementation. Assumes 
TM-owned state - so no
+ * re-uploads.
+ *
+ * <p>On {@link #append(int, byte[]) append}, it stores the changes locally in 
memory (without any
+ * thread synchronization); {@link SequenceNumber} is not changed.
+ *
+ * <p>However, if they exceed {@link #preEmptivePersistThresholdInBytes} then 
{@link
+ * #persistInternal(SequenceNumber) persist} is called.
+ *
+ * <p>On {@link #persist(SequenceNumber) persist}, accumulated changes are 
sent to the {@link
+ * StateChangeUploader} as an immutable {@link StateChangeUploader.UploadTask 
task}. An {@link
+ * FsStateChangelogWriter.UploadCompletionListener upload listener} is also 
registered. Upon
+ * notification it updates the Writer local state (for future persist calls) 
and completes the
+ * future returned to the original caller. The uploader notifies all listeners 
via a callback in a
+ * task.
+ *
+ * <p>If persist() is called for the same state change before the upload 
completion then the
+ * listener is added but not the upload task (which must already exist).
+ *
+ * <p>Invariants:
+ *
+ * <ol>
+ *   <li>Every change has at most one associated upload (retries are performed 
at a lower level)
+ *   <li>Every change is present in at most one collection: either uploaded OR 
notUploaded

Review comment:
       what about `activeChangeSet`?

##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.changelog.fs.StateChangeUploader.UploadTask;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.ChangelogStateHandleStreamImpl;
+import org.apache.flink.runtime.state.changelog.SequenceNumber;
+import org.apache.flink.runtime.state.changelog.SequenceNumberRange;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.changelog.StateChangelogWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Filesystem-based {@link StateChangelogWriter} implementation. Assumes 
TM-owned state - so no
+ * re-uploads.
+ *
+ * <p>On {@link #append(int, byte[]) append}, it stores the changes locally in 
memory (without any
+ * thread synchronization); {@link SequenceNumber} is not changed.
+ *
+ * <p>However, if they exceed {@link #preEmptivePersistThresholdInBytes} then 
{@link
+ * #persistInternal(SequenceNumber) persist} is called.
+ *
+ * <p>On {@link #persist(SequenceNumber) persist}, accumulated changes are 
sent to the {@link
+ * StateChangeUploader} as an immutable {@link StateChangeUploader.UploadTask 
task}. An {@link
+ * FsStateChangelogWriter.UploadCompletionListener upload listener} is also 
registered. Upon
+ * notification it updates the Writer local state (for future persist calls) 
and completes the
+ * future returned to the original caller. The uploader notifies all listeners 
via a callback in a
+ * task.
+ *
+ * <p>If persist() is called for the same state change before the upload 
completion then the
+ * listener is added but not the upload task (which must already exist).
+ *
+ * <p>Invariants:
+ *
+ * <ol>
+ *   <li>Every change has at most one associated upload (retries are performed 
at a lower level)
+ *   <li>Every change is present in at most one collection: either uploaded OR 
notUploaded
+ *   <li>Changes BEING uploaded are NOT referenced locally - they will be 
added to uploaded upon
+ *       completion
+ *   <li>Failed and truncated changes are NOT stored - only their respective 
highest sequence
+ *       numbers
+ * </ol>
+ */
+@NotThreadSafe
+class FsStateChangelogWriter implements 
StateChangelogWriter<ChangelogStateHandleStreamImpl> {
+    private static final Logger LOG = 
LoggerFactory.getLogger(FsStateChangelogWriter.class);
+    private static final SequenceNumber INITIAL_SQN = SequenceNumber.of(0L);
+
+    private final UUID logId;
+    private final KeyGroupRange keyGroupRange;
+    private final StateChangeUploader uploader;
+    private final long preEmptivePersistThresholdInBytes;
+
+    /** Lock to synchronize handling of upload completion with new upload 
requests. */
+    // todo: replace with mailbox executor (after FLINK-23204)
+    private final Object lock = new Object();
+
+    /** A list of listener per upload (~ per checkpoint plus pre-emptive 
uploads). */
+    @GuardedBy("lock")
+    private final List<UploadCompletionListener> uploadCompletionListeners = 
new ArrayList<>();
+
+    private final FsStateChangelogCleaner cleaner;
+
+    /** Current {@link SequenceNumber}. */
+    private SequenceNumber activeSequenceNumber = INITIAL_SQN;
+
+    /**
+     * {@link SequenceNumber} before which changes will NOT be requested, 
exclusive. Increased after
+     * materialization.
+     */
+    @GuardedBy("lock")
+    private SequenceNumber lowestSequenceNumber = INITIAL_SQN;
+
+    /** Active changes, that will all use {@link #activeSequenceNumber}. */
+    private List<StateChange> activeChangeSet = new ArrayList<>();
+
+    /** {@link #activeChangeSet} size in bytes. */
+    private long activeChangeSetSize;
+
+    /** Changes that are not yet uploaded (upload not requested). */
+    private final NavigableMap<SequenceNumber, StateChangeSet> notUploaded = 
new TreeMap<>();
+
+    /** Uploaded changes, ready for use in snapshots. */
+    @GuardedBy("lock")
+    private final NavigableMap<SequenceNumber, UploadResult> uploaded = new 
TreeMap<>();

Review comment:
       I'm lacking somewhere an explanation what is the relationship between 
activeChangeSet and notUploaded?

##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeFsUploader.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.SnappyStreamCompressionDecorator;
+import org.apache.flink.runtime.state.StreamCompressionDecorator;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.UncompressedStreamCompressionDecorator;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+
+import org.apache.flink.shaded.guava18.com.google.common.io.Closer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import static java.util.Collections.singletonList;
+import static java.util.stream.Collectors.toList;
+import static org.apache.flink.core.fs.FileSystem.WriteMode.NO_OVERWRITE;
+
+class StateChangeFsUploader implements StateChangeUploader {

Review comment:
       This is a synchronous implementation the `StateChangeUploader`, that is 
doing the IO operations in the `#upload()` call? If so, could you add this 
explanation in the java doc?

##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.changelog.fs.StateChangeUploader.UploadTask;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.ChangelogStateHandleStreamImpl;
+import org.apache.flink.runtime.state.changelog.SequenceNumber;
+import org.apache.flink.runtime.state.changelog.SequenceNumberRange;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.changelog.StateChangelogWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Filesystem-based {@link StateChangelogWriter} implementation. Assumes 
TM-owned state - so no
+ * re-uploads.
+ *
+ * <p>On {@link #append(int, byte[]) append}, it stores the changes locally in 
memory (without any
+ * thread synchronization); {@link SequenceNumber} is not changed.
+ *
+ * <p>However, if they exceed {@link #preEmptivePersistThresholdInBytes} then 
{@link
+ * #persistInternal(SequenceNumber) persist} is called.
+ *
+ * <p>On {@link #persist(SequenceNumber) persist}, accumulated changes are 
sent to the {@link
+ * StateChangeUploader} as an immutable {@link StateChangeUploader.UploadTask 
task}. An {@link
+ * FsStateChangelogWriter.UploadCompletionListener upload listener} is also 
registered. Upon
+ * notification it updates the Writer local state (for future persist calls) 
and completes the
+ * future returned to the original caller. The uploader notifies all listeners 
via a callback in a
+ * task.
+ *
+ * <p>If persist() is called for the same state change before the upload 
completion then the
+ * listener is added but not the upload task (which must already exist).
+ *
+ * <p>Invariants:
+ *
+ * <ol>
+ *   <li>Every change has at most one associated upload (retries are performed 
at a lower level)
+ *   <li>Every change is present in at most one collection: either uploaded OR 
notUploaded
+ *   <li>Changes BEING uploaded are NOT referenced locally - they will be 
added to uploaded upon
+ *       completion
+ *   <li>Failed and truncated changes are NOT stored - only their respective 
highest sequence
+ *       numbers
+ * </ol>
+ */
+@NotThreadSafe
+class FsStateChangelogWriter implements 
StateChangelogWriter<ChangelogStateHandleStreamImpl> {
+    private static final Logger LOG = 
LoggerFactory.getLogger(FsStateChangelogWriter.class);
+    private static final SequenceNumber INITIAL_SQN = SequenceNumber.of(0L);
+
+    private final UUID logId;
+    private final KeyGroupRange keyGroupRange;
+    private final StateChangeUploader uploader;
+    private final long preEmptivePersistThresholdInBytes;
+
+    /** Lock to synchronize handling of upload completion with new upload 
requests. */
+    // todo: replace with mailbox executor (after FLINK-23204)
+    private final Object lock = new Object();
+
+    /** A list of listener per upload (~ per checkpoint plus pre-emptive 
uploads). */
+    @GuardedBy("lock")
+    private final List<UploadCompletionListener> uploadCompletionListeners = 
new ArrayList<>();
+
+    private final FsStateChangelogCleaner cleaner;
+
+    /** Current {@link SequenceNumber}. */
+    private SequenceNumber activeSequenceNumber = INITIAL_SQN;
+
+    /**
+     * {@link SequenceNumber} before which changes will NOT be requested, 
exclusive. Increased after
+     * materialization.
+     */
+    @GuardedBy("lock")
+    private SequenceNumber lowestSequenceNumber = INITIAL_SQN;
+
+    /** Active changes, that will all use {@link #activeSequenceNumber}. */
+    private List<StateChange> activeChangeSet = new ArrayList<>();
+
+    /** {@link #activeChangeSet} size in bytes. */
+    private long activeChangeSetSize;
+
+    /** Changes that are not yet uploaded (upload not requested). */
+    private final NavigableMap<SequenceNumber, StateChangeSet> notUploaded = 
new TreeMap<>();
+
+    /** Uploaded changes, ready for use in snapshots. */
+    @GuardedBy("lock")
+    private final NavigableMap<SequenceNumber, UploadResult> uploaded = new 
TreeMap<>();
+
+    /**
+     * Highest {@link SequenceNumber} for which upload has failed (won't be 
restarted), inclusive.
+     */
+    @Nullable
+    @GuardedBy("lock")
+    private Tuple2<SequenceNumber, Throwable> highestFailed;
+
+    @GuardedBy("lock")
+    private boolean closed;
+
+    FsStateChangelogWriter(
+            UUID logId,
+            KeyGroupRange keyGroupRange,
+            StateChangeUploader uploader,
+            long preEmptivePersistThresholdInBytes,
+            FsStateChangelogCleaner cleaner) {
+        this.logId = logId;
+        this.keyGroupRange = keyGroupRange;
+        this.uploader = uploader;
+        this.preEmptivePersistThresholdInBytes = 
preEmptivePersistThresholdInBytes;
+        this.cleaner = cleaner;
+    }
+
+    @Override
+    public void append(int keyGroup, byte[] value) throws IOException {
+        LOG.trace("append to {}: keyGroup={} {} bytes", logId, keyGroup, 
value.length);
+        checkState(!closed, "%s is closed", logId);
+        activeChangeSet.add(new StateChange(keyGroup, value));
+        activeChangeSetSize += value.length;
+        if (activeChangeSetSize >= preEmptivePersistThresholdInBytes) {
+            LOG.debug(
+                    "pre-emptively flush {}Mb of appended changes to the 
common store",
+                    activeChangeSetSize / 1024 / 1024);
+            persistInternal(notUploaded.isEmpty() ? activeSequenceNumber : 
notUploaded.firstKey());
+        }
+    }
+
+    @Override
+    public SequenceNumber initialSequenceNumber() {
+        return INITIAL_SQN;
+    }
+
+    @Override
+    public SequenceNumber lastAppendedSequenceNumber() {
+        LOG.trace("query {} sqn: {}", logId, activeSequenceNumber);
+        SequenceNumber tmp = activeSequenceNumber;
+        rollover();
+        return tmp;
+    }
+
+    @Override
+    public CompletableFuture<ChangelogStateHandleStreamImpl> 
persist(SequenceNumber from)
+            throws IOException {
+        LOG.debug(
+                "persist {} starting from sqn {} (incl.), active sqn: {}",
+                logId,
+                from,
+                activeSequenceNumber);
+        return persistInternal(from);
+    }
+
+    private CompletableFuture<ChangelogStateHandleStreamImpl> 
persistInternal(SequenceNumber from)
+            throws IOException {
+        synchronized (lock) {
+            ensureCanPersist(from);
+            rollover();
+            Map<SequenceNumber, StateChangeSet> tailMap = 
notUploaded.tailMap(from, true);
+            Map<SequenceNumber, StateChangeSet> toUpload = new 
HashMap<>(tailMap);
+            tailMap.clear(); // prevent re-uploads

Review comment:
       nit: `tailMap` variable is a bit confusing given that you are calling 
`tailMap` method twice here. Is the the intention to get the `tailMap(from, 
true)` from the `notUploaded` AND remove those elements from the `notUploaded` 
at the same time? And the issue is that there is no method like 
`NavigableMap#tailMapAndRemove`? If so, maybe a `private static` helper method 
here in this class to do just that?




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