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

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

GitHub user bowenli86 opened a pull request:

    https://github.com/apache/flink/pull/5326

    [FLINK-8365] [State Backend] Relax List type in HeapListState and 
HeapKeyedStateBackend

    ## What is the purpose of the change
    
    `stateTable` in `HeapListState` and 
`HeapKeyedStateBackend#createListState()` are both strongly typed to 
`ArrayList` right now. Relaxing that to `List` so we can avoid extra copies in 
some cases.
    
    Well, the copies in `HeapListState#update()` cannot be completely avoided. 
When users pass in an `AbstractList` to `update()` when there's no state 
before, it will break and we have to convert it to an `ArrayList` explicitly
    
    ## Brief change log
    
    Relax List type in HeapListState and HeapKeyedStateBackend
    
    ## Verifying this change
    
    This change is already covered by existing tests, such as 
*MemoryStateBackendTest*.
    
    ## Does this pull request potentially affect one of the following parts:
    
    none
    
    ## Documentation
    
    none

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/bowenli86/flink FLINK-8365

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/5326.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #5326
    
----
commit dd2ec2fb3ce53549ba2f0162591658db34f1ac13
Author: Bowen Li <bowenli86@...>
Date:   2018-01-21T05:29:38Z

    [FLINK-8365] Relax List type in HeapListState and HeapKeyedStateBackend

commit f9d648fe6d26129a5a126cc860ccc0a58484a193
Author: Bowen Li <bowenli86@...>
Date:   2018-01-21T05:31:33Z

    remove unused code

----


> Relax List type in HeapListState and HeapKeyedStateBackend
> ----------------------------------------------------------
>
>                 Key: FLINK-8365
>                 URL: https://issues.apache.org/jira/browse/FLINK-8365
>             Project: Flink
>          Issue Type: Improvement
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.5.0
>            Reporter: Bowen Li
>            Assignee: Bowen Li
>            Priority: Major
>             Fix For: 1.5.0
>
>
> {{stateTable}} in HeapListState and 
> {{HeapKeyedStateBackend#createListState()}} are both strongly typed to 
> {{ArrayList}} right now.
> As discussed with [~StephanEwen] and [[email protected]] in 
> https://github.com/apache/flink/pull/4963, we may want to relax the type to 
> {{List}}.
> Problems discovered now:
> 1. That may require changing serializer from {{ArrayListSerializer}} to 
> {{ListSerializer}} in the following code, and we need to discuss the pros and 
> cons
> {code:java}
> @Override
>       public <N, T> InternalListState<N, T> createListState(
>                       TypeSerializer<N> namespaceSerializer,
>                       ListStateDescriptor<T> stateDesc) throws Exception {
>               // the list state does some manual mapping, because the state 
> is typed to the generic
>               // 'List' interface, but we want to use an implementation typed 
> to ArrayList
>               // using a more specialized implementation opens up runtime 
> optimizations
>               StateTable<K, N, ArrayList<T>> stateTable = 
> tryRegisterStateTable(
>                               stateDesc.getName(),
>                               stateDesc.getType(),
>                               namespaceSerializer,
>                               new 
> ArrayListSerializer<T>(stateDesc.getElementSerializer()));
>               return new HeapListState<>(stateDesc, stateTable, 
> keySerializer, namespaceSerializer);
>       }
> {code}
> 2. for non-RocksDBStateBackend (AsyncFileStateBackendTest, 
> AsyncMemoryStateBackendTest, FileStateBackendTest, and 
> MemoryStateBackendTest), unit tests testListState and 
> testListStateAddUpdateAndGet fail



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to