pnowojski commented on a change in pull request #15371: URL: https://github.com/apache/flink/pull/15371#discussion_r603113393
########## 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(); + LOG.debug("collected readyToReturn: {}, toUpload: {}", readyToReturn, toUpload); + + StoreTask task = new StoreTask(toUpload); + // Handle the result result in supplied executor (mailbox) thread + CompletableFuture<StateChangelogHandleStreamImpl> resultFuture = + task.getResultFuture() + .thenApplyAsync( + (results) -> { + results.forEach(e -> uploaded.put(e.sequenceNumber, e)); Review comment: I think you are right. This mailbox WIP had simplify too much the re-uploading tracking logic. On the other hand, maybe fixing it now would be a wasted effort, as we are not sure how final re-uploading logic will be handled as it even might be completely removed in the final version? Maybe let's say `notifyCheckpointAbort` is not supported (and throw an exception) until we figure out a solution for the JM/TM changelog ownership question? ########## File path: flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/BatchingStateChangeStore.java ########## @@ -0,0 +1,173 @@ +/* + * 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.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.CompletableFuture; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; + +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 StateChangeStore} that waits for some configured amount of time before passing the + * accumulated state changes to the actual store. The writes are executed asynchronously. + */ +@ThreadSafe +class BatchingStateChangeStore implements StateChangeStore { + private static final Logger LOG = LoggerFactory.getLogger(BatchingStateChangeStore.class); + + private final StateChangeStore delegate; + private final ScheduledExecutorService scheduler; + private final long scheduleDelayMs; + private final long sizeThresholdBytes; + + @GuardedBy("scheduled") + private final Queue<StoreTask> scheduled; + + @GuardedBy("scheduled") + private long scheduledSizeInBytes; + + @GuardedBy("scheduled") + private Future<?> scheduledFuture = CompletableFuture.completedFuture(null); + + private AtomicReference<Throwable> error = new AtomicReference<>(null); + + BatchingStateChangeStore( + long persistDelayMs, long sizeThresholdBytes, StateChangeStore delegate) { + this( + persistDelayMs, + sizeThresholdBytes, + delegate, + SchedulerFactory.create(1, "ChangelogRetryScheduler", LOG)); + } + + BatchingStateChangeStore( + long persistDelayMs, + long sizeThresholdBytes, + StateChangeStore delegate, + ScheduledExecutorService scheduler) { + this.scheduleDelayMs = persistDelayMs; + this.scheduled = new LinkedList<>(); + this.scheduler = scheduler; + this.sizeThresholdBytes = sizeThresholdBytes; + this.delegate = delegate; + } + + @Override + public void save(StoreTask storeTask) { + Collection<StateChangeSet> changeSets = storeTask.changeSets; + if (error.get() != null) { + LOG.debug("don't persist {} changesets, already failed", changeSets.size()); + storeTask.fail(error.get()); + return; + } + LOG.debug("persist {} changeSets", changeSets.size()); + try { + synchronized (scheduled) { + scheduled.add(storeTask); + scheduledSizeInBytes += storeTask.getSize(); + scheduleUploadIfNeeded(); + } + } catch (Exception e) { + storeTask.fail(e); + throw e; + } + } + + private void scheduleUploadIfNeeded() { + checkState(holdsLock(scheduled)); + if (scheduledFuture.isDone() || isOverSizeThresholdAndCancellationSucceded()) { + scheduleUpload(); + } + } + + private boolean isOverSizeThresholdAndCancellationSucceded() { + return scheduledSizeInBytes >= sizeThresholdBytes && scheduledFuture.cancel(false); + } Review comment: My intention was that if cancellation doesn't succeed, it means it's already running, and `drainAndSave` will take care of rescheduling next upload. Isn't this happening? ########## File path: flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/BatchingStateChangeStore.java ########## @@ -0,0 +1,173 @@ +/* + * 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.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.CompletableFuture; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; + +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 StateChangeStore} that waits for some configured amount of time before passing the + * accumulated state changes to the actual store. The writes are executed asynchronously. + */ +@ThreadSafe +class BatchingStateChangeStore implements StateChangeStore { + private static final Logger LOG = LoggerFactory.getLogger(BatchingStateChangeStore.class); + + private final StateChangeStore delegate; + private final ScheduledExecutorService scheduler; + private final long scheduleDelayMs; + private final long sizeThresholdBytes; + + @GuardedBy("scheduled") + private final Queue<StoreTask> scheduled; + + @GuardedBy("scheduled") + private long scheduledSizeInBytes; + + @GuardedBy("scheduled") + private Future<?> scheduledFuture = CompletableFuture.completedFuture(null); + + private AtomicReference<Throwable> error = new AtomicReference<>(null); + + BatchingStateChangeStore( + long persistDelayMs, long sizeThresholdBytes, StateChangeStore delegate) { + this( + persistDelayMs, + sizeThresholdBytes, + delegate, + SchedulerFactory.create(1, "ChangelogRetryScheduler", LOG)); + } + + BatchingStateChangeStore( + long persistDelayMs, + long sizeThresholdBytes, + StateChangeStore delegate, + ScheduledExecutorService scheduler) { + this.scheduleDelayMs = persistDelayMs; + this.scheduled = new LinkedList<>(); + this.scheduler = scheduler; + this.sizeThresholdBytes = sizeThresholdBytes; + this.delegate = delegate; + } + + @Override + public void save(StoreTask storeTask) { + Collection<StateChangeSet> changeSets = storeTask.changeSets; + if (error.get() != null) { + LOG.debug("don't persist {} changesets, already failed", changeSets.size()); + storeTask.fail(error.get()); + return; + } + LOG.debug("persist {} changeSets", changeSets.size()); + try { + synchronized (scheduled) { + scheduled.add(storeTask); + scheduledSizeInBytes += storeTask.getSize(); + scheduleUploadIfNeeded(); + } + } catch (Exception e) { + storeTask.fail(e); + throw e; + } + } + + private void scheduleUploadIfNeeded() { + checkState(holdsLock(scheduled)); + if (scheduledFuture.isDone() || isOverSizeThresholdAndCancellationSucceded()) { + scheduleUpload(); + } + } + + private boolean isOverSizeThresholdAndCancellationSucceded() { + return scheduledSizeInBytes >= sizeThresholdBytes && scheduledFuture.cancel(false); + } + + private void scheduleUpload() { + checkState(scheduledFuture.isDone()); + long delay = scheduleDelayMs; + if (scheduleDelayMs == 0 || scheduledSizeInBytes >= sizeThresholdBytes) { + scheduledFuture = scheduler.submit(this::drainAndSave); + } else { + scheduledFuture = scheduler.schedule(this::drainAndSave, delay, MILLISECONDS); + } + } + + private void drainAndSave() { + Collection<StoreTask> tasks; + synchronized (scheduled) { + tasks = new ArrayList<>(scheduled); + scheduled.clear(); + scheduledSizeInBytes = 0; + } + try { + if (error.get() != null) { + tasks.forEach(task -> task.fail(error.get())); + return; + } + delegate.save(tasks); Review comment: Yes we do, I wanted to simplify the first version for reviewing and later add a bit more sophisticated logic for handling concurrent writes. At the moment I'm not sure how multi threaded writes should actually be working in the first place (each one writing to an independent file? I'm a bit worried about this approach from the perspective of the recovery). ########## File path: flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/BatchingStateChangeStore.java ########## @@ -136,21 +133,23 @@ private void drainAndSave() { tasks = new ArrayList<>(scheduled); scheduled.clear(); scheduledSizeInBytes = 0; - scheduledFuture = null; } try { - if (error != null) { - tasks.forEach(task -> task.fail(error)); + if (error.get() != null) { + tasks.forEach(task -> task.fail(error.get())); return; } - retryingExecutor.execute(retryPolicy, () -> delegate.save(tasks)); + delegate.save(tasks); + + synchronized (scheduled) { + scheduleUploadIfNeeded(); + } Review comment: I think you are right, this should be moved to finally block? ########## File path: flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/BatchingStateChangeStore.java ########## @@ -136,21 +133,23 @@ private void drainAndSave() { tasks = new ArrayList<>(scheduled); scheduled.clear(); scheduledSizeInBytes = 0; - scheduledFuture = null; } try { - if (error != null) { - tasks.forEach(task -> task.fail(error)); + if (error.get() != null) { + tasks.forEach(task -> task.fail(error.get())); return; } - retryingExecutor.execute(retryPolicy, () -> delegate.save(tasks)); + delegate.save(tasks); + + synchronized (scheduled) { + scheduleUploadIfNeeded(); + } } catch (Throwable t) { tasks.forEach(task -> task.fail(t)); if (findThrowable(t, IOException.class).isPresent()) { LOG.warn("Caught IO exception while uploading", t); } else { - error = t; - throw t; + error.compareAndSet(null, t); Review comment: This is a top level method from the uploader/executor thread pool and the only meaningful way of returning the error to the task thread is via `error`? If we throw here, it would have no effect, or am I missing something? ########## 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: First and foremost, I'm not sure how much longer we will have this re-uploading logic and if we will, in what form? But assuming no changes to the ownership of the changelog, why couldn't we just move/call the code from the `persist()` to `preEmptiveUpload()`? One issue would be that we would have to store the result of `preEmptiveUpload()` as `FsStateChangelogWriter` state instead of just returning it, so that we can combine the final `StateChangelogHandleStreamImpl` in the `persist()` call, but isn't that all? -- 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]
