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



##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.StateChangeStore.StoreTask;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.SequenceNumber;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.changelog.StateChangelogHandleStreamImpl;
+import org.apache.flink.runtime.state.changelog.StateChangelogWriter;
+import 
org.apache.flink.runtime.state.changelog.StateChangelogWriterFactory.ChangelogCallbackExecutor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Basic implementation of {@link StateChangelogWriter}. It's main purpose is 
to keep track which
+ * changes are required for the given {@link #persist(SequenceNumber)} call. 
Especially it takes
+ * care of re-uploading changes from the previous {@link 
#persist(SequenceNumber)} call, if those
+ * changes haven't been yet {@link #confirm(SequenceNumber, 
SequenceNumber)}'ed. This is crucial as
+ * until changes are {@link #confirm(SequenceNumber, SequenceNumber)}, they 
still can be aborted and
+ * removed/deleted.
+ *
+ * <p>For example if checkpoint N-1 fails and is disposed, after checkpoint N 
has already started.
+ * In this case, when we are persisting {@link StateChangeSet}s for checkpoint 
N, we need to
+ * re-upload {@link StateChangeSet}s that belonged to checkpoint N-1.
+ */
+@NotThreadSafe
+class FsStateChangelogWriter implements 
StateChangelogWriter<StateChangelogHandleStreamImpl> {
+    private static final Logger LOG = 
LoggerFactory.getLogger(FsStateChangelogWriter.class);
+
+    private final UUID logId;
+    private final KeyGroupRange keyGroupRange;
+    private final StateChangeStore store;
+    private final NavigableMap<SequenceNumber, StateChangeSet> changeSets = 
new TreeMap<>();
+    private final NavigableMap<SequenceNumber, StoreResult> uploaded = new 
TreeMap<>();
+    private final NavigableMap<SequenceNumber, StoreResult> confirmed = new 
TreeMap<>();
+    private List<StateChange> activeChangeSet = new ArrayList<>();
+    private SequenceNumber lastAppendedSequenceNumber = SequenceNumber.of(0L);
+    private boolean closed;
+    private final ChangelogCallbackExecutor executor;
+
+    FsStateChangelogWriter(
+            UUID logId,
+            KeyGroupRange keyGroupRange,
+            StateChangeStore store,
+            ChangelogCallbackExecutor executor) {
+        this.logId = logId;
+        this.keyGroupRange = keyGroupRange;
+        this.store = store;
+        this.executor = checkNotNull(executor);
+    }
+
+    @Override
+    public void append(int keyGroup, byte[] value) {
+        LOG.trace("append to {}: keyGroup={} {} bytes", logId, keyGroup, 
value.length);
+        checkState(!closed, "%s is closed", logId);
+        activeChangeSet.add(new StateChange(keyGroup, value));
+        // size threshold could be added to call persist when reached. 
considerations:
+        // 0. can actually degrade performance by amplifying number of requests
+        // 1. which range to persist?
+        // 2. how to deal with retries/aborts?
+    }
+
+    @Override
+    public SequenceNumber lastAppendedSequenceNumber() {
+        rollover();
+        LOG.trace("query {} sqn: {}", logId, lastAppendedSequenceNumber);
+        return lastAppendedSequenceNumber;
+    }
+
+    @Override
+    public CompletableFuture<StateChangelogHandleStreamImpl> 
persist(SequenceNumber from)
+            throws IOException {
+        LOG.debug("persist {} from {}", logId, from);
+        checkNotNull(from);
+        // todo: check range
+
+        rollover();
+        Collection<StoreResult> readyToReturn = confirmed.tailMap(from, 
true).values();
+        Collection<StateChangeSet> toUpload = changeSets.tailMap(from, 
true).values();

Review comment:
       So for pre-emptive uploads there will be a separate collection which 
will also be updated by a separate callback. We'll also have to truncate it. 
   Sounds a bit complex but I think it should work.




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

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


Reply via email to