flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Yun Tang <myas...@live.com>
Subject Re: StandAlone job on k8s fails with "Unknown method truncate" on restore
Date Fri, 15 Feb 2019 07:33:02 GMT
Hi

When 'RollingSink' try to initialize state, it would first check current file system supported
truncate method. If file system not supported, it would use another work-around solution,
which means you should not meet the problem. Otherwise 'RollingSink' thought and found the
reflection method of 'truncate' while the file system actually not support. You could try
to open DEBUG level to see whether log below could  be printed:
Truncate not found. Will write a file with suffix '.valid-length' and prefix '_' to specify
how many bytes in a bucket are valid.

However, from your second email, the more serious problem should be using 'Buckets' with Hadoop-2.6.
From what I know the `RecoverableWriter` within 'Buckets' can only support Hadoop-2.7+ , I'm
not sure whether existed work around solution.

Best
Yun Tang
________________________________
From: Vishal Santoshi <vishal.santoshi@gmail.com>
Sent: Friday, February 15, 2019 3:43
To: user
Subject: Re: StandAlone job on k8s fails with "Unknown method truncate" on restore

And yes  cannot work with RollingFleSink for hadoop 2.6 release of 1.7.1 b'coz of this.


java.lang.UnsupportedOperationException: Recoverable writers on Hadoop are only supported
for HDFS and for Hadoop version 2.7 or newer
        at org.apache.flink.runtime.fs.hdfs.HadoopRecoverableWriter.<init>(HadoopRecoverableWriter.java:57)
        at org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.createRecoverableWriter(HadoopFileSystem.java:202)
        at org.apache.flink.core.fs.SafetyNetWrapperFileSystem.createRecoverableWriter(SafetyNetWrapperFileSystem.java:69)
        at org.apache.flink.streaming.api.functions.sink.filesystem.Buckets.<init>(Buckets.java:112)


Any work around ?

On Thu, Feb 14, 2019 at 1:42 PM Vishal Santoshi <vishal.santoshi@gmail.com<mailto:vishal.santoshi@gmail.com>>
wrote:
The job uses a RolllingFileSink to push data to hdfs. Run an HA standalone cluster on k8s,

* get the job running
* kill the pod.

The k8s deployment relaunches the pod but fails with


java.io.IOException: Missing data in tmp file: hdfs://nn-crunchy:8020/tmp/kafka-to-hdfs/ls_kraken_events/dt=2019-02-14/evt=ad_fill/.part-2-16.inprogress.449e8668-e886-4f89-b5f6-45ac68e25987


Unknown method truncate called on org.apache.hadoop.hdfs.protocol.ClientProtocol protocol.


The file does exist. We work with hadoop 2.6 , which does no have truncate. The previous version
would see that "truncate" was not supported and drop a length file for the ,inprogress file
and rename it to a valid part file.



Is this a known issue ?


Regards.




Mime
View raw message