Hi Aljoscha,

thank you for the fast answer. The files in HDFS change as follows:

-before task manager is killed:
[user@host user]$ hdfs dfs -ls -R /hdfs/dir/outbound
-rw-r--r--   2 user hadoop    2435461 2016-03-03 14:51 
/hdfs/dir/outbound/_part-0-0.in-progress
-rw-r--r--   2 user hadoop    2404604 2016-03-03 14:51 
/hdfs/dir/outbound/_part-1-0.in-progress
-rw-r--r--   2 user hadoop    2453796 2016-03-03 14:51 
/hdfs/dir/outbound/_part-10-0.in-progress
-rw-r--r--   2 user hadoop    2447794 2016-03-03 14:51 
/hdfs/dir/outbound/_part-11-0.in-progress
-rw-r--r--   2 user hadoop    2453479 2016-03-03 14:51 
/hdfs/dir/outbound/_part-2-0.in-progress
-rw-r--r--   2 user hadoop    2512413 2016-03-03 14:51 
/hdfs/dir/outbound/_part-3-0.in-progress
-rw-r--r--   2 user hadoop    2431290 2016-03-03 14:51 
/hdfs/dir/outbound/_part-4-0.in-progress
-rw-r--r--   2 user hadoop    2457820 2016-03-03 14:51 
/hdfs/dir/outbound/_part-5-0.in-progress
-rw-r--r--   2 user hadoop    2430218 2016-03-03 14:51 
/hdfs/dir/outbound/_part-6-0.in-progress
-rw-r--r--   2 user hadoop    2482970 2016-03-03 14:51 
/hdfs/dir/outbound/_part-7-0.in-progress
-rw-r--r--   2 user hadoop    2452622 2016-03-03 14:51 
/hdfs/dir/outbound/_part-8-0.in-progress
-rw-r--r--   2 user hadoop    2467733 2016-03-03 14:51 
/hdfs/dir/outbound/_part-9-0.in-progress

-shortly after task manager is killed:
[user@host user]$ hdfs dfs -ls -R /hdfs/dir/outbound
-rw-r--r--   2 user hadoop   12360198 2016-03-03 14:52 
/hdfs/dir/outbound/_part-0-0.pending
-rw-r--r--   2 user hadoop    9350654 2016-03-03 14:52 
/hdfs/dir/outbound/_part-1-0.in-progress
-rw-r--r--   2 user hadoop    9389872 2016-03-03 14:52 
/hdfs/dir/outbound/_part-10-0.in-progress
-rw-r--r--   2 user hadoop   12236015 2016-03-03 14:52 
/hdfs/dir/outbound/_part-11-0.pending
-rw-r--r--   2 user hadoop   12256483 2016-03-03 14:52 
/hdfs/dir/outbound/_part-2-0.pending
-rw-r--r--   2 user hadoop   12261730 2016-03-03 14:52 
/hdfs/dir/outbound/_part-3-0.pending
-rw-r--r--   2 user hadoop    9418913 2016-03-03 14:52 
/hdfs/dir/outbound/_part-4-0.in-progress
-rw-r--r--   2 user hadoop   12176987 2016-03-03 14:52 
/hdfs/dir/outbound/_part-5-0.pending
-rw-r--r--   2 user hadoop   12165782 2016-03-03 14:52 
/hdfs/dir/outbound/_part-6-0.pending
-rw-r--r--   2 user hadoop    9474037 2016-03-03 14:52 
/hdfs/dir/outbound/_part-7-0.in-progress
-rw-r--r--   2 user hadoop   12136347 2016-03-03 14:52 
/hdfs/dir/outbound/_part-8-0.pending
-rw-r--r--   2 user hadoop   12305943 2016-03-03 14:52 
/hdfs/dir/outbound/_part-9-0.pending

-still a bit later:
[user@host user]$ hdfs dfs -ls -R /hdfs/dir/outbound
-rw-r--r--   2 user hadoop          9 2016-03-03 14:52 
/hdfs/dir/outbound/_part-0-0.valid-length
-rw-r--r--   2 user hadoop    8026667 2016-03-03 14:52 
/hdfs/dir/outbound/_part-0-1.pending
-rw-r--r--   2 user hadoop          9 2016-03-03 14:52 
/hdfs/dir/outbound/_part-1-0.valid-length
-rw-r--r--   2 user hadoop    8097752 2016-03-03 14:52 
/hdfs/dir/outbound/_part-1-1.pending
-rw-r--r--   2 user hadoop          9 2016-03-03 14:52 
/hdfs/dir/outbound/_part-10-0.valid-length
-rw-r--r--   2 user hadoop    7109259 2016-03-03 14:52 
/hdfs/dir/outbound/_part-10-1.pending
-rw-r--r--   2 user hadoop          0 2016-03-03 14:52 
/hdfs/dir/outbound/_part-2-0.valid-length
-rw-r--r--   2 user hadoop          9 2016-03-03 14:52 
/hdfs/dir/outbound/_part-3-0.valid-length
-rw-r--r--   2 user hadoop    7974655 2016-03-03 14:52 
/hdfs/dir/outbound/_part-3-1.pending
-rw-r--r--   2 user hadoop          9 2016-03-03 14:52 
/hdfs/dir/outbound/_part-4-0.valid-length
-rw-r--r--   2 user hadoop    5753163 2016-03-03 14:52 
/hdfs/dir/outbound/_part-4-1.pending
-rw-r--r--   2 user hadoop          0 2016-03-03 14:52 
/hdfs/dir/outbound/_part-5-0.valid-length
-rw-r--r--   2 user hadoop          9 2016-03-03 14:52 
/hdfs/dir/outbound/_part-6-0.valid-length
-rw-r--r--   2 user hadoop    7904468 2016-03-03 14:52 
/hdfs/dir/outbound/_part-6-1.pending
-rw-r--r--   2 user hadoop          9 2016-03-03 14:52 
/hdfs/dir/outbound/_part-7-0.valid-length
-rw-r--r--   2 user hadoop    7477004 2016-03-03 14:52 
/hdfs/dir/outbound/_part-7-1.pending
-rw-r--r--   2 user hadoop          0 2016-03-03 14:52 
/hdfs/dir/outbound/_part-8-0.valid-length
-rw-r--r--   2 user hadoop          9 2016-03-03 14:52 
/hdfs/dir/outbound/_part-9-0.valid-length
-rw-r--r--   2 user hadoop    7979307 2016-03-03 14:52 
/hdfs/dir/outbound/_part-9-1.pending
-rw-r--r--   2 user hadoop   12360198 2016-03-03 14:52 
/hdfs/dir/outbound/part-0-0
-rw-r--r--   2 user hadoop    9350654 2016-03-03 14:52 
/hdfs/dir/outbound/part-1-0
-rw-r--r--   2 user hadoop    9389872 2016-03-03 14:52 
/hdfs/dir/outbound/part-10-0
-rw-r--r--   2 user hadoop   12256483 2016-03-03 14:52 
/hdfs/dir/outbound/part-2-0
-rw-r--r--   2 user hadoop   12261730 2016-03-03 14:52 
/hdfs/dir/outbound/part-3-0
-rw-r--r--   2 user hadoop    9418913 2016-03-03 14:52 
/hdfs/dir/outbound/part-4-0
-rw-r--r--   2 user hadoop   12176987 2016-03-03 14:52 
/hdfs/dir/outbound/part-5-0
-rw-r--r--   2 user hadoop   12165782 2016-03-03 14:52 
/hdfs/dir/outbound/part-6-0
-rw-r--r--   2 user hadoop    9474037 2016-03-03 14:52 
/hdfs/dir/outbound/part-7-0
-rw-r--r--   2 user hadoop   12136347 2016-03-03 14:52 
/hdfs/dir/outbound/part-8-0
-rw-r--r--   2 user hadoop   12305943 2016-03-03 14:52 
/hdfs/dir/outbound/part-9-0

Can you see from this what is going wrong?

Cheers,
 Max
—
Maximilian Bode * Junior Consultant * [email protected]
TNG Technology Consulting GmbH, Betastr. 13a, 85774 Unterföhring
Geschäftsführer: Henrik Klagges, Christoph Stock, Dr. Robert Dahlke
Sitz: Unterföhring * Amtsgericht München * HRB 135082

> Am 03.03.2016 um 14:50 schrieb Aljoscha Krettek <[email protected]>:
> 
> Hi,
> did you check whether there are any files at your specified HDFS output 
> location? If yes, which files are there?
> 
> Cheers,
> Aljoscha
>> On 03 Mar 2016, at 14:29, Maximilian Bode <[email protected]> 
>> wrote:
>> 
>> Just for the sake of completeness: this also happens when killing a task 
>> manager and is therefore probably unrelated to job manager HA.
>> 
>>> Am 03.03.2016 um 14:17 schrieb Maximilian Bode 
>>> <[email protected]>:
>>> 
>>> Hi everyone,
>>> 
>>> unfortunately, I am running into another problem trying to establish 
>>> exactly once guarantees (Kafka -> Flink 1.0.0-rc3 -> HDFS).
>>> 
>>> When using
>>> 
>>> RollingSink<Tuple3<Integer,Integer,String>> sink = new 
>>> RollingSink<Tuple3<Integer,Integer,String>>("hdfs://our.machine.com:8020/hdfs/dir/outbound");
>>> sink.setBucketer(new NonRollingBucketer());
>>> output.addSink(sink);
>>> 
>>> and then killing the job manager, the new job manager is unable to restore 
>>> the old state throwing
>>> ---
>>> java.lang.Exception: Could not restore checkpointed state to operators and 
>>> functions
>>>     at 
>>> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreState(StreamTask.java:454)
>>>     at 
>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:209)
>>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:559)
>>>     at java.lang.Thread.run(Thread.java:744)
>>> Caused by: java.lang.Exception: Failed to restore state to function: 
>>> In-Progress file hdfs://our.machine.com:8020/hdfs/dir/outbound/part-5-0 was 
>>> neither moved to pending nor is still in progress.
>>>     at 
>>> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreState(AbstractUdfStreamOperator.java:168)
>>>     at 
>>> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreState(StreamTask.java:446)
>>>     ... 3 more
>>> Caused by: java.lang.RuntimeException: In-Progress file 
>>> hdfs://our.machine.com:8020/hdfs/dir/outbound/part-5-0 was neither moved to 
>>> pending nor is still in progress.
>>>     at 
>>> org.apache.flink.streaming.connectors.fs.RollingSink.restoreState(RollingSink.java:686)
>>>     at 
>>> org.apache.flink.streaming.connectors.fs.RollingSink.restoreState(RollingSink.java:122)
>>>     at 
>>> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreState(AbstractUdfStreamOperator.java:165)
>>>     ... 4 more
>>> ---
>>> I found a resolved issue [1] concerning Hadoop 2.7.1. We are in fact using 
>>> 2.4.0 – might this be the same issue?
>>> 
>>> Another thing I could think of is that the job is not configured correctly 
>>> and there is some sort of timing issue. The checkpoint interval is 10 
>>> seconds, everything else was left at default value. Then again, as the 
>>> NonRollingBucketer is used, there should not be any timing issues, right?
>>> 
>>> Cheers,
>>> Max
>>> 
>>> [1] https://issues.apache.org/jira/browse/FLINK-2979
>>> 
>>> —
>>> Maximilian Bode * Junior Consultant * [email protected]
>>> TNG Technology Consulting GmbH, Betastr. 13a, 85774 Unterföhring
>>> Geschäftsführer: Henrik Klagges, Christoph Stock, Dr. Robert Dahlke
>>> Sitz: Unterföhring * Amtsgericht München * HRB 135082
>>> 
>> 
> 

Attachment: signature.asc
Description: Message signed with OpenPGP using GPGMail

Reply via email to