curcur commented on a change in pull request #15200:
URL: https://github.com/apache/flink/pull/15200#discussion_r598428534
##########
File path:
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogValueState.java
##########
@@ -38,8 +40,23 @@
extends AbstractChangelogState<K, N, V, InternalValueState<K, N, V>>
implements InternalValueState<K, N, V> {
- ChangelogValueState(InternalValueState<K, N, V> delegatedState) {
- super(delegatedState);
+ ChangelogValueState(
+ InternalValueState<K, N, V> state,
+ StateChangelogWriter<?> stateChangelogWriter,
+ InternalReadOnlyKeyContext<K> keyContext) {
+ this(
+ state,
+ new StateChangeLoggerImpl<>(
+ state.getKeySerializer(),
+ state.getNamespaceSerializer(),
+ state.getValueSerializer(),
+ keyContext,
+ stateChangelogWriter));
+ }
+
+ ChangelogValueState(
+ InternalValueState<K, N, V> delegatedState, StateChangeLogger<V,
N> changeLogger) {
+ super(delegatedState, changeLogger);
Review comment:
Is this only used by
`ChangelogValueState(InternalValueState<K, N, V> delegatedState,
StateChangeLogger<V, N> changeLogger)`?
Maybe we can remove this and call super in the first constructor directly.
same for other ChangelogXXXState.
##########
File path:
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogValueState.java
##########
@@ -49,17 +66,25 @@ public V value() throws IOException {
@Override
public void update(V value) throws IOException {
+ changeLogger.stateUpdated(value, currentNamespace);
delegatedState.update(value);
Review comment:
The updates here seem sequential.
This may not be a problem for memory writes for now. But would we consider
parallelizing the writes for the DFS version?
##########
File path:
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/StateChangeLoggerImpl.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.state.changelog.StateChangelogWriter;
+import org.apache.flink.runtime.state.heap.InternalReadOnlyKeyContext;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Collection;
+
+import static
org.apache.flink.state.changelog.StateChangeLoggerImpl.StateChangeOperation.ADD;
+import static
org.apache.flink.state.changelog.StateChangeLoggerImpl.StateChangeOperation.CHANGE_ELEMENT;
+import static
org.apache.flink.state.changelog.StateChangeLoggerImpl.StateChangeOperation.CLEAR;
+import static
org.apache.flink.state.changelog.StateChangeLoggerImpl.StateChangeOperation.MERGE_NS;
+import static
org.apache.flink.state.changelog.StateChangeLoggerImpl.StateChangeOperation.REMOVE_ELEMENT;
+import static
org.apache.flink.state.changelog.StateChangeLoggerImpl.StateChangeOperation.SET;
+
+class StateChangeLoggerImpl<Key, State, Ns> implements
StateChangeLogger<State, Ns> {
+
+ private final StateChangelogWriter<?> stateChangelogWriter;
+ private final InternalReadOnlyKeyContext<Key> keyContext;
+ private final TypeSerializer<Key> keySerializer;
+ private final TypeSerializer<Ns> namespaceSerializer;
+ private final TypeSerializer<State> valueSerializer;
+
+ public StateChangeLoggerImpl(
+ TypeSerializer<Key> keySerializer,
+ TypeSerializer<Ns> namespaceSerializer,
+ TypeSerializer<State> valueSerializer,
+ InternalReadOnlyKeyContext<Key> keyContext,
+ StateChangelogWriter<?> stateChangelogWriter) {
+ this.stateChangelogWriter = stateChangelogWriter;
+ this.keyContext = keyContext;
+ this.valueSerializer = valueSerializer;
+ this.keySerializer = keySerializer;
+ this.namespaceSerializer = namespaceSerializer;
+ }
+
+ @Override
+ public void stateUpdated(State newState, Ns ns) throws IOException {
+ if (newState == null) {
+ stateCleared(ns);
+ } else {
+ log(serialize(SET, ns, out -> valueSerializer.serialize(newState,
out)));
+ }
+ }
+
+ @Override
+ public void stateAdded(State addedState, Ns ns) throws IOException {
+ log(serialize(ADD, ns, out -> valueSerializer.serialize(addedState,
out)));
+ }
+
+ @Override
+ public void stateCleared(Ns ns) {
+ try {
+ log(serialize(CLEAR, ns, out -> {}));
+ } catch (IOException e) {
+ ExceptionUtils.rethrow(e);
+ }
+ }
Review comment:
why this not throw IOException, similar to other state methods in
`StateChangeLogger`?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/InternalReadOnlyKeyContext.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.runtime.state.heap;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.state.KeyGroupRange;
+
+/**
+ * This interface is the current context of a keyed state. It provides
information about the
+ * currently selected key in the context, the corresponding key-group, and
other key and
+ * key-grouping related information.
+ */
+@Internal
+public interface InternalReadOnlyKeyContext<K> {
Review comment:
Why we have to introduce `InternalReadOnlyKeyContext`. Is it possible to
just use `InternalKeyContext` instead?
We can use the InternalKeyContext maintained in `AbstractKeyedStateBackend`
to initiate `StateChangeLogger`, so that we do not need to have
AbstractKeyedStateBackend implement InternalReadOnlyKeyContext?
--
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]