[ 
https://issues.apache.org/jira/browse/FLINK-10157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16621875#comment-16621875
 ] 

ASF GitHub Bot commented on FLINK-10157:
----------------------------------------

StefanRRichter 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_r219131023
 
 

 ##########
 File path: 
flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullableSerializer.java
 ##########
 @@ -56,33 +59,51 @@
 public class NullableSerializer<T> extends TypeSerializer<T> {
        private static final long serialVersionUID = 3335569358214720033L;
 
+       @Nonnull
        private final TypeSerializer<T> originalSerializer;
 
-       private NullableSerializer(TypeSerializer<T> originalSerializer) {
+       private NullableSerializer(@Nonnull TypeSerializer<T> 
originalSerializer) {
                Preconditions.checkNotNull(originalSerializer, "The original 
serializer cannot be null");
                this.originalSerializer = originalSerializer;
        }
 
        /**
-        * This method tries to serialize null value with the {@code 
originalSerializer}
+        * This method tries to serialize {@code null} value with the {@code 
originalSerializer}
         * and wraps it in case of {@link NullPointerException}, otherwise it 
returns the {@code originalSerializer}.
+        *
+        * @param originalSerializer serializer to wrap and add {@code null} 
support
+        * @return serializer which supports {@code null} values
         */
-       public static <T> TypeSerializer<T> 
wrapIfNullIsNotSupported(TypeSerializer<T> originalSerializer) {
+       public static <T> TypeSerializer<T> wrapIfNullIsNotSupported(@Nonnull 
TypeSerializer<T> originalSerializer) {
                return checkIfNullSupported(originalSerializer) ? 
originalSerializer : wrap(originalSerializer);
        }
 
-       private static <T> boolean checkIfNullSupported(TypeSerializer<T> 
originalSerializer) {
+       /**
+        * This method checks if {@code serializer} supports {@code null} value.
+        *
+        * @param serializer serializer to check
+        */
+       public static <T> boolean checkIfNullSupported(@Nonnull 
TypeSerializer<T> serializer) {
                try {
-                       originalSerializer.serialize(null, new 
DataOutputSerializer(1));
-                       
Preconditions.checkArgument(originalSerializer.copy(null) == null);
-               } catch (NullPointerException | IOException e) {
+                       int length = serializer.getLength() > 0 ? 
serializer.getLength() : 1;
+                       DataOutputSerializer dos = new 
DataOutputSerializer(length);
+                       serializer.serialize(null, dos);
+                       DataInputDeserializer dis = new 
DataInputDeserializer(dos.getSharedBuffer());
+                       Preconditions.checkArgument(serializer.deserialize(dis) 
== null);
 
 Review comment:
   My main point is, you use precondition that could produce an exception and 
then immediately catch the exception and translate it to `false`. So my 
question is, why use precondition and not just an `if` that affects the result. 
Or in other words, why do we need to go through exceptions for this (nobody 
takes note that the `false` came from an exceptional case as well)?

----------------------------------------------------------------
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)

Reply via email to