Zakelly commented on a change in pull request #15420: URL: https://github.com/apache/flink/pull/15420#discussion_r647183702
########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/StateChangeLogger.java ########## @@ -0,0 +1,122 @@ +/* + * 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 org.apache.flink.util.function.ThrowingConsumer; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + +interface StateChangeLogger<State, Namespace> { + static <Namespace, State, StateElement> Iterator<StateElement> loggingIterator( + @Nullable Iterator<StateElement> iterator, + StateChangeLogger<State, Namespace> changeLogger, + BiConsumerWithException<StateElement, DataOutputViewStreamWrapper, IOException> + changeWriter, + Namespace ns) { + if (iterator == null) { + return null; + } + return new Iterator<StateElement>() { + + @Nullable private StateElement lastReturned; + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public StateElement next() { + return lastReturned = iterator.next(); + } + + @Override + public void remove() { + try { + changeLogger.stateElementRemoved( + out -> changeWriter.accept(lastReturned, out), ns); + } catch (IOException e) { + ExceptionUtils.rethrow(e); + } + iterator.remove(); Review comment: Maybe should try ```iterator.remove()``` first since it may produce an ```IllegalStateException```. ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogMapState.java ########## @@ -71,22 +102,66 @@ 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 loggingIterator( + new Iterator<Map.Entry<UK, UV>>() { + @Override + public Map.Entry<UK, UV> next() { + return loggingMapEntry( + iterator.next(), changeLogger, changeWriter, currentNamespace); Review comment: This ```changeWriter``` here only write the key out, maybe should write the value as well? ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/StateChangeLogger.java ########## @@ -0,0 +1,122 @@ +/* + * 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 org.apache.flink.util.function.ThrowingConsumer; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + +interface StateChangeLogger<State, Namespace> { + static <Namespace, State, StateElement> Iterator<StateElement> loggingIterator( + @Nullable Iterator<StateElement> iterator, + StateChangeLogger<State, Namespace> changeLogger, + BiConsumerWithException<StateElement, DataOutputViewStreamWrapper, IOException> + changeWriter, + Namespace ns) { + if (iterator == null) { + return null; + } + return new Iterator<StateElement>() { + + @Nullable private StateElement lastReturned; + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public StateElement next() { + return lastReturned = iterator.next(); + } + + @Override + public void remove() { + try { + changeLogger.stateElementRemoved( + out -> changeWriter.accept(lastReturned, out), ns); + } catch (IOException e) { + ExceptionUtils.rethrow(e); + } + iterator.remove(); + } + }; + } + + static <Namespace, State, StateElement> Iterable<StateElement> loggingIterable( + @Nullable Iterable<StateElement> iterable, + KvStateChangeLogger<State, Namespace> changeLogger, + BiConsumerWithException<StateElement, DataOutputViewStreamWrapper, IOException> + changeWriter, + Namespace ns) { + if (iterable == null) { + return null; + } + return () -> loggingIterator(iterable.iterator(), changeLogger, changeWriter, ns); + } + + static <UK, UV, State, Namespace> Map.Entry<UK, UV> loggingMapEntry( + Map.Entry<UK, UV> entry, + KvStateChangeLogger<State, Namespace> changeLogger, + BiConsumerWithException<Map.Entry<UK, UV>, DataOutputViewStreamWrapper, IOException> + changeWriter, + Namespace ns) { + return new Map.Entry<UK, UV>() { + @Override + public UK getKey() { + return entry.getKey(); + } + + @Override + public UV getValue() { + return entry.getValue(); + } + + @Override + public UV setValue(UV value) { + try { + changeLogger.stateElementChanged(out -> changeWriter.accept(entry, out), ns); + } catch (IOException e) { + ExceptionUtils.rethrow(e); + } + return entry.setValue(value); Review comment: Should call ```entry.setValue(value)``` first, since the new value should be recorded by ```changeWriter```. ########## File path: flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/TestChangeLoggerKv.java ########## @@ -0,0 +1,74 @@ +/* + * 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.function.ThrowingConsumer; + +import java.io.IOException; +import java.util.Collection; + +class TestChangeLoggerKv<State> implements KvStateChangeLogger<State, String> { Review comment: Maybe add some correctness test for writing the right key/value when state changed? ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogMapState.java ########## @@ -71,22 +102,66 @@ 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 loggingIterator( + new Iterator<Map.Entry<UK, UV>>() { + @Override + public Map.Entry<UK, UV> next() { + return loggingMapEntry( + iterator.next(), changeLogger, changeWriter, currentNamespace); Review comment: Moreover, I suggest we define a unified writer for ```StateChangeLogger#stateElementRemoved```. For example, define a function here like: ``` private ThrowingConsumer<DataOutputViewStreamWrapper, IOException> getStateChangeWriter(UK key, UV value) { return out -> { serializeKey(key, out); serializeValue(value, out); }; } ``` In ```ChangelogMapState#put```, we call it like: ``` @Override public void put(UK key, UV value) throws Exception { if (getValueSerializer() instanceof MapSerializer) { changeLogger.stateElementChanged( getStateChangeWriter(key, value), getCurrentNamespace()); } else { changeLogger.stateAdded(singletonMap(key, value), getCurrentNamespace()); } delegatedState.put(key, value); } ``` As for the inner ```Iterator#next``` of ```getEntryIterator``` here, we call: ```suggestion iterator.next(), changeLogger, (entry, out) -> getStateChangeWriter(entry.getKey(), entry.getValue()).accept(out), currentNamespace); ``` It is easier to maintain the output format for a state change. ########## File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/StateChangeLogger.java ########## @@ -0,0 +1,122 @@ +/* + * 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 org.apache.flink.util.function.ThrowingConsumer; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + Review comment: Better add some javedoc. -- 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]
