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

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

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

    https://github.com/apache/flink/pull/4355#discussion_r135555411
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/ListView.scala
 ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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.table.api.dataview
    +
    +import java.lang.{Iterable => JIterable}
    +import java.util
    +
    +import org.apache.flink.api.common.typeinfo.{TypeInfo, TypeInformation}
    +import org.apache.flink.table.dataview.ListViewTypeInfoFactory
    +
    +/**
    +  * ListView provides List functionality for accumulators used by 
user-defined aggregate functions
    +  * {{AggregateFunction}}.
    +  *
    +  * A ListView can be backed by a Java ArrayList or a state backend, 
depending on the context in
    +  * which the function is used.
    +  *
    +  * At runtime `ListView` will be replaced by a 
[[org.apache.flink.table.dataview.StateListView]]
    +  * if it is backed by a state backend.
    +  *
    +  * Example:
    +  * {{{
    +  *
    +  *  public class MyAccum {
    +  *    public ListView<String> list;
    +  *    public long count;
    +  *  }
    +  *
    +  *  public class MyAgg extends AggregateFunction<Long, MyAccum> {
    +  *
    +  *   @Override
    +  *   public MyAccum createAccumulator() {
    +  *     MyAccum accum = new MyAccum();
    +  *     accum.list = new ListView<>(Types.STRING);
    +  *     accum.count = 0L;
    +  *     return accum;
    +  *   }
    +  *
    +  *   public void accumulate(MyAccum accumulator, String id) {
    +  *     accumulator.list.add(id);
    +  *     ... ...
    +  *     accumulator.get()
    +  *     ... ...
    +  *   }
    +  *
    +  *   @Override
    +  *   public Long getValue(MyAccum accumulator) {
    +  *     accumulator.list.add(id);
    +  *     ... ...
    +  *     accumulator.get()
    +  *     ... ...
    +  *     return accumulator.count;
    +  *   }
    +  * }
    +  *
    +  * }}}
    +  *
    +  * @param elementTypeInfo element type information
    +  * @tparam T element type
    +  */
    +@TypeInfo(classOf[ListViewTypeInfoFactory[_]])
    +class ListView[T](
    +    @transient private[flink] val elementTypeInfo: TypeInformation[T])
    +  extends DataView {
    +
    +  def this() = this(null)
    +
    +  private[flink] var list: util.List[T] = new util.ArrayList[T]()
    --- End diff --
    
    We can refactor the `ListView` constructors as follows:
    
    ```
    class ListView[T] private[flink](
        @transient private[flink] val elementTypeInfo: TypeInformation[T],
        private[flink] val list: util.List[T])
      extends DataView {
    
      def this(elementTypeInfo: TypeInformation[T]) {
        this(elementTypeInfo, new util.ArrayList[T]())
      }
    
      def this() = {
        this(null, new util.ArrayList[T]())
      }
    
      ...
    }
    ```
    
    and call the primary constructor in the `ListSerializer` with `null` for 
the type information.


> Implementation of DataView to support state access for UDAGG
> ------------------------------------------------------------
>
>                 Key: FLINK-7206
>                 URL: https://issues.apache.org/jira/browse/FLINK-7206
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Kaibo Zhou
>            Assignee: Kaibo Zhou
>
> Implementation of MapView and ListView to support state access for UDAGG.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to