Zakelly commented on a change in pull request #15420: URL: https://github.com/apache/flink/pull/15420#discussion_r655861032
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java ########## @@ -0,0 +1,154 @@ +/* + * 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.changelog; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.util.ExceptionUtils; + +import org.apache.flink.shaded.guava18.com.google.common.io.Closer; + +import javax.annotation.Nullable; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +import static java.util.Collections.unmodifiableList; +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * A handle to ChangelogStateBackend state. Consists of the base and delta parts. Base part + * references materialized state (e.g. SST files), while delta part references state changes that + * were not not materialized at the time of the snapshot. Both are potentially empty lists as there + * can be no state or multiple states (e.g. after rescaling). + */ +@Internal +public interface ChangelogStateBackendHandle extends KeyedStateHandle { + List<KeyedStateHandle> getMaterializedStateHandles(); + + List<ChangelogStateHandle> getNonMaterializedStateHandles(); + + class ChangelogStateBackendHandleImpl implements ChangelogStateBackendHandle { + private static final long serialVersionUID = 1L; + private final List<KeyedStateHandle> materialized; + private final List<ChangelogStateHandle> nonMaterialized; + private final KeyGroupRange keyGroupRange; + + public ChangelogStateBackendHandleImpl( + List<KeyedStateHandle> materialized, + List<ChangelogStateHandle> nonMaterialized, + KeyGroupRange keyGroupRange) { + this.materialized = unmodifiableList(materialized); + this.nonMaterialized = unmodifiableList(nonMaterialized); + this.keyGroupRange = keyGroupRange; + checkArgument(keyGroupRange.getNumberOfKeyGroups() > 0); + } + + @Override + public void registerSharedStates(SharedStateRegistry stateRegistry) { + stateRegistry.registerAll(materialized); + stateRegistry.registerAll(nonMaterialized); + } + + @Override + public void discardState() throws Exception { + try (Closer closer = Closer.create()) { + materialized.forEach(h -> closer.register(asCloseable(h))); + nonMaterialized.forEach(h -> closer.register(asCloseable(h))); + } + } + + @Override + public KeyGroupRange getKeyGroupRange() { + return keyGroupRange; + } + + @Nullable + @Override + public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) { + // todo: revisit/review + KeyGroupRange intersection = this.keyGroupRange.getIntersection(keyGroupRange); + if (intersection.getNumberOfKeyGroups() == 0) { + return null; + } + List<KeyedStateHandle> basePart = + this.materialized.stream() + .map(handle -> handle.getIntersection(keyGroupRange)) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + List<ChangelogStateHandle> deltaPart = + this.nonMaterialized.stream() + .map( + handle -> + (ChangelogStateHandle) + handle.getIntersection(keyGroupRange)) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + return new ChangelogStateBackendHandleImpl(basePart, deltaPart, intersection); + } + + @Override + public long getStateSize() { + return materialized.stream().mapToLong(StateObject::getStateSize).sum() + + nonMaterialized.stream().mapToLong(StateObject::getStateSize).sum(); + } + + @Override + public List<KeyedStateHandle> getMaterializedStateHandles() { + return materialized; + } + + @Override + public List<ChangelogStateHandle> getNonMaterializedStateHandles() { + return nonMaterialized; + } + + public ChangelogStateBackendHandleImpl withDelta(ChangelogStateHandle delta) { Review comment: This method is useless currently? ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java ########## @@ -358,7 +461,23 @@ public void notifyCheckpointAborted(long checkpointId) throws Exception { keyedStateBackend.getKeyContext(), stateChangelogWriter, meta); - return stateFactory.create(state, kvStateChangeLogger); + return stateFactory.create( + state, + kvStateChangeLogger, + keyedStateBackend /* pass the nested backend as key context so that it get key updates on recovery*/); + } + + private void completeRestore(Collection<ChangelogStateBackendHandle> stateHandles) { + if (!stateHandles.isEmpty()) { + synchronized (materialized) { // ensure visibility Review comment: Do we really need this ```synchronized ```? I mean it seems only the main thread will access this ```materialized ```. ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java ########## @@ -116,16 +140,41 @@ /** For caching the last accessed partitioned state. */ private String lastName; + private final FunctionDelegationHelper functionDelegationHelper = + new FunctionDelegationHelper(); + + /** Updated initially on restore and later upon materialization (after FLINK-21356). */ + private final List<KeyedStateHandle> materialized = new ArrayList<>(); + + /** Updated initially on restore and later cleared upon materialization (after FLINK-21356). */ + private final List<ChangelogStateHandle> restoredNonMaterialized = new ArrayList<>(); + + @Nullable private SequenceNumber lastUploadedFrom; + @Nullable private SequenceNumber lastUploadedTo; Review comment: IIUC, ```lastUploadedFrom``` is an inclusive bound and ```lastUploadedTo``` is an exclusive one? Better add some doc. ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/restore/KvStateStateChangeApplier.java ########## @@ -0,0 +1,89 @@ +/* + * 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.restore; + +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.runtime.state.heap.InternalKeyContext; +import org.apache.flink.runtime.state.internal.InternalKvState; +import org.apache.flink.state.changelog.StateChangeOperation; + +abstract class KvStateStateChangeApplier<K, N> implements StateChangeApplier { + private final InternalKeyContext<K> keyContext; + + protected abstract InternalKvState<K, N, ?> getState(); + + protected KvStateStateChangeApplier(InternalKeyContext<K> keyContext) { + this.keyContext = keyContext; + } + + @Override + public void apply(StateChangeOperation operation, DataInputView in) throws Exception { + K key = getState().getKeySerializer().deserialize(in); + keyContext.setCurrentKey(key); Review comment: Will this trigger the ```keySelectionListeners``` in ```AbstractKeyedStateBackend```? Any side effect? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/inmemory/InMemoryStateChangelogWriter.java ########## @@ -61,20 +64,26 @@ public SequenceNumber lastAppendedSequenceNumber() { } Review comment: I have a question here. Since the ```sqn``` starts from 0L (aka ```SequenceNumber.FIRST```) and the sqn of appended value starts from 1L, is it correct to initialize the exclusive bound ```materializedTo``` in ```ChangelogKeyedStateBackend``` as ```SequenceNumber.FIRST```? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/inmemory/InMemoryChangelogStateHandle.java ########## @@ -17,49 +17,50 @@ package org.apache.flink.runtime.state.changelog.inmemory; +import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.changelog.ChangelogStateHandle; +import org.apache.flink.runtime.state.changelog.SequenceNumber; import org.apache.flink.runtime.state.changelog.StateChange; -import org.apache.flink.runtime.state.changelog.StateChangelogHandle; -import org.apache.flink.util.CloseableIterator; import javax.annotation.Nullable; +import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.stream.Stream; -import static java.util.stream.Collectors.toList; - -class InMemoryStateChangelogHandle implements StateChangelogHandle<Void> { +/** In-memory {@link ChangelogStateHandle}. */ +@Internal +public class InMemoryChangelogStateHandle implements ChangelogStateHandle { private static final long serialVersionUID = 1L; - private final Map<Integer, List<byte[]>> changes; + private final List<StateChange> changes; + private final SequenceNumber from; // for debug purposes + private final SequenceNumber to; // for debug purposes + + public InMemoryChangelogStateHandle(List<StateChange> changes, long from, long to) { + this(changes, SequenceNumber.of(from), SequenceNumber.of(to)); + } - public InMemoryStateChangelogHandle(Map<Integer, List<byte[]>> changes) { + public InMemoryChangelogStateHandle( + List<StateChange> changes, SequenceNumber from, SequenceNumber to) { this.changes = changes; + this.from = from; + this.to = to; } @Override public void discardState() {} @Override public long getStateSize() { - return 0; - } - - @Override - public CloseableIterator<StateChange> getChanges(Void unused) { - return CloseableIterator.fromList( - changes.entrySet().stream().flatMap(this::mapEntryToChangeStream).collect(toList()), - change -> {}); + return changes.stream().mapToInt(change -> change.getChange().length).sum(); Review comment: Since the ```IntStream::sum()``` won't handle the integer overflow case, I suggest apply a ```mapToLong(e -> e)``` to this stream and then apply ```LongStream::sum()``` -- 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]
