Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,6 @@
* the current element. That way, the system can handle stream and state partitioning consistently
* together.
*
* <p>The user value could be null, but change log state backend is not compatible with the user
* value is null, see FLINK-38144 for more details.
*
* @param <UK> Type of the keys in the state.
* @param <UV> Type of the values in the state.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,6 @@
* the current element. That way, the system can handle stream and state partitioning consistently
* together.
*
* <p>The user value could be null, but change log state backend is not compatible with the user
* value is null, see FLINK-38144 for more details.
*
* @param <UK> Type of the keys in the state.
* @param <UV> Type of the values in the state.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.flink.util.function.ThrowingConsumer;

import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
import java.util.Map;
import java.util.Objects;
Expand Down Expand Up @@ -77,8 +78,12 @@ public UV getValue() {
public UV setValue(UV value) {
UV oldValue = entry.setValue(value);
try {
changeLogger.valueElementAddedOrUpdated(
getWriter(entry.getKey(), entry.getValue()), ns);
if (value == null) {
changeLogger.valueAdded(Collections.singletonMap(entry.getKey(), null), ns);
} else {
changeLogger.valueElementAddedOrUpdated(
getWriter(entry.getKey(), entry.getValue()), ns);
}
} catch (IOException e) {
ExceptionUtils.rethrow(e);
}
Expand All @@ -105,7 +110,11 @@ public UV get(UK key) throws Exception {
@Override
public void put(UK key, UV value) throws Exception {
delegatedState.put(key, value);
changeLogger.valueElementAddedOrUpdated(getWriter(key, value), getCurrentNamespace());
if (value == null) {
changeLogger.valueAdded(Collections.singletonMap(key, null), getCurrentNamespace());
} else {
changeLogger.valueElementAddedOrUpdated(getWriter(key, value), getCurrentNamespace());
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,6 @@ protected boolean checkMetrics() {
// Follow https://issues.apache.org/jira/browse/FLINK-38144
@Override
@TestTemplate
@Disabled("Currently, ChangelogStateBackend does not support null values for map state")
public void testMapStateWithNullValue() throws Exception {
super.testMapStateWithNullValue();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@
import org.apache.flink.runtime.state.TestTaskStateManager;
import org.apache.flink.testutils.junit.utils.TempDirUtils;

import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.TestTemplate;
import org.junit.jupiter.api.io.TempDir;

Expand Down Expand Up @@ -115,7 +114,6 @@ public void testMaterializedRestorePriorityQueue() throws Exception {
// Follow https://issues.apache.org/jira/browse/FLINK-38144
@Override
@TestTemplate
@Disabled("Currently, ChangelogStateBackend does not support null values for map state")
public void testMapStateWithNullValue() throws Exception {
super.testMapStateWithNullValue();
}
Expand Down