[ 
https://issues.apache.org/jira/browse/FLINK-9514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16520566#comment-16520566
 ] 

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_r197501902
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlFoldFunction.java
 ---
    @@ -0,0 +1,43 @@
    +/*
    + * 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.functions.FoldFunction;
    +
    +/**
    + * This class wraps folding function with TTL logic.
    + *
    + * @param <T> Type of the values folded into the state
    + * @param <ACC> Type of the value in the state
    + *
    + * @deprecated use {@link TtlAggregateFunction} instead
    + */
    +@Deprecated
    +class TtlFoldFunction<T, ACC>
    +   extends AbstractTtlDecorator<FoldFunction<T, ACC>>
    +   implements FoldFunction<T, TtlValue<ACC>> {
    +   TtlFoldFunction(FoldFunction<T, ACC> original, TtlConfig config, 
TtlTimeProvider timeProvider) {
    +           super(original, config, timeProvider);
    +   }
    +
    +   @Override
    +   public TtlValue<ACC> fold(TtlValue<ACC> accumulator, T value) throws 
Exception {
    +           return wrapWithTs(original.fold(getUnexpried(accumulator), 
value));
    --- End diff --
    
    I think that (similar to`TtlAggregationFunction`) you need to intercept 
`null` values of accumulator here and replace them by the default value.


> 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)

Reply via email to