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

    https://github.com/apache/flink/pull/6186#discussion_r197077377
  
    --- 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;
    +           return () -> new IteratorWithCleanup(finalResult.iterator());
    +   }
    +
    +   private void updateTs(List<TtlValue<T>> ttlValue) throws Exception {
    +           List<TtlValue<T>> unexpiredWithUpdatedTs = ttlValue.stream()
    +                   .filter(v -> !expired(v))
    +                   .map(this::rewrapWithNewTs)
    +                   .collect(Collectors.toList());
    +           if (!unexpiredWithUpdatedTs.isEmpty()) {
    +                   original.update(unexpiredWithUpdatedTs);
    +           }
    +   }
    +
    +   @Override
    +   public void add(T value) throws Exception {
    +           Preconditions.checkNotNull(value, "You cannot add null to a 
ListState.");
    +           original.add(wrapWithTs(value));
    +   }
    +
    +   @Override
    +   public void clear() {
    +           original.clear();
    +   }
    +
    +   @Override
    +   public void mergeNamespaces(N target, Collection<N> sources) throws 
Exception {
    +           original.mergeNamespaces(target, sources);
    +   }
    +
    +   @Override
    +   public List<T> getInternal() throws Exception {
    +           return collect(get());
    +   }
    +
    +   private <E> List<E> collect(Iterable<E> iterable) {
    --- End diff --
    
    In general the idea is to keep `get()` method as lazy as possible, return 
`Iterable` and avoid querying backend if not needed. At the moment list state 
internally stores `List` according to API, not `Iterable` (something to think 
about in future). So `getInternal()` returns `List` - collected `Iterable`. I 
agree it looks weird but if `get()` calls `getInternal()` it forces collecting 
`List`.
    
    If we do not need update on read, everything can still stay lazy in final 
resulting `IteratorWithCleanup` - one more potential user iteration over 
original `Iterable` from backend.
    
    If we do need update on read, we have to collect it anyways fully, so 
potentially query backend. That is why it is collected and cached to avoid 
iterating over original `Iterable` from backend. `updateTs` and 
`IteratorWithCleanup` perform a bit different iterations over `collected` but 
already materialised in this case.
    
    Currently we return `List` object in both backends. I think, it is a good 
idea to check `Iterable` in `collect` whether it is already a collected `List` 
and skip actual collecting then, thanks. I will add it.


---

Reply via email to