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

ASF GitHub Bot commented on FLINK-2991:
---------------------------------------

Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1605#discussion_r52445522
  
    --- Diff: 
flink-core/src/main/java/org/apache/flink/api/common/state/FoldingStateDescriptor.java
 ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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.api.common.state;
    +
    +import org.apache.flink.api.common.functions.FoldFunction;
    +import org.apache.flink.api.common.functions.RichFunction;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * {@link StateDescriptor} for {@link ReducingState}. This can be used to 
create partitioned
    + * folding state.
    + *
    + * @param <T> Type of the values folded int othe state
    + * @param <ACC> Type of the value in the state
    + */
    +public class FoldingStateDescriptor<T, ACC> extends 
StateDescriptor<FoldingState<T, ACC>, ACC> {
    +   private static final long serialVersionUID = 1L;
    +
    +
    +   private final FoldFunction<T, ACC> foldFunction;
    +
    +   /**
    +    * Creates a new {@code FoldingStateDescriptor} with the given name, 
type, and initial value.
    +    *
    +    * <p>If this constructor fails (because it is not possible to describe 
the type via a class),
    +    * consider using the {@link #FoldingStateDescriptor(String, ACC, 
FoldFunction, TypeInformation)} constructor.
    +    *
    +    * @param name The (unique) name for the state.
    +    * @param initialValue The initial value of the fold.
    +    * @param foldFunction The {@code FoldFunction} used to aggregate the 
state.
    +    * @param typeClass The type of the values in the state.
    +    */
    +   public FoldingStateDescriptor(String name, ACC initialValue, 
FoldFunction<T, ACC> foldFunction, Class<ACC> typeClass) {
    +           super(name, typeClass, initialValue);
    +           this.foldFunction = requireNonNull(foldFunction);
    +
    +           if (foldFunction instanceof RichFunction) {
    +                   throw new UnsupportedOperationException("FoldFunction 
of FoldingState can not be a RichFunction.");
    +           }
    +   }
    +
    +   /**
    +    * Creates a new {@code ReducingStateDescriptor} with the given name 
and default value.
    --- End diff --
    
    FoldingStateDescriptor


> Extend Window Operators to Allow Efficient Fold Operation
> ---------------------------------------------------------
>
>                 Key: FLINK-2991
>                 URL: https://issues.apache.org/jira/browse/FLINK-2991
>             Project: Flink
>          Issue Type: Improvement
>          Components: Streaming
>            Reporter: Aljoscha Krettek
>            Assignee: Aljoscha Krettek
>
> Right now, a window fold is implemented as a WindowFunction that gets all the 
> elements as input. No pre-aggregation is performed. The window operator 
> should be extended to also allow the fold to also be pre-aggregated.
> This requires changing the signature of the {{WindowBuffer}} so that it can 
> emit a type other than the input type. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to