Gyula Fora created FLINK-37621: ---------------------------------- Summary: EnableAsyncState doesn't seem to do anything on DataStream API and misleading error message Key: FLINK-37621 URL: https://issues.apache.org/jira/browse/FLINK-37621 Project: Flink Issue Type: Bug Components: API / DataStream Affects Versions: 2.0.0 Reporter: Gyula Fora
eventStream .keyBy(e -> e.key) .enableAsyncState() .process(new EventHistoryProcessor(params)) .enableAsyncState(); Leads to: Caused by: java.lang.IllegalStateException: Current operator integrates the async processing logic, thus only supports state v2 APIs. Please use StateDescriptor under 'org.apache.flink.runtime.state.v2'. The error is misleading because the v2 apis are used but the check combines the async enabled check. We need to split the error reporting. Replacing with: eventStream .keyBy(e -> e.key) .transform( "Event History", BasicTypeInfo.LONG_TYPE_INFO, new AsyncKeyedProcessOperator<>(new EventHistoryProcessor(params))); Would fix the problem but that doesn't seem right -- This message was sent by Atlassian Jira (v8.20.10#820010)