rkhachatryan commented on a change in pull request #16606: URL: https://github.com/apache/flink/pull/16606#discussion_r691590608
########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/materializer/PeriodicMaterializer.java ########## @@ -0,0 +1,263 @@ +/* + * 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.state.changelog.materializer; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.core.fs.FileSystemSafetyNet; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.CheckpointType; +import org.apache.flink.runtime.mailbox.MailboxExecutor; +import org.apache.flink.runtime.state.CheckpointStorageLocationReference; +import org.apache.flink.runtime.state.CheckpointStorageWorkerView; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.Snapshotable; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.runtime.state.changelog.ChangelogStateHandle; +import org.apache.flink.runtime.state.changelog.SequenceNumber; +import org.apache.flink.runtime.state.changelog.StateChangelogWriter; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.RunnableFuture; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonList; +import static org.apache.flink.util.Preconditions.checkState; + +/** Periodically make materialization for the delegated state backend. */ +public class PeriodicMaterializer { + private static final Logger LOG = LoggerFactory.getLogger(PeriodicMaterializer.class); + + /** + * ChangelogStateBackend only supports CheckpointType.CHECKPOINT; The rest of information in + * CheckpointOptions is not used in Snapshotable#snapshot(). More details in FLINK-23441. + */ + private static final CheckpointOptions CHECKPOINT_OPTIONS = + new CheckpointOptions( + CheckpointType.CHECKPOINT, CheckpointStorageLocationReference.getDefault()); + + /** task mailbox executor, execute from Task Thread. */ + private final MailboxExecutor mailboxExecutor; + + /** Async thread pool, to complete async phase of materialization. */ + private final ExecutorService asyncOperationsThreadPool; + + /** scheduled executor, periodically trigger materialization. */ + private final ScheduledExecutorService periodicExecutor; + + private final CheckpointStreamFactory streamFactory; + + private final Snapshotable keyedStateBackend; + + private final StateChangelogWriter<ChangelogStateHandle> stateChangelogWriter; + + private final AsyncExceptionHandler asyncExceptionHandler; + + private final int allowedNumberOfFailures; + + /** Materialization failure retries. */ + private final AtomicInteger retries; + + /** Making sure only one materialization on going at a time. */ + private final AtomicBoolean materializationOnGoing; + + private long materializedId; + + private MaterializedState materializedState; + + public PeriodicMaterializer( + MailboxExecutor mailboxExecutor, + ExecutorService asyncOperationsThreadPool, + Snapshotable keyedStateBackend, + CheckpointStorageWorkerView checkpointStorageWorkerView, + StateChangelogWriter<ChangelogStateHandle> stateChangelogWriter, + AsyncExceptionHandler asyncExceptionHandler, + MaterializedState materializedState, + long periodicMaterializeInitDelay, + long periodicMaterializeInterval, + int allowedNumberOfFailures, + boolean materializationEnabled) { + this.mailboxExecutor = mailboxExecutor; + this.asyncOperationsThreadPool = asyncOperationsThreadPool; + this.keyedStateBackend = keyedStateBackend; + this.stateChangelogWriter = stateChangelogWriter; + this.asyncExceptionHandler = asyncExceptionHandler; + this.periodicExecutor = + Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("periodic-materialization")); + this.streamFactory = shared -> checkpointStorageWorkerView.createTaskOwnedStateStream(); + this.allowedNumberOfFailures = allowedNumberOfFailures; + this.materializationOnGoing = new AtomicBoolean(false); + this.retries = new AtomicInteger(allowedNumberOfFailures); + + this.materializedId = 0; + this.materializedState = materializedState; + + if (materializationEnabled) { + this.periodicExecutor.scheduleAtFixedRate( + this::triggerMaterialization, + periodicMaterializeInitDelay, + periodicMaterializeInterval, + TimeUnit.MILLISECONDS); + } + } + + @VisibleForTesting + public void triggerMaterialization() { + mailboxExecutor.execute( + () -> { + // Only one materialization ongoing at a time + if (!materializationOnGoing.compareAndSet(false, true)) { + return; + } + + SequenceNumber upTo = stateChangelogWriter.lastAppendedSequenceNumber().next(); + + if (upTo.equals(materializedState.lastMaterializedTo())) { + return; + } + + RunnableFuture<SnapshotResult<KeyedStateHandle>> materializedRunnableFuture = + keyedStateBackend.snapshot( + materializedId++, // TO DISCUSS: seems this does not matter, + // based on + System.currentTimeMillis(), + streamFactory, + CHECKPOINT_OPTIONS); + + // TODO: add metadata to log FLINK-23170 + asyncOperationsThreadPool.execute( + () -> asyncMaterialization(materializedRunnableFuture, upTo)); + }, + "materialization"); + } + + private void asyncMaterialization( + RunnableFuture<SnapshotResult<KeyedStateHandle>> materializedRunnableFuture, + SequenceNumber upTo) { + + FileSystemSafetyNet.initializeSafetyNetForThread(); + try { + FutureUtils.runIfNotDoneAndGet(materializedRunnableFuture); + mailboxExecutor.execute( + () -> { + materializedState = + new MaterializedState( + getMaterializedResult(materializedRunnableFuture.get()), + new ArrayList<>(), + upTo); + + checkState( + materializationOnGoing.compareAndSet(true, false), + "expect to finish materialization successfully, " + + "flag materializationOnGoing should be true before finishing."); + retries.set(0); + }, + "update materializedSnapshot up to changelog sequence number: {}", + upTo); + } catch (Exception e) { + int retryTime = retries.incrementAndGet(); + + LOG.info( + "Asynchronous part of materialization could not be completed for the {} time.", + retryTime, + e); + + // ToDO: Double check + // how the async phase is related to materialized state tracking + // refer to AsyncCheckpointRunnable#run#catch + // cleanup logic in StateUtil#discardStateFuture + handleExecutionException(materializedRunnableFuture); + + if (retryTime == allowedNumberOfFailures) { + asyncExceptionHandler.handleAsyncException( + "Fail to complete the asynchronous part of materialization", e); + } Review comment: Maybe let's discuss first what should happen with subsequent materialization once `numberOfConsecutiveFailures` reaches the limit? If we want them to run, should they have the same (configured) number of attempts? Then we should reset it here? If we don't, i.e. we prevent any further materializations, then we should check this counter before starting materialization? Currently, the latter option I think is enforced implicitly by `asyncExceptionHandler` - which fails the job. -- 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