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



##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/BatchingStateChangeUploader.java
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+
+import static java.lang.Thread.holdsLock;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.flink.util.ExceptionUtils.findThrowable;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link StateChangeUploader} that waits for some configured amount of time 
before passing the
+ * accumulated state changes to the actual store.
+ */
+@ThreadSafe
+class BatchingStateChangeUploader implements StateChangeUploader {
+    private static final Logger LOG = 
LoggerFactory.getLogger(BatchingStateChangeUploader.class);
+
+    private final RetryingExecutor retryingExecutor;
+    private final RetryPolicy retryPolicy;
+    private final StateChangeUploader delegate;
+    private final ScheduledExecutorService scheduler;
+    private final long scheduleDelayMs;
+    private final long sizeThresholdBytes;
+
+    @GuardedBy("scheduled")
+    private final Queue<UploadTask> scheduled;
+
+    @GuardedBy("scheduled")
+    private long scheduledSizeInBytes;
+
+    @Nullable
+    @GuardedBy("scheduled")
+    private ScheduledFuture<?> scheduledFuture;
+
+    private volatile Throwable error;

Review comment:
       Good point, I'll remove `volatile` and move accesses under 
`synchronized`.

##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogCleanerImpl.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+class FsStateChangelogCleanerImpl implements FsStateChangelogCleaner {
+    private static final Logger LOG = 
LoggerFactory.getLogger(FsStateChangelogCleanerImpl.class);
+    private static final int MAX_TASKS_PER_THREAD = 100;
+
+    private final Executor executor;
+
+    public FsStateChangelogCleanerImpl(int nThreads) {
+        // Use a fixed-size thread pool with a bounded queue so that 
cleanupAsync back pressures
+        // callers if the cleanup doesn't keep up.
+        // In all cases except abort this are uploader threads; while on abort 
this is the task
+        // thread.
+        this(
+                new ThreadPoolExecutor(
+                        nThreads,
+                        nThreads,
+                        0L,
+                        TimeUnit.MILLISECONDS,
+                        new LinkedBlockingQueue<>(nThreads * 
MAX_TASKS_PER_THREAD),
+                        (ThreadFactory) Thread::new));

Review comment:
       Could you explain why not?
   
   I see the following reasons to use a separate pool:
   1. (as mentioned in the comment): use a blocking queue to provide 
back-pressure
   2. It's easier to configure size with separate pools
   3. Easier to debug
   
   I'm assuming the overhead of having more threads per task bounded by 
`nThreads` negligible on modern OSes.

##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogStorage.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.Experimental;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.changelog.ChangelogStateHandleStreamImpl;
+import org.apache.flink.runtime.state.changelog.StateChangelogHandleReader;
+import 
org.apache.flink.runtime.state.changelog.StateChangelogHandleStreamHandleReader;
+import org.apache.flink.runtime.state.changelog.StateChangelogStorage;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static 
org.apache.flink.changelog.fs.FsStateChangelogOptions.PREEMPTIVE_PERSIST_THRESHOLD;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** Filesystem-based implementation of {@link StateChangelogStorage}. */
+@Experimental
+public class FsStateChangelogStorage
+        implements StateChangelogStorage<ChangelogStateHandleStreamImpl>, 
Serializable {
+    private static final Logger LOG = 
LoggerFactory.getLogger(FsStateChangelogStorage.class);
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * The log id is only needed on write to separate changes from different 
backends (i.e.
+     * operators) in the resulting file.
+     */
+    private transient AtomicInteger logIdGenerator = new AtomicInteger(0);
+
+    private transient volatile StateChangeUploader uploader;
+    private transient volatile FsStateChangelogCleaner cleaner;
+    private volatile ReadableConfig config; // todo: make final after 
FLINK-21804

Review comment:
       All this was supposed to go away after FLINK-21804 (as I mentioned in 
the comment - but I should definitely have elaborate more).
   FLINK-21804 is now merged, so I'll rework this part after rebasing the PR.

##########
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
+            NavigableMap<SequenceNumber, UploadResult> readyToReturn = 
uploaded.tailMap(from, true);
+            LOG.debug("collected readyToReturn: {}, toUpload: {}", 
readyToReturn, toUpload);
+
+            SequenceNumberRange range = SequenceNumberRange.generic(from, 
activeSequenceNumber);
+            if (range.size() == readyToReturn.size()) {
+                checkState(toUpload.isEmpty());
+                return completedFuture(buildHandle(keyGroupRange, 
readyToReturn));
+            } else {
+                CompletableFuture<ChangelogStateHandleStreamImpl> future =
+                        new CompletableFuture<>();
+                uploadCompletionListeners.add(
+                        new UploadCompletionListener(keyGroupRange, range, 
readyToReturn, future));
+                if (!toUpload.isEmpty()) {
+                    uploader.upload(
+                            new UploadTask(
+                                    toUpload.values(),
+                                    this::handleUploadSuccess,
+                                    this::handleUploadFailure));

Review comment:
       I think it's doable and I left a comment about it:
   ```
       /** 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();
   ```
   But as FLINK-23204 is currently not implemented (and there is no consensus 
on how to do that), I'd like not to add such a dependency.




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to