rkhachatryan commented on a change in pull request #15322: URL: https://github.com/apache/flink/pull/15322#discussion_r662574161
########## File path: flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogWriter.java ########## @@ -0,0 +1,207 @@ +/* + * 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.apache.flink.util.Preconditions; + +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; + +@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 = Preconditions.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(); + LOG.debug("collected readyToReturn: {}, toUpload: {}", readyToReturn, toUpload); + CompletableFuture<StateChangelogHandleStreamImpl> future = new CompletableFuture<>(); + store.save(createTask(readyToReturn, toUpload, future)); + return future; + } + + private StoreTask createTask( + Collection<StoreResult> readyToReturn, + Collection<StateChangeSet> toUpload, + CompletableFuture<StateChangelogHandleStreamImpl> future) { + return new StoreTask( + toUpload, + future::completeExceptionally, + results -> + executor.execute( + () -> { + results.forEach(e -> uploaded.put(e.sequenceNumber, e)); + future.complete(buildHandle(results, readyToReturn)); + })); + } + + @Override + public void close() { + LOG.debug("close {}", logId); + checkState(!closed); + closed = true; + activeChangeSet.clear(); + // todo in MVP or later: cleanup if transition succeeded and had non-shared state + changeSets.clear(); + uploaded.clear(); + confirmed.clear(); + // the store is closed from the owning FsStateChangelogClient + } + + @Override + public void confirm(SequenceNumber from, SequenceNumber to) { + LOG.debug("confirm {} from {} to {}", logId, from, to); + uploaded.subMap(from, true, to, false) + .forEach( + (sequenceNumber, changeSetAndResult) -> { + changeSets.remove(sequenceNumber); + uploaded.remove(sequenceNumber); + confirmed.put(sequenceNumber, changeSetAndResult); + }); + } + + @Override + public void reset(SequenceNumber from, SequenceNumber to) { + LOG.debug("reset {} from {} to {}", logId, from, to); Review comment: I guess that's not relevant after adopting TM-side state ownership (`reset` is now no-op). -- 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]
