[
https://issues.apache.org/jira/browse/FLINK-9514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16520498#comment-16520498
]
ASF GitHub Bot commented on FLINK-9514:
---------------------------------------
Github user StefanRRichter commented on a diff in the pull request:
https://github.com/apache/flink/pull/6186#discussion_r197482482
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/AbstractTtlDecorator.java
---
@@ -0,0 +1,99 @@
+/*
+ * 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.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+import org.apache.flink.util.function.ThrowingConsumer;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+/**
+ * Base class for TTL logic wrappers.
+ *
+ * @param <T> Type of originally wrapped object
+ */
+abstract class AbstractTtlDecorator<T> {
+ final T original;
+ final TtlConfig config;
+ final TtlTimeProvider timeProvider;
+ final boolean updateTsOnRead;
+ final boolean returnExpired;
+ final long ttl;
+
+ AbstractTtlDecorator(
+ T original,
+ TtlConfig config,
+ TtlTimeProvider timeProvider) {
+ Preconditions.checkNotNull(original);
+ Preconditions.checkNotNull(config);
+ Preconditions.checkNotNull(timeProvider);
+ Preconditions.checkArgument(config.getTtlUpdateType() !=
TtlUpdateType.Disabled,
+ "State does not need to be wrapped with TTL if it is
configured as disabled.");
+ this.original = original;
+ this.config = config;
+ this.timeProvider = timeProvider;
+ this.updateTsOnRead = config.getTtlUpdateType() ==
TtlUpdateType.OnReadAndWrite;
+ this.returnExpired = config.getStateVisibility() ==
TtlStateVisibility.Relaxed;
+ this.ttl = config.getTtl().toMilliseconds();
+ }
+
+ <V> V getUnexpried(TtlValue<V> ttlValue) {
+ return ttlValue == null || (expired(ttlValue) &&
!returnExpired) ? null : ttlValue.getUserValue();
+ }
+
+ <V> boolean expired(TtlValue<V> ttlValue) {
+ return ttlValue != null && ttlValue.getExpirationTimestamp() <=
timeProvider.currentTimestamp();
+ }
+
+ <V> TtlValue<V> wrapWithTs(V value) {
+ return wrapWithTs(value, newExpirationTimestamp());
+ }
+
+ static <V> TtlValue<V> wrapWithTs(V value, long ts) {
+ return value == null ? null : new TtlValue<>(value, ts);
+ }
+
+ <V> TtlValue<V> rewrapWithNewTs(TtlValue<V> ttlValue) {
+ return wrapWithTs(ttlValue.getUserValue());
+ }
+
+ private long newExpirationTimestamp() {
+ long currentTs = timeProvider.currentTimestamp();
+ long ttlWithoutOverflow = currentTs > 0 ?
Math.min(Long.MAX_VALUE - currentTs, ttl) : ttl;
+ return currentTs + ttlWithoutOverflow;
+ }
+
+ <SE extends Throwable, CE extends Throwable, CLE extends Throwable, V>
V getWithTtlCheckAndUpdate(
--- End diff --
Seems also like this method almost fits better into `AbstractTtlState`, for
example you can access `clear()` directly.
> 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
> Labels: pull-request-available
> 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)