rkhachatryan commented on a change in pull request #14943: URL: https://github.com/apache/flink/pull/14943#discussion_r585608826
########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java ########## @@ -0,0 +1,105 @@ +/* + * 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; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.state.AbstractKeyedStateBackend; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.delegate.DelegateStateBackend; +import org.apache.flink.runtime.state.ttl.TtlTimeProvider; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.util.Collection; + +/** + * This state backend holds the working state in the underlying delegatedStateBackend, and forwards + * state changes to State Changelog. + */ +public class ChangelogStateBackend implements DelegateStateBackend { Review comment: I think `ChangelogStateBackend` should also implement `ConfigurableStateBackend` and delegate `configure` call if needed. Alteratively, it can be done in the `DelegateStateBackend` interface (default method). ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java ########## @@ -0,0 +1,331 @@ +/* + * 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; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.AggregatingStateDescriptor; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.state.ReducingStateDescriptor; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.CheckpointType; +import org.apache.flink.runtime.state.AbstractKeyedStateBackend; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; +import org.apache.flink.runtime.state.Keyed; +import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.runtime.state.KeyedStateFunction; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.PriorityComparable; +import org.apache.flink.runtime.state.SavepointResources; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.StateSnapshotTransformer; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement; +import org.apache.flink.runtime.state.internal.InternalKvState; +import org.apache.flink.runtime.state.ttl.TtlStateFactory; +import org.apache.flink.runtime.state.ttl.TtlTimeProvider; +import org.apache.flink.util.FlinkRuntimeException; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.RunnableFuture; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link KeyedStateBackend} that keeps state on the underlying delegated keyed state backend as + * well as on the state change log. + * + * @param <K> The key by which state is keyed. + */ +@Internal +class ChangelogKeyedStateBackend<K> + implements CheckpointableKeyedStateBackend<K>, CheckpointListener { + + private static final Map<Class<? extends StateDescriptor>, StateFactory> STATE_FACTORIES = + Stream.of( + Tuple2.of( + ValueStateDescriptor.class, + (StateFactory) ChangelogValueState::create), + Tuple2.of( + ListStateDescriptor.class, + (StateFactory) ChangelogListState::create), + Tuple2.of( + ReducingStateDescriptor.class, + (StateFactory) ChangelogReducingState::create), + Tuple2.of( + AggregatingStateDescriptor.class, + (StateFactory) ChangelogAggregatingState::create), + Tuple2.of( + MapStateDescriptor.class, + (StateFactory) ChangelogMapState::create)) + .collect(Collectors.toMap(t -> t.f0, t -> t.f1)); + + /** delegated keyedStateBackend. */ + private final AbstractKeyedStateBackend<K> keyedStateBackend; + + /** + * This is the cache maintained by the DelegateKeyedStateBackend itself. It is not the same as + * the underlying delegated keyedStateBackend. InternalKvState is a delegated state. + */ + private final HashMap<String, InternalKvState<K, ?, ?>> keyValueStatesByName; + + private final ExecutionConfig executionConfig; + + private final TtlTimeProvider ttlTimeProvider; + + /** last accessed partitioned state. */ + @SuppressWarnings("rawtypes") + private InternalKvState lastState; + + /** For caching the last accessed partitioned state. */ + private String lastName; + + public ChangelogKeyedStateBackend( + AbstractKeyedStateBackend<K> keyedStateBackend, + ExecutionConfig executionConfig, + TtlTimeProvider ttlTimeProvider) { + this.keyedStateBackend = keyedStateBackend; + this.executionConfig = executionConfig; + this.ttlTimeProvider = ttlTimeProvider; + this.keyValueStatesByName = new HashMap<>(); + } + + // -------------------- CheckpointableKeyedStateBackend -------------------------------- + @Override + public KeyGroupRange getKeyGroupRange() { + return keyedStateBackend.getKeyGroupRange(); + } + + @Override + public void close() throws IOException { + keyedStateBackend.close(); + } + + @Override + public void setCurrentKey(K newKey) { + keyedStateBackend.setCurrentKey(newKey); + } + + @Override + public K getCurrentKey() { + return keyedStateBackend.getCurrentKey(); + } + + @Override + public TypeSerializer<K> getKeySerializer() { + return keyedStateBackend.getKeySerializer(); + } + + @Override + public <N> Stream<K> getKeys(String state, N namespace) { + return keyedStateBackend.getKeys(state, namespace); + } + + @Override + public <N> Stream<Tuple2<K, N>> getKeysAndNamespaces(String state) { + return keyedStateBackend.getKeysAndNamespaces(state); + } + + @Override + public void dispose() { + keyedStateBackend.dispose(); + lastName = null; + lastState = null; + keyValueStatesByName.clear(); + } + + @Override + public void registerKeySelectionListener(KeySelectionListener<K> listener) { + keyedStateBackend.registerKeySelectionListener(listener); + } + + @Override + public boolean deregisterKeySelectionListener(KeySelectionListener<K> listener) { + return keyedStateBackend.deregisterKeySelectionListener(listener); + } + + @Override + public <N, S extends State, T> void applyToAllKeys( + N namespace, + TypeSerializer<N> namespaceSerializer, + StateDescriptor<S, T> stateDescriptor, + KeyedStateFunction<K, S> function) + throws Exception { + + keyedStateBackend.applyToAllKeys( + namespace, + namespaceSerializer, + stateDescriptor, + function, + this::getPartitionedState); + } + + @Override + @SuppressWarnings("unchecked") + public <N, S extends State> S getPartitionedState( + N namespace, + TypeSerializer<N> namespaceSerializer, + StateDescriptor<S, ?> stateDescriptor) + throws Exception { + + checkNotNull(namespace, "Namespace"); + + if (lastName != null && lastName.equals(stateDescriptor.getName())) { + lastState.setCurrentNamespace(namespace); + return (S) lastState; + } + + final InternalKvState<K, ?, ?> previous = + keyValueStatesByName.get(stateDescriptor.getName()); + if (previous != null) { + lastState = previous; + lastState.setCurrentNamespace(namespace); + lastName = stateDescriptor.getName(); + return (S) previous; + } + + final S state = getOrCreateKeyedState(namespaceSerializer, stateDescriptor); + final InternalKvState<K, N, ?> kvState = (InternalKvState<K, N, ?>) state; + + lastName = stateDescriptor.getName(); + lastState = kvState; + kvState.setCurrentNamespace(namespace); + + return state; + } + + @Nonnull + @Override + public RunnableFuture<SnapshotResult<KeyedStateHandle>> snapshot( + long checkpointId, + long timestamp, + @Nonnull CheckpointStreamFactory streamFactory, + @Nonnull CheckpointOptions checkpointOptions) + throws Exception { + return keyedStateBackend.snapshot( + checkpointId, timestamp, streamFactory, checkpointOptions); + } + + @Nonnull + @Override + public <T extends HeapPriorityQueueElement & PriorityComparable<? super T> & Keyed<?>> + KeyGroupedInternalPriorityQueue<T> create( + @Nonnull String stateName, + @Nonnull TypeSerializer<T> byteOrderedElementSerializer) { + return keyedStateBackend.create(stateName, byteOrderedElementSerializer); + } + + @Override + public int numKeyValueStateEntries() { + return keyedStateBackend.numKeyValueStateEntries(); + } + + @Override + public boolean isStateImmutableInStateBackend(CheckpointType checkpointOptions) { + return keyedStateBackend.requiresLegacySynchronousTimerSnapshots(checkpointOptions); + } Review comment: ```suggestion @Override public boolean isStateImmutableInStateBackend(CheckpointType checkpointOptions) { return keyedStateBackend.isStateImmutableInStateBackend(checkpointOptions); } ``` ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java ########## @@ -249,6 +266,66 @@ public static StateBackend fromApplicationOrConfigOrDefault( return backend; } + /** + * This is the state backend loader that loads a {@link DelegateStateBackend} wrapping the state + * backend loaded from {@link StateBackendLoader#fromApplicationOrConfigOrDefault} when + * delegation is enabled. If delegation is not enabled, the underlying wrapped state backend is + * returned instead. + * + * <p>{@link DelegateStateBackend} can only be enabled through configuration. + * + * @param fromApplication StateBackend defined from application + * @param config The configuration to load the state backend from + * @param classLoader The class loader that should be used to load the state backend + * @param logger Optionally, a logger to log actions to (may be null) + * @return The instantiated state backend. + * @throws DynamicCodeLoadingException Thrown if a state backend (factory) is configured and the + * (factory) class was not found or could not be instantiated + * @throws IllegalConfigurationException May be thrown by the StateBackendFactory when creating + * / configuring the state backend in the factory + * @throws IOException May be thrown by the StateBackendFactory when instantiating the state + * backend + */ + public static StateBackend loadStateBackend( + @Nullable StateBackend fromApplication, + Configuration config, + ClassLoader classLoader, + @Nullable Logger logger) + throws IllegalConfigurationException, DynamicCodeLoadingException, IOException { + + final StateBackend backend = + fromApplicationOrConfigOrDefault(fromApplication, config, classLoader, logger); + + if (config.get(CheckpointingOptions.ENABLE_STATE_CHANGE_LOG)) { Review comment: Is a similar branch missing in `loadStateBackendFromConfig`? It is called for example from `StreamExecutionEnvironment.loadStateBackend` which I think should get a wrapped version. ---------------------------------------------------------------- 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]
