[ https://issues.apache.org/jira/browse/FLINK-10157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16621945#comment-16621945 ]
ASF GitHub Bot commented on FLINK-10157: ---------------------------------------- azagrebin commented on a change in pull request #6707: [FLINK-10157] [State TTL] Allow `null` user values in map state with TTL URL: https://github.com/apache/flink/pull/6707#discussion_r219146864 ########## File path: flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullableSerializer.java ########## @@ -116,7 +139,7 @@ public T copy(T from, T reuse) { @Override public int getLength() { int len = originalSerializer.getLength(); - return len < 0 ? len : len + 1; + return len == 0 ? 1 : -1; Review comment: The serializer does not have fixed length in common case now (1 or 1 + original length). It is fixed to 1 only of original length is zero. Let's add an option to pad the null value and preserve the fixed length if original serializer has the fixed length. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Allow `null` user values in map state with TTL > ---------------------------------------------- > > Key: FLINK-10157 > URL: https://issues.apache.org/jira/browse/FLINK-10157 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing > Affects Versions: 1.6.0 > Environment: Flink:1.6.0 > Scala:2.11 > JDK:1.8 > Reporter: chengjie.wu > Assignee: Andrey Zagrebin > Priority: Minor > Labels: pull-request-available > Attachments: StateWithOutTtlTest.scala, StateWithTtlTest.scala > > > Thanks for the StateTtl feature,this is exactly what I need now! But I found > an issue. > In the previous version or when StateTtl is not enabled,MapState allows > `null` value,that means after > {code:java} > mapState.put("key", null){code} > , then > {code:java} > mapState.contains("key"){code} > will return {color:#ff0000}*true*{color}, but when StateTtl is enabled, > {code:java} > mapState.contains("key"){code} > will return {color:#ff0000}*false*{color}(*the key has not expired*). > So I think the field `userValue` in > `org.apache.flink.runtime.state.ttl.TtlValue` should allow `null` value. User > state is null may not means the TtlValue should be null. > > {code:java} > /** > * This class wraps user value of state with TTL. > * > * @param <T> Type of the user value of state with TTL > */ > class TtlValue<T> implements Serializable { > private final T userValue; > private final long lastAccessTimestamp; > TtlValue(T userValue, long lastAccessTimestamp) { > Preconditions.checkNotNull(userValue); > this.userValue = userValue; > this.lastAccessTimestamp = lastAccessTimestamp; > } > T getUserValue() { > return userValue; > } > long getLastAccessTimestamp() { > return lastAccessTimestamp; > } > } > {code} > Am I understanding right? > This is my test class. > [^StateWithTtlTest.scala] [^StateWithOutTtlTest.scala] > ^Thanks!:)^ -- This message was sent by Atlassian JIRA (v7.6.3#76005)