pnowojski commented on a change in pull request #15322:
URL: https://github.com/apache/flink/pull/15322#discussion_r669637796
##########
File path:
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogOptions.java
##########
@@ -57,4 +87,18 @@
"Number of threads to use to perform cleanup in
case an upload is discarded "
+ "(and not cleaned up by JM). "
+ "If the cleanup doesn't keep up then
task might be back-pressured.");
+
+ public static final ConfigOption<Integer> NUM_UPLOAD_THREADS =
+ ConfigOptions.key("dstl.dfs.upload.num-threads")
+ .intType()
+ .defaultValue(5)
+ .withDescription("Number of threads to use for upload.");
Review comment:
I have the same concern as with previous cleaner thread pool. Why do we
want to have a separate pool for changelog uploads and async checkpoint part
uploads?
##########
File path:
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/RetryingExecutor.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.util.function.RunnableWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+/**
+ * A {@link RunnableWithException} executor that schedules a next attempt upon
timeout based on
+ * {@link RetryPolicy}. Aimed to curb tail latencies
+ */
+class RetryingExecutor implements AutoCloseable {
+ private static final Logger LOG =
LoggerFactory.getLogger(RetryingExecutor.class);
+
+ private final ScheduledExecutorService scheduler;
+
+ RetryingExecutor(int nThreads) {
+ this(SchedulerFactory.create(nThreads, "ChangelogRetryScheduler",
LOG));
+ }
+
+ RetryingExecutor(ScheduledExecutorService scheduler) {
+ this.scheduler = scheduler;
+ }
+
+ void execute(RetryPolicy retryPolicy, RunnableWithException action) {
+ LOG.debug("execute with retryPolicy: {}", retryPolicy);
+ RetriableTask task = new RetriableTask(action, retryPolicy, scheduler);
+ scheduler.submit(task);
+ }
+
+ @Override
+ public void close() throws Exception {
+ LOG.debug("close");
+ scheduler.shutdownNow();
+ if (!scheduler.awaitTermination(1, TimeUnit.SECONDS)) {
+ LOG.warn("Unable to cleanly shutdown executorService in 1s");
+ }
+ }
+
+ private static final class RetriableTask implements Runnable {
+ private final RunnableWithException runnable;
+ private final ScheduledExecutorService executorService;
+ private final int current;
+ private final RetryPolicy retryPolicy;
+ private final AtomicBoolean actionCompleted;
+ private final AtomicBoolean attemptCompleted = new
AtomicBoolean(false);
+
+ RetriableTask(
+ RunnableWithException runnable,
+ RetryPolicy retryPolicy,
+ ScheduledExecutorService executorService) {
+ this(1, new AtomicBoolean(false), runnable, retryPolicy,
executorService);
+ }
+
+ private RetriableTask(
+ int current,
+ AtomicBoolean actionCompleted,
+ RunnableWithException runnable,
+ RetryPolicy retryPolicy,
+ ScheduledExecutorService executorService) {
+ this.current = current;
+ this.runnable = runnable;
+ this.retryPolicy = retryPolicy;
+ this.executorService = executorService;
+ this.actionCompleted = actionCompleted;
+ }
+
+ @Override
+ public void run() {
+ if (!actionCompleted.get()) {
+ Optional<ScheduledFuture<?>> timeoutFuture = scheduleTimeout();
+ try {
+ runnable.run();
+ actionCompleted.set(true);
+ attemptCompleted.set(true);
+ } catch (Exception e) {
+ handleError(e);
+ } finally {
+ timeoutFuture.ifPresent(f -> f.cancel(true));
+ }
+ }
+ }
+
+ private void handleError(Exception e) {
+ LOG.trace("execution attempt {} failed: {}", current,
e.getMessage());
Review comment:
debug? info? But I would say almost for certain not just trace?
##########
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:
Still I would explain this in the java doc as it's customary/expected in
rest of the code base with at least a single sentence.
##########
File path:
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/RetryingExecutor.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.util.function.RunnableWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+/**
+ * A {@link RunnableWithException} executor that schedules a next attempt upon
timeout based on
+ * {@link RetryPolicy}. Aimed to curb tail latencies
+ */
+class RetryingExecutor implements AutoCloseable {
+ private static final Logger LOG =
LoggerFactory.getLogger(RetryingExecutor.class);
+
+ private final ScheduledExecutorService scheduler;
+
+ RetryingExecutor(int nThreads) {
+ this(SchedulerFactory.create(nThreads, "ChangelogRetryScheduler",
LOG));
+ }
+
+ RetryingExecutor(ScheduledExecutorService scheduler) {
+ this.scheduler = scheduler;
+ }
+
+ void execute(RetryPolicy retryPolicy, RunnableWithException action) {
+ LOG.debug("execute with retryPolicy: {}", retryPolicy);
+ RetriableTask task = new RetriableTask(action, retryPolicy, scheduler);
+ scheduler.submit(task);
+ }
+
+ @Override
+ public void close() throws Exception {
+ LOG.debug("close");
+ scheduler.shutdownNow();
+ if (!scheduler.awaitTermination(1, TimeUnit.SECONDS)) {
+ LOG.warn("Unable to cleanly shutdown executorService in 1s");
+ }
+ }
+
+ private static final class RetriableTask implements Runnable {
+ private final RunnableWithException runnable;
+ private final ScheduledExecutorService executorService;
+ private final int current;
+ private final RetryPolicy retryPolicy;
+ private final AtomicBoolean actionCompleted;
+ private final AtomicBoolean attemptCompleted = new
AtomicBoolean(false);
Review comment:
replace with a single lock? As it is it's not obvious what kind of race
conditions are possible and how are you dealing with them.
##########
File path:
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/RetryingExecutor.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.util.function.RunnableWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+/**
+ * A {@link RunnableWithException} executor that schedules a next attempt upon
timeout based on
+ * {@link RetryPolicy}. Aimed to curb tail latencies
+ */
+class RetryingExecutor implements AutoCloseable {
+ private static final Logger LOG =
LoggerFactory.getLogger(RetryingExecutor.class);
+
+ private final ScheduledExecutorService scheduler;
+
+ RetryingExecutor(int nThreads) {
+ this(SchedulerFactory.create(nThreads, "ChangelogRetryScheduler",
LOG));
+ }
+
+ RetryingExecutor(ScheduledExecutorService scheduler) {
+ this.scheduler = scheduler;
+ }
+
+ void execute(RetryPolicy retryPolicy, RunnableWithException action) {
+ LOG.debug("execute with retryPolicy: {}", retryPolicy);
+ RetriableTask task = new RetriableTask(action, retryPolicy, scheduler);
+ scheduler.submit(task);
+ }
+
+ @Override
+ public void close() throws Exception {
+ LOG.debug("close");
+ scheduler.shutdownNow();
+ if (!scheduler.awaitTermination(1, TimeUnit.SECONDS)) {
+ LOG.warn("Unable to cleanly shutdown executorService in 1s");
+ }
+ }
+
+ private static final class RetriableTask implements Runnable {
+ private final RunnableWithException runnable;
+ private final ScheduledExecutorService executorService;
+ private final int current;
+ private final RetryPolicy retryPolicy;
+ private final AtomicBoolean actionCompleted;
+ private final AtomicBoolean attemptCompleted = new
AtomicBoolean(false);
+
+ RetriableTask(
+ RunnableWithException runnable,
+ RetryPolicy retryPolicy,
+ ScheduledExecutorService executorService) {
+ this(1, new AtomicBoolean(false), runnable, retryPolicy,
executorService);
+ }
+
+ private RetriableTask(
+ int current,
+ AtomicBoolean actionCompleted,
+ RunnableWithException runnable,
+ RetryPolicy retryPolicy,
+ ScheduledExecutorService executorService) {
+ this.current = current;
+ this.runnable = runnable;
+ this.retryPolicy = retryPolicy;
+ this.executorService = executorService;
+ this.actionCompleted = actionCompleted;
+ }
+
+ @Override
+ public void run() {
+ if (!actionCompleted.get()) {
+ Optional<ScheduledFuture<?>> timeoutFuture = scheduleTimeout();
+ try {
+ runnable.run();
Review comment:
How are you solving a (race?) condition when two attempts complete? Do
you have a requirement that `RunnableWithException runnable` must be idempotent
and you just ignore that issue?
##########
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 think with the code as it is, it should be documented what
`activeChangeSet` is doing.
However maybe we can rename `activeChangeSet`? `pendingBatch`?
`activePendingBatch`? `pendingChangeSetBatch`? Or maybe both:
1. `activePendingBatch`
2. plus comment "Changes are accumulated in the activePendingBatch until
they are scheduled to be uploaded"?
##########
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:
nit: could you also add:
> At the same time, we don't want to increment SQN on each append (to avoid
too many objects and segments in the resulting file).
to the comment as well?
--
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]