Hi Timo,

Should not we delegate the recovery option to the user?
I think we can ask the user to provide Reader to respective Writer
class and save valid-length info in operator state apart from the
current flow. According to user chosen recovery option, we can stream
the Reader output to Writer class till the valid-length limit.

We see there would be an issue in eventual consistent storage like S3
as its consumer can read stale data on the intermittent basis or
duplicate data if the correct filename is not chosen.
--
Thanks,
Amit

On Tue, May 15, 2018 at 5:00 PM, Timo Walther <twal...@apache.org> wrote:
> I guess writing a new file would take much longer than just using the
> .valid-length file, especially if the files are very large. The restoring
> time should be as minimal as possible to ensure little downtime on restarts.
>
> Regards,
> Timo
>
>
> Am 15.05.18 um 09:31 schrieb Gary Yao:
>
>> Hi,
>>
>> The BucketingSink truncates the file if the Hadoop FileSystem supports
>> this
>> operation (Hadoop 2.7 and above) [1]. What version of Hadoop are you
>> using?
>>
>> Best,
>> Gary
>>
>> [1]
>>
>> https://github.com/apache/flink/blob/bcd028d75b0e5c5c691e24640a2196b2fdaf85e0/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java#L301
>>
>> On Mon, May 14, 2018 at 1:37 PM, 张馨予 <342689...@qq.com> wrote:
>>
>>> Hi
>>>
>>>
>>> I'm trying to copy data from kafka to HDFS . The data in HDFS is used to
>>> do other computations by others in map/reduce.
>>> If some tasks failed, the ".valid-length" file is created for the low
>>> version hadoop. The problem is other people must know how to deal with
>>> the
>>> ".valid-length" file, otherwise, the data may be not exactly-once.
>>> Hence, why not rewrite a new file when restoring instead of writing a
>>> ".valid-length" file. In this way, others who use the data in HDFS don't
>>> need to know how to deal with the ".valid-length" file.
>>>
>>>
>>> Thanks!
>>>
>>>
>>> Zhang Xinyu
>
>
>

Reply via email to