org.apache.flink.runtime.state.ttl.TtlListState Maven / Gradle / Ivy
/*
* 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.api.common.typeutils.base.ListSerializer;
import org.apache.flink.runtime.state.internal.InternalListState;
import org.apache.flink.util.Preconditions;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
/**
* This class wraps list state with TTL logic.
*
* @param The type of key the state is associated to
* @param The type of the namespace
* @param Type of the user entry value of state with TTL
*/
class TtlListState extends
AbstractTtlState, List>, InternalListState>>
implements InternalListState {
TtlListState(TtlStateContext>, List> ttlStateContext) {
super(ttlStateContext);
}
@Override
public void update(List values) throws Exception {
accessCallback.run();
updateInternal(values);
}
@Override
public void addAll(List values) throws Exception {
accessCallback.run();
Preconditions.checkNotNull(values, "List of values to add cannot be null.");
original.addAll(withTs(values));
}
@Override
public Iterable get() throws Exception {
accessCallback.run();
Iterable> ttlValue = original.get();
ttlValue = ttlValue == null ? Collections.emptyList() : ttlValue;
if (updateTsOnRead) {
List> collected = collect(ttlValue);
ttlValue = collected;
// the underlying state in backend is iterated in updateTs anyways
// to avoid reiterating backend in IteratorWithCleanup
// it is collected and iterated next time in memory
updateTs(collected);
}
final Iterable> finalResult = ttlValue;
return () -> new IteratorWithCleanup(finalResult.iterator());
}
private void updateTs(List> ttlValues) throws Exception {
List> unexpiredWithUpdatedTs = new ArrayList<>(ttlValues.size());
long currentTimestamp = timeProvider.currentTimestamp();
for (TtlValue ttlValue : ttlValues) {
if (!TtlUtils.expired(ttlValue, ttl, currentTimestamp)) {
unexpiredWithUpdatedTs.add(TtlUtils.wrapWithTs(ttlValue.getUserValue(), currentTimestamp));
}
}
if (!unexpiredWithUpdatedTs.isEmpty()) {
original.update(unexpiredWithUpdatedTs);
}
}
@Override
public void add(T value) throws Exception {
accessCallback.run();
Preconditions.checkNotNull(value, "You cannot add null to a ListState.");
original.add(wrapWithTs(value));
}
@Nullable
@Override
public List> getUnexpiredOrNull(@Nonnull List> ttlValues) {
long currentTimestamp = timeProvider.currentTimestamp();
List> unexpired = new ArrayList<>(ttlValues.size());
TypeSerializer> elementSerializer =
((ListSerializer>) original.getValueSerializer()).getElementSerializer();
for (TtlValue ttlValue : ttlValues) {
if (!TtlUtils.expired(ttlValue, ttl, currentTimestamp)) {
// we have to do the defensive copy to update the value
unexpired.add(elementSerializer.copy(ttlValue));
}
}
if (!unexpired.isEmpty()) {
return unexpired;
} else {
return ttlValues.size() == unexpired.size() ? ttlValues : unexpired;
}
}
@Override
public void clear() {
original.clear();
}
@Override
public void mergeNamespaces(N target, Collection sources) throws Exception {
original.mergeNamespaces(target, sources);
}
@Override
public List getInternal() throws Exception {
return collect(get());
}
private List collect(Iterable iterable) {
if (iterable instanceof List) {
return (List) iterable;
} else {
List list = new ArrayList<>();
for (E element : iterable) {
list.add(element);
}
return list;
}
}
@Override
public void updateInternal(List valueToStore) throws Exception {
Preconditions.checkNotNull(valueToStore, "List of values to update cannot be null.");
original.update(withTs(valueToStore));
}
private List> withTs(List values) {
long currentTimestamp = timeProvider.currentTimestamp();
List> withTs = new ArrayList<>(values.size());
for (T value : values) {
Preconditions.checkNotNull(value, "You cannot have null element in a ListState.");
withTs.add(TtlUtils.wrapWithTs(value, currentTimestamp));
}
return withTs;
}
private class IteratorWithCleanup implements Iterator {
private final Iterator> originalIterator;
private boolean anyUnexpired = false;
private boolean uncleared = true;
private T nextUnexpired = null;
private IteratorWithCleanup(Iterator> ttlIterator) {
this.originalIterator = ttlIterator;
}
@Override
public boolean hasNext() {
findNextUnexpired();
cleanupIfEmpty();
return nextUnexpired != null;
}
private void cleanupIfEmpty() {
boolean endOfIter = !originalIterator.hasNext() && nextUnexpired == null;
if (uncleared && !anyUnexpired && endOfIter) {
original.clear();
uncleared = false;
}
}
@Override
public T next() {
if (hasNext()) {
T result = nextUnexpired;
nextUnexpired = null;
return result;
}
throw new NoSuchElementException();
}
private void findNextUnexpired() {
while (nextUnexpired == null && originalIterator.hasNext()) {
TtlValue ttlValue = originalIterator.next();
if (ttlValue == null) {
break;
}
boolean unexpired = !expired(ttlValue);
if (unexpired) {
anyUnexpired = true;
}
if (unexpired || returnExpired) {
nextUnexpired = ttlValue.getUserValue();
}
}
}
}
}