flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-9034) State Descriptors drop TypeInformation on serialization
Date Wed, 21 Mar 2018 20:17:00 GMT

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

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

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

    https://github.com/apache/flink/pull/5732#discussion_r176220108
  
    --- Diff: flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java
---
    @@ -118,17 +115,14 @@ public void testValueStateDescriptorAutoSerializer() throws Exception
{
     	@SuppressWarnings("unchecked")
     	@Test
     	public void testSerializerDuplication() {
    -		TypeSerializer<String> statefulSerializer = mock(TypeSerializer.class);
    -		when(statefulSerializer.duplicate()).thenAnswer(new Answer<TypeSerializer<String>>()
{
    -			@Override
    -			public TypeSerializer<String> answer(InvocationOnMock invocation) throws Throwable
{
    -				return mock(TypeSerializer.class);
    -			}
    -		});
    -
    -		ReduceFunction<String> reducer = mock(ReduceFunction.class);
    -
    -		ReducingStateDescriptor<String> descr = new ReducingStateDescriptor<>("foobar",
reducer, statefulSerializer);
    +		// we need a serializer that actually duplicates for testing (a stateful one)
    --- End diff --
    
    See above


> State Descriptors drop TypeInformation on serialization
> -------------------------------------------------------
>
>                 Key: FLINK-9034
>                 URL: https://issues.apache.org/jira/browse/FLINK-9034
>             Project: Flink
>          Issue Type: Bug
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.5.0, 1.4.2
>            Reporter: Stephan Ewen
>            Assignee: Stephan Ewen
>            Priority: Major
>             Fix For: 1.6.0
>
>
> The following code currently causes problems
> {code}
> public class MyFunction extends RichMapFunction<A, B>  {
>     private final ValueStateDescriptor<MyType> descr = new ValueStateDescriptor<>("state
name", MyType.class);
>     private ValueState<MyType> state;
>     @Override
>     public void open() {
>         state = getRuntimeContext().getValueState(descr);
>     }
> }
> {code}
> The problem is that the state descriptor drops the type information and creates a serializer
before serialization as part of shipping the function in the cluster. To do that, it initializes
the serializer with an empty execution config, making serialization inconsistent.
> This is mainly an artifact from the days when dropping the type information before shipping
was necessary, because the type info was not serializable. It now is, and we can fix that
bug.



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

Mime
View raw message