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

chengjie.wu updated FLINK-10157:
--------------------------------
    Description: 
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!:)^

  was:
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!:)^


> The TTL state not allow `null` state value
> ------------------------------------------
>
>                 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
>            Priority: Minor
>         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)

Reply via email to