You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Stefan Richter (JIRA)" <ji...@apache.org> on 2018/09/21 13:36:00 UTC

[jira] [Closed] (FLINK-10157) Allow `null` user values in map state with TTL

     [ https://issues.apache.org/jira/browse/FLINK-10157?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Stefan Richter closed FLINK-10157.
----------------------------------
       Resolution: Fixed
    Fix Version/s: 1.6.2
                   1.7.0

Merged in:
master: f3432042fe
release-1.6: 3558d1586c

> 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
>             Fix For: 1.7.0, 1.6.2
>
>         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)