brucewoo created FLINK-8281: ------------------------------- Summary: org.apache.flink.runtime.fs.hdfs.HadoopDataOutputStream cannot be cast to org.apache.flink.core.fs.WrappingProxyCloseable Key: FLINK-8281 URL: https://issues.apache.org/jira/browse/FLINK-8281 Project: Flink Issue Type: Bug Components: Core Affects Versions: 1.4.0 Environment: Linux 4.4.0-104-generic #127-Ubuntu SMP Mon Dec 11 12:16:42 UTC 2017 x86_64 x86_64 x86_64 GNU/Linux
Reporter: brucewoo Priority: Critical org.apache.flink.streaming.runtime.tasks.AsynchronousException: java.lang.Exception: Could not materialize checkpoint 1 for operator window: (TumblingGroupWindow('w$, 'RowTime, 60000.millis)), select: (COUNT(*) AS api_call_count, SUM(bytes) AS total_bytes, SUM(numbers) AS total_numbers, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime) -> select: (CAST(w$end) AS proc_end_time, api_call_count, total_bytes, total_numbers) -> to: Row (1/1). at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:945) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_151] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_151] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[na:1.8.0_151] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[na:1.8.0_151] at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_151] Caused by: java.lang.Exception: Could not materialize checkpoint 1 for operator window: (TumblingGroupWindow('w$, 'RowTime, 60000.millis)), select: (COUNT(*) AS api_call_count, SUM(bytes) AS total_bytes, SUM(numbers) AS total_numbers, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime) -> select: (CAST(w$end) AS proc_end_time, api_call_count, total_bytes, total_numbers) -> to: Row (1/1). at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:946) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] ... 5 common frames omitted Caused by: java.util.concurrent.ExecutionException: java.io.IOException: Could not open output stream for state backend at java.util.concurrent.FutureTask.report(FutureTask.java:122) ~[na:1.8.0_151] at java.util.concurrent.FutureTask.get(FutureTask.java:192) ~[na:1.8.0_151] at org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:894) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] ... 5 common frames omitted Suppressed: java.lang.Exception: Could not properly cancel managed keyed state future. at org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:92) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.cleanup(StreamTask.java:976) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:939) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] ... 5 common frames omitted Caused by: java.util.concurrent.ExecutionException: java.io.IOException: Could not open output stream for state backend at java.util.concurrent.FutureTask.report(FutureTask.java:122) at java.util.concurrent.FutureTask.get(FutureTask.java:192) at org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43) at org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:66) at org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:89) ... 7 common frames omitted Caused by: java.io.IOException: Could not open output stream for state backend at org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.createStream(FsCheckpointStreamFactory.java:371) at org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.flush(FsCheckpointStreamFactory.java:228) at org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.write(FsCheckpointStreamFactory.java:212) at java.io.DataOutputStream.write(DataOutputStream.java:107) at org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil.writeSerializersAndConfigsWithResilience(TypeSerializerSerializationUtil.java:236) at org.apache.flink.runtime.state.KeyedBackendStateMetaInfoSnapshotReaderWriters$KeyedBackendStateMetaInfoWriterV3.writeStateMetaInfo(KeyedBackendStateMetaInfoSnapshotReaderWriters.java:108) at org.apache.flink.runtime.state.KeyedBackendSerializationProxy.write(KeyedBackendSerializationProxy.java:118) at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBFullSnapshotOperation.writeKVStateMetaData(RocksDBKeyedStateBackend.java:635) at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBFullSnapshotOperation.writeDBSnapshot(RocksDBKeyedStateBackend.java:555) at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$3.performOperation(RocksDBKeyedStateBackend.java:466) at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$3.performOperation(RocksDBKeyedStateBackend.java:424) at org.apache.flink.runtime.io.async.AbstractAsyncCallableWithResources.call(AbstractAsyncCallableWithResources.java:75) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:40) at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:894) ... 5 common frames omitted Caused by: java.lang.ClassCastException: org.apache.flink.runtime.fs.hdfs.HadoopDataOutputStream cannot be cast to org.apache.flink.core.fs.WrappingProxyCloseable at org.apache.flink.core.fs.SafetyNetCloseableRegistry$PhantomDelegatingCloseableRef.<init>(SafetyNetCloseableRegistry.java:156) at org.apache.flink.core.fs.SafetyNetCloseableRegistry.doRegister(SafetyNetCloseableRegistry.java:96) at org.apache.flink.core.fs.SafetyNetCloseableRegistry.doRegister(SafetyNetCloseableRegistry.java:1) at org.apache.flink.util.AbstractCloseableRegistry.registerCloseable(AbstractCloseableRegistry.java:79) at org.apache.flink.core.fs.ClosingFSDataOutputStream.wrapSafe(ClosingFSDataOutputStream.java:101) at org.apache.flink.core.fs.SafetyNetWrapperFileSystem.create(SafetyNetWrapperFileSystem.java:122) at org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.createStream(FsCheckpointStreamFactory.java:362) ... 19 common frames omitted Caused by: java.io.IOException: Could not open output stream for state backend at org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.createStream(FsCheckpointStreamFactory.java:371) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.flush(FsCheckpointStreamFactory.java:228) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.write(FsCheckpointStreamFactory.java:212) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at java.io.DataOutputStream.write(DataOutputStream.java:107) ~[na:1.8.0_151] at org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil.writeSerializersAndConfigsWithResilience(TypeSerializerSerializationUtil.java:236) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.runtime.state.KeyedBackendStateMetaInfoSnapshotReaderWriters$KeyedBackendStateMetaInfoWriterV3.writeStateMetaInfo(KeyedBackendStateMetaInfoSnapshotReaderWriters.java:108) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.runtime.state.KeyedBackendSerializationProxy.write(KeyedBackendSerializationProxy.java:118) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBFullSnapshotOperation.writeKVStateMetaData(RocksDBKeyedStateBackend.java:635) ~[flink-dist_2.11-1.4.0.jar:1.4.0] at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBFullSnapshotOperation.writeDBSnapshot(RocksDBKeyedStateBackend.java:555) ~[flink-dist_2.11-1.4.0.jar:1.4.0] at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$3.performOperation(RocksDBKeyedStateBackend.java:466) ~[flink-dist_2.11-1.4.0.jar:1.4.0] at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$3.performOperation(RocksDBKeyedStateBackend.java:424) ~[flink-dist_2.11-1.4.0.jar:1.4.0] at org.apache.flink.runtime.io.async.AbstractAsyncCallableWithResources.call(AbstractAsyncCallableWithResources.java:75) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_151] at org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:40) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] ... 6 common frames omitted Caused by: java.lang.ClassCastException: org.apache.flink.runtime.fs.hdfs.HadoopDataOutputStream cannot be cast to org.apache.flink.core.fs.WrappingProxyCloseable at org.apache.flink.core.fs.SafetyNetCloseableRegistry$PhantomDelegatingCloseableRef.<init>(SafetyNetCloseableRegistry.java:156) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.core.fs.SafetyNetCloseableRegistry.doRegister(SafetyNetCloseableRegistry.java:96) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.core.fs.SafetyNetCloseableRegistry.doRegister(SafetyNetCloseableRegistry.java:1) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.util.AbstractCloseableRegistry.registerCloseable(AbstractCloseableRegistry.java:79) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.core.fs.ClosingFSDataOutputStream.wrapSafe(ClosingFSDataOutputStream.java:101) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.core.fs.SafetyNetWrapperFileSystem.create(SafetyNetWrapperFileSystem.java:122) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] at org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.createStream(FsCheckpointStreamFactory.java:362) ~[flink-connector-cdp_2.11-1.4.0-jar-with-dependencies.jar:na] ... 19 common frames omitted -- This message was sent by Atlassian JIRA (v6.4.14#64029)