Myasuka commented on a change in pull request #15200: URL: https://github.com/apache/flink/pull/15200#discussion_r648207635
########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/AbstractStateChangeLogger.java ########## @@ -0,0 +1,172 @@ +/* + * 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.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.state.changelog.StateChangelogWriter; +import org.apache.flink.runtime.state.heap.InternalKeyContext; +import org.apache.flink.util.function.ThrowingConsumer; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.ADD; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.ADD_ELEMENT; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.ADD_OR_UPDATE_ELEMENT; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.CLEAR; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.REMOVE_ELEMENT; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.SET; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.SET_INTERNAL; +import static org.apache.flink.util.Preconditions.checkNotNull; + +abstract class AbstractStateChangeLogger<Key, Value, Ns> implements StateChangeLogger<Value, Ns> { + protected final StateChangelogWriter<?> stateChangelogWriter; + protected final InternalKeyContext<Key> keyContext; + + public AbstractStateChangeLogger( + StateChangelogWriter<?> stateChangelogWriter, InternalKeyContext<Key> keyContext) { + this.stateChangelogWriter = checkNotNull(stateChangelogWriter); + this.keyContext = checkNotNull(keyContext); + } + + @Override + public void valueUpdated(Value newValue, Ns ns) throws IOException { + if (newValue == null) { + valueCleared(ns); + } else { + log(SET, out -> serializeValue(newValue, out), ns); + } + } + + @Override + public void valueUpdatedInternal(Value newValue, Ns ns) throws IOException { + if (newValue == null) { + valueCleared(ns); + } else { + log(SET_INTERNAL, out -> serializeValue(newValue, out), ns); + } + } + + protected abstract void serializeValue(Value value, DataOutputViewStreamWrapper out) + throws IOException; + + @Override + public void valueAdded(Value addedValue, Ns ns) throws IOException { + log(ADD, out -> serializeValue(addedValue, out), ns); + } + + @Override + public void valueCleared(Ns ns) throws IOException { + log(CLEAR, out -> {}, ns); + } + + @Override + public void valueElementAdded( + ThrowingConsumer<DataOutputViewStreamWrapper, IOException> dataSerializer, Ns ns) + throws IOException { + log(ADD_ELEMENT, dataSerializer, ns); + } + + @Override + public void valueElementAddedOrUpdated( + ThrowingConsumer<DataOutputViewStreamWrapper, IOException> dataSerializer, Ns ns) + throws IOException { + log(ADD_OR_UPDATE_ELEMENT, dataSerializer, ns); + } + + @Override + public void valueElementRemoved( + ThrowingConsumer<DataOutputViewStreamWrapper, IOException> dataSerializer, Ns ns) + throws IOException { + log(REMOVE_ELEMENT, dataSerializer, ns); + } + + protected void log( + StateChangeOperation op, + ThrowingConsumer<DataOutputViewStreamWrapper, IOException> dataWriter, + Ns ns) + throws IOException { + // todo: log metadata (FLINK-22808) + stateChangelogWriter.append( + keyContext.getCurrentKeyGroupIndex(), serialize(op, ns, dataWriter)); + } + + private byte[] serialize( + StateChangeOperation op, + Ns ns, + ThrowingConsumer<DataOutputViewStreamWrapper, IOException> dataWriter) + throws IOException { + return serializeRaw( + wrapper -> { + wrapper.writeByte(op.code); + serializeScope(ns, wrapper); + dataWriter.accept(wrapper); Review comment: Since some `dataWriter` could be empty as `out -> {}`, I suppose it might help improve a bit performance by passing a `null` dataWriter here and check whether this is null to avoid creating a new lambda `out -> {}` here. ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogMapState.java ########## @@ -71,22 +108,72 @@ public boolean contains(UK key) throws Exception { @Override public Iterable<Map.Entry<UK, UV>> entries() throws Exception { - return delegatedState.entries(); + Iterator<Map.Entry<UK, UV>> iterator = delegatedState.iterator(); + return () -> getEntryIterator(iterator); + } + + private Iterator<Map.Entry<UK, UV>> getEntryIterator(Iterator<Map.Entry<UK, UV>> iterator) { + final N currentNamespace = getCurrentNamespace(); + return StateChangeLoggingIterator.create( + new Iterator<Map.Entry<UK, UV>>() { + @Override + public Map.Entry<UK, UV> next() { + return loggingMapEntry(iterator.next(), changeLogger, currentNamespace); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public void remove() { + iterator.remove(); + } + }, + changeLogger, + (entry, out) -> serializeKey(entry.getKey(), out), + currentNamespace); } @Override public Iterable<UK> keys() throws Exception { - return delegatedState.keys(); + Iterable<UK> iterable = delegatedState.keys(); + return iterable == null + ? null Review comment: First of all, the `delegatedState` would not return `null`. Secondly, if we want a check here, we should better return `Collections::emptyIterator` here. ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/KvStateChangeLogger.java ########## @@ -0,0 +1,26 @@ +/* + * 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 java.io.IOException; +import java.util.Collection; + +interface KvStateChangeLogger<State, Namespace> extends StateChangeLogger<State, Namespace> { Review comment: I think a javadoc telling that this changelogger is not only used for value state but all kinds of keyed state is better. Or maybe another name which would let developer mix with the usage of value state. ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/AbstractStateChangeLogger.java ########## @@ -0,0 +1,172 @@ +/* + * 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.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.state.changelog.StateChangelogWriter; +import org.apache.flink.runtime.state.heap.InternalKeyContext; +import org.apache.flink.util.function.ThrowingConsumer; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.ADD; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.ADD_ELEMENT; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.ADD_OR_UPDATE_ELEMENT; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.CLEAR; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.REMOVE_ELEMENT; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.SET; +import static org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.SET_INTERNAL; +import static org.apache.flink.util.Preconditions.checkNotNull; + +abstract class AbstractStateChangeLogger<Key, Value, Ns> implements StateChangeLogger<Value, Ns> { + protected final StateChangelogWriter<?> stateChangelogWriter; + protected final InternalKeyContext<Key> keyContext; + + public AbstractStateChangeLogger( + StateChangelogWriter<?> stateChangelogWriter, InternalKeyContext<Key> keyContext) { + this.stateChangelogWriter = checkNotNull(stateChangelogWriter); + this.keyContext = checkNotNull(keyContext); + } + + @Override + public void valueUpdated(Value newValue, Ns ns) throws IOException { + if (newValue == null) { + valueCleared(ns); + } else { + log(SET, out -> serializeValue(newValue, out), ns); + } + } + + @Override + public void valueUpdatedInternal(Value newValue, Ns ns) throws IOException { + if (newValue == null) { + valueCleared(ns); + } else { + log(SET_INTERNAL, out -> serializeValue(newValue, out), ns); + } + } + + protected abstract void serializeValue(Value value, DataOutputViewStreamWrapper out) + throws IOException; + + @Override + public void valueAdded(Value addedValue, Ns ns) throws IOException { + log(ADD, out -> serializeValue(addedValue, out), ns); + } + + @Override + public void valueCleared(Ns ns) throws IOException { + log(CLEAR, out -> {}, ns); + } + + @Override + public void valueElementAdded( + ThrowingConsumer<DataOutputViewStreamWrapper, IOException> dataSerializer, Ns ns) + throws IOException { + log(ADD_ELEMENT, dataSerializer, ns); + } + + @Override + public void valueElementAddedOrUpdated( + ThrowingConsumer<DataOutputViewStreamWrapper, IOException> dataSerializer, Ns ns) + throws IOException { + log(ADD_OR_UPDATE_ELEMENT, dataSerializer, ns); + } + + @Override + public void valueElementRemoved( + ThrowingConsumer<DataOutputViewStreamWrapper, IOException> dataSerializer, Ns ns) + throws IOException { + log(REMOVE_ELEMENT, dataSerializer, ns); + } + + protected void log( + StateChangeOperation op, + ThrowingConsumer<DataOutputViewStreamWrapper, IOException> dataWriter, + Ns ns) + throws IOException { + // todo: log metadata (FLINK-22808) + stateChangelogWriter.append( + keyContext.getCurrentKeyGroupIndex(), serialize(op, ns, dataWriter)); + } + + private byte[] serialize( + StateChangeOperation op, + Ns ns, + ThrowingConsumer<DataOutputViewStreamWrapper, IOException> dataWriter) + throws IOException { + return serializeRaw( + wrapper -> { + wrapper.writeByte(op.code); + serializeScope(ns, wrapper); + dataWriter.accept(wrapper); + }); + } + + protected abstract void serializeScope(Ns ns, DataOutputViewStreamWrapper out) + throws IOException; + + private byte[] serializeRaw( + ThrowingConsumer<DataOutputViewStreamWrapper, IOException> dataWriter) + throws IOException { + try (ByteArrayOutputStream out = new ByteArrayOutputStream(); Review comment: I believe you will add TODO for performance in https://github.com/apache/flink/pull/15420, right? ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/StateChangeLoggingIterator.java ########## @@ -0,0 +1,81 @@ +/* + * 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.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.function.BiConsumerWithException; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Iterator; + +class StateChangeLoggingIterator<State, StateElement, Namespace> implements Iterator<StateElement> { + + private final Iterator<StateElement> iterator; + private final StateChangeLogger<State, Namespace> changeLogger; + private final BiConsumerWithException<StateElement, DataOutputViewStreamWrapper, IOException> + removalWriter; + private final Namespace ns; + @Nullable private StateElement lastReturned; + + private StateChangeLoggingIterator( + Iterator<StateElement> iterator, + StateChangeLogger<State, Namespace> changeLogger, + BiConsumerWithException<StateElement, DataOutputViewStreamWrapper, IOException> + removalWriter, + Namespace ns) { + this.iterator = iterator; + this.changeLogger = changeLogger; + this.removalWriter = removalWriter; + this.ns = ns; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public StateElement next() { + return lastReturned = iterator.next(); + } + + @Override + public void remove() { + iterator.remove(); + try { + changeLogger.valueElementRemoved(out -> removalWriter.accept(lastReturned, out), ns); + } catch (IOException e) { + ExceptionUtils.rethrow(e); + } + } + + @Nullable + public static <Namespace, State, StateElement> Iterator<StateElement> create( + @Nullable Iterator<StateElement> iterator, + StateChangeLogger<State, Namespace> changeLogger, + BiConsumerWithException<StateElement, DataOutputViewStreamWrapper, IOException> + removalWriter, + Namespace ns) { + return iterator == null + ? null Review comment: We should return `Collections.emptyIterator()` here and tag this method as `@Nonnull` which aligns with `InternalPriorityQueue#iterator` -- 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]
