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

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

Github user azagrebin commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6186#discussion_r196826339
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlListState.java
 ---
    @@ -0,0 +1,171 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.state.ttl;
    +
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.runtime.state.internal.InternalListState;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.stream.Collectors;
    +import java.util.stream.StreamSupport;
    +
    +/**
    + * This class wraps list state with TTL logic.
    + *
    + * @param <K> The type of key the state is associated to
    + * @param <N> The type of the namespace
    + * @param <T> Type of the user entry value of state with TTL
    + */
    +class TtlListState<K, N, T> extends
    +   AbstractTtlState<K, N, List<T>, List<TtlValue<T>>, InternalListState<K, 
N, TtlValue<T>>>
    +   implements InternalListState<K, N, T> {
    +   TtlListState(
    +           InternalListState<K, N, TtlValue<T>> originalState,
    +           TtlConfig config,
    +           TtlTimeProvider timeProvider,
    +           TypeSerializer<List<T>> valueSerializer) {
    +           super(originalState, config, timeProvider, valueSerializer);
    +   }
    +
    +   @Override
    +   public void update(List<T> values) throws Exception {
    +           updateInternal(values);
    +   }
    +
    +   @Override
    +   public void addAll(List<T> values) throws Exception {
    +           Preconditions.checkNotNull(values, "List of values to add 
cannot be null.");
    +           original.addAll(withTs(values));
    +   }
    +
    +   @Override
    +   public Iterable<T> get() throws Exception {
    +           Iterable<TtlValue<T>> ttlValue = original.get();
    +           ttlValue = ttlValue == null ? Collections.emptyList() : 
ttlValue;
    +           if (updateTsOnRead) {
    +                   List<TtlValue<T>> collected = collect(ttlValue);
    +                   ttlValue = collected;
    +                   updateTs(collected);
    +           }
    +           final Iterable<TtlValue<T>> finalResult = ttlValue;
    --- End diff --
    
    The `ttlValue` is changed before the lambda from return statement so it is 
not effectively immutable any more to be used in lambda, that is why 
`finalResult` is formally needed to avoid compilation error.


> Create wrapper with TTL logic for value state
> ---------------------------------------------
>
>                 Key: FLINK-9514
>                 URL: https://issues.apache.org/jira/browse/FLINK-9514
>             Project: Flink
>          Issue Type: Sub-task
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.6.0
>            Reporter: Andrey Zagrebin
>            Assignee: Andrey Zagrebin
>            Priority: Major
>             Fix For: 1.6.0
>
>
> TTL state decorator uses original state with packed TTL and add TTL logic 
> using time provider:
> {code:java}
> TtlValueState<V> implements ValueState<V> {
>   ValueState<TtlValue<V>> underlyingState;
>   InternalTimeService timeProvider;
>   V value() {
>     TtlValue<V> valueWithTtl = underlyingState.get();
>     // ttl logic here (e.g. update timestamp)
>     return valueWithTtl.getValue();
>   }
>   void update() { ... underlyingState.update(valueWithTtl) ...  }
> }
> {code}
> TTL decorators are apply to state produced by normal state binder in its TTL 
> wrapper from FLINK-9513



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to