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-5289) NPE when using value state on non-keyed stream
Date Fri, 09 Dec 2016 11:52:59 GMT

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

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

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

    https://github.com/apache/flink/pull/2969#discussion_r91703632
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
---
    @@ -106,17 +109,45 @@ public boolean hasBroadcastVariable(String name) {
     
     	@Override
     	public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties)
{
    -		return operator.getKeyedStateStore().getState(stateProperties);
    +		KeyedStateStore keyedStateStore = checkPreconditionsAndGetKeyedStateStore(stateProperties);
    +
    +		try {
    +			stateProperties.initializeSerializerUnlessSet(getExecutionConfig());
    +			return keyedStateStore.getState(stateProperties);
    +		} catch (Exception e) {
    +			throw new RuntimeException("Error while getting state", e);
    +		}
     	}
     
     	@Override
     	public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties)
{
    -		return operator.getKeyedStateStore().getListState(stateProperties);
    +		KeyedStateStore keyedStateStore = checkPreconditionsAndGetKeyedStateStore(stateProperties);
    +
    +		try {
    +			stateProperties.initializeSerializerUnlessSet(getExecutionConfig());
    +			return keyedStateStore.getListState(stateProperties);
    +		} catch (Exception e) {
    +			throw new RuntimeException("Error while getting state", e);
    +		}
     	}
     
     	@Override
     	public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T>
stateProperties) {
    -		return operator.getKeyedStateStore().getReducingState(stateProperties);
    +		KeyedStateStore keyedStateStore = checkPreconditionsAndGetKeyedStateStore(stateProperties);
    +
    +		try {
    +			stateProperties.initializeSerializerUnlessSet(getExecutionConfig());
    +			return keyedStateStore.getReducingState(stateProperties);
    +		} catch (Exception e) {
    +			throw new RuntimeException("Error while getting state", e);
    --- End diff --
    
    Do we need to re-wrap the exceptions here? Can we simply let the original exception bubble
up?


> NPE when using value state on non-keyed stream
> ----------------------------------------------
>
>                 Key: FLINK-5289
>                 URL: https://issues.apache.org/jira/browse/FLINK-5289
>             Project: Flink
>          Issue Type: Bug
>          Components: Streaming
>            Reporter: Timo Walther
>            Assignee: Stefan Richter
>
> Using a {{ValueStateDescriptor}} and {{getRuntimeContext().getState(descriptor)}} on
a non-keyed stream leads to {{NullPointerException}} which is not very helpful for users:
> {code}
> java.lang.NullPointerException
> 	at org.apache.flink.streaming.api.operators.StreamingRuntimeContext.getState(StreamingRuntimeContext.java:109)
> {code}



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

Mime
View raw message