flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From azagrebin <...@git.apache.org>
Subject [GitHub] flink pull request #6186: [FLINK-9514,FLINK-9515,FLINK-9516] State ttl wrapp...
Date Wed, 20 Jun 2018 15:25:37 GMT
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.


---

Mime
View raw message