Zakelly commented on code in PR #25515:
URL: https://github.com/apache/flink/pull/25515#discussion_r1804486595


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/AbstractAggregatingState.java:
##########
@@ -59,7 +59,7 @@ public AbstractAggregatingState(
     }
 
     protected StateFuture<ACC> asyncGetAccumulator() {

Review Comment:
   How about remove this method?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/ttl/TtlAggregatingStateV2.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.v2.ttl;
+
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.runtime.state.ttl.TtlStateContext;
+import org.apache.flink.runtime.state.ttl.TtlValue;
+import org.apache.flink.runtime.state.v2.internal.InternalAggregatingState;
+
+import java.util.Collection;
+
+/**
+ * This class wraps aggregating state with TTL logic.
+ *
+ * @param <K> The type of key the state is associated to
+ * @param <N> The type of the namespace
+ * @param <IN> Type of the value added to the state
+ * @param <ACC> The type of the accumulator (intermediate aggregate state).
+ * @param <OUT> Type of the value extracted from the state
+ */
+class TtlAggregatingStateV2<K, N, IN, ACC, OUT>
+        extends AbstractTtlStateV2<
+                K, N, ACC, TtlValue<ACC>, InternalAggregatingState<K, N, IN, 
TtlValue<ACC>, OUT>>
+        implements InternalAggregatingState<K, N, IN, ACC, OUT> {
+
+    TtlAggregatingStateV2(
+            TtlStateContext<InternalAggregatingState<K, N, IN, TtlValue<ACC>, 
OUT>, ACC>
+                    ttlStateContext,
+            TtlAggregateFunctionV2<IN, ACC, OUT> aggregateFunction) {
+        super(ttlStateContext);
+        aggregateFunction.updater = (ttlValue) -> 
original.updateInternal(ttlValue);
+    }
+
+    @Override
+    public StateFuture<Void> asyncMergeNamespaces(N target, Collection<N> 
sources) {
+        return original.asyncMergeNamespaces(target, sources);
+    }
+
+    @Override
+    public void mergeNamespaces(N target, Collection<N> sources) {
+        original.mergeNamespaces(target, sources);
+    }
+
+    @Override
+    public StateFuture<OUT> asyncGet() {
+        return original.asyncGet();
+    }
+
+    @Override
+    public StateFuture<Void> asyncAdd(IN value) {
+        return original.asyncAdd(value);
+    }
+
+    @Override
+    public OUT get() {
+        return original.get();
+    }
+
+    @Override
+    public void add(IN value) {
+        original.add(value);
+    }
+
+    @Override
+    public void clear() {
+        original.clear();
+    }
+
+    @Override
+    public StateFuture<ACC> asyncGetInternal() {
+        return original.asyncGetInternal()
+                .thenApply(ttlValue -> getElementWithTtlCheck(ttlValue, 
original::updateInternal));

Review Comment:
   How about update the value asynchronously when `updateOnRead`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/ttl/AbstractTtlStateV2.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.v2.ttl;
+
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.runtime.state.ttl.AbstractTtlDecorator;
+import org.apache.flink.runtime.state.ttl.TtlStateContext;
+import org.apache.flink.runtime.state.v2.internal.InternalKeyedState;
+
+/**
+ * Base class for TTL logic wrappers of state objects. StateV2 does not support
+ * FULL_STATE_SCAN_SNAPSHOT and INCREMENTAL_CLEANUP, only supports 
ROCKSDB_COMPACTION_FILTER.
+ *
+ * @param <K> The type of key the state is associated to
+ * @param <N> The type of the namespace
+ * @param <SV> The type of values kept internally in state without TTL
+ * @param <TTLSV> The type of values kept internally in state with TTL
+ * @param <S> Type of originally wrapped state object
+ */
+abstract class AbstractTtlStateV2<K, N, SV, TTLSV, S extends 
InternalKeyedState<K, N, TTLSV>>

Review Comment:
   I'd suggest remove all the `V2` in class name since the package location 
contains `v2` already



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/ttl/TtlListStateV2.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.v2.ttl;
+
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.state.v2.StateIterator;
+import org.apache.flink.runtime.state.ttl.TtlStateContext;
+import org.apache.flink.runtime.state.ttl.TtlUtils;
+import org.apache.flink.runtime.state.ttl.TtlValue;
+import org.apache.flink.runtime.state.v2.adaptor.CompleteStateIterator;
+import org.apache.flink.runtime.state.v2.internal.InternalListState;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * 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 TtlListStateV2<K, N, T>
+        extends AbstractTtlStateV2<K, N, T, TtlValue<T>, InternalListState<K, 
N, TtlValue<T>>>
+        implements InternalListState<K, N, T> {
+
+    protected TtlListStateV2(
+            TtlStateContext<InternalListState<K, N, TtlValue<T>>, T> 
ttlStateContext) {
+        super(ttlStateContext);
+    }
+
+    @Override
+    public StateFuture<Void> asyncUpdate(List<T> values) {
+        Preconditions.checkNotNull(values, "List of values to add cannot be 
null.");
+        return original.asyncUpdate(withTs(values));
+    }
+
+    @Override
+    public StateFuture<Void> asyncAddAll(List<T> values) {
+        Preconditions.checkNotNull(values, "List of values to add cannot be 
null.");
+        return original.asyncAddAll(withTs(values));
+    }
+
+    @Override
+    public StateFuture<StateIterator<T>> asyncGet() {
+        final List<T> result = new ArrayList<>();
+        AtomicBoolean encounteredNull = new AtomicBoolean(false);
+        AtomicBoolean anyUnexpired = new AtomicBoolean(false);
+        return original.asyncGet()
+                .thenAccept(

Review Comment:
   Same for the iteration interface of `MapState`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/ttl/TtlListStateV2.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.v2.ttl;
+
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.state.v2.StateIterator;
+import org.apache.flink.runtime.state.ttl.TtlStateContext;
+import org.apache.flink.runtime.state.ttl.TtlUtils;
+import org.apache.flink.runtime.state.ttl.TtlValue;
+import org.apache.flink.runtime.state.v2.adaptor.CompleteStateIterator;
+import org.apache.flink.runtime.state.v2.internal.InternalListState;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * 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 TtlListStateV2<K, N, T>
+        extends AbstractTtlStateV2<K, N, T, TtlValue<T>, InternalListState<K, 
N, TtlValue<T>>>
+        implements InternalListState<K, N, T> {
+
+    protected TtlListStateV2(
+            TtlStateContext<InternalListState<K, N, TtlValue<T>>, T> 
ttlStateContext) {
+        super(ttlStateContext);
+    }
+
+    @Override
+    public StateFuture<Void> asyncUpdate(List<T> values) {
+        Preconditions.checkNotNull(values, "List of values to add cannot be 
null.");
+        return original.asyncUpdate(withTs(values));
+    }
+
+    @Override
+    public StateFuture<Void> asyncAddAll(List<T> values) {
+        Preconditions.checkNotNull(values, "List of values to add cannot be 
null.");
+        return original.asyncAddAll(withTs(values));
+    }
+
+    @Override
+    public StateFuture<StateIterator<T>> asyncGet() {
+        final List<T> result = new ArrayList<>();
+        AtomicBoolean encounteredNull = new AtomicBoolean(false);
+        AtomicBoolean anyUnexpired = new AtomicBoolean(false);
+        return original.asyncGet()
+                .thenAccept(

Review Comment:
   How about using a iterator wrapper that cleans up the value during iteration 
instead of collect items in advance?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to