Hi Andrea

This should be a bug already fixed by 
https://issues.apache.org/jira/browse/FLINK-12042 , you could upgrade to at 
least 1.7.3 version to avoid this bug.

Best
Yun Tang
________________________________
From: Andrea Spina <andrea.sp...@radicalbit.io>
Sent: Wednesday, July 3, 2019 15:46
To: dev@flink.apache.org
Subject: Re: Flink 1.6.4 Issue on RocksDB incremental checkpoints and 
fs.default-scheme

Hi, I attached also the JM log. Thereby you can appreciate the exception. I 
hope that can help.
As I said previously, disabling fs.default-scheme property solved my issue.

cheers,

Il giorno lun 1 lug 2019 alle ore 18:17 Yun Tang 
<myas...@live.com<mailto:myas...@live.com>> ha scritto:
Hi Andrea

Unfortunately, the tm log provided is not the task manager in which 
RocksDBStateBackend first failed. All tasks on this task manager are actually 
canceled by job manager, you could find a lot of "Attempting to cancel task" 
before any task failed.

>From your latest description, this problem happened without any relationship 
>to fs.default-schema. And actually I wonder the previous error "Could not 
>materialize checkpoint 1 for operator Service Join SuperService (6/8)" was 
>whether the root cause of your job's first failover, it might be caused by 
>other task failure and then cancelled via JM leading to that directory cleaned 
>up.

I think you could search your job manager's log to find the first failed task 
exception and locate which task manager that task run. That task manager would 
contain useful messages. If possible, please provide your job manager's log.

Best
Yun Tang
________________________________
From: Andrea Spina 
<andrea.sp...@radicalbit.io<mailto:andrea.sp...@radicalbit.io>>
Sent: Monday, July 1, 2019 23:14
To: dev@flink.apache.org<mailto:dev@flink.apache.org>
Subject: Re: Flink 1.6.4 Issue on RocksDB incremental checkpoints and 
fs.default-scheme

Hi Yun,
rocksDB configuration is set as follows:
```
RocksDB write-buffer size: 512MB
RocksDB BlockSize (cache) [B/K/M]: 128MB
Checkpoints directory: hdfs:///address-to-hdfs-chkp-dir:8020/flink/checkpoints
enable Checkpoints: true
Rocksdb cache index and filters true
RocksDB thread No.: 4
Checkpoints interval: 60000
RocksDB BlockSize [B/K/M]: 16KB
RocksDB write-buffer count: 5
Use incremental checkpoints: true
Rocksdb optimize hits: true
RocksDB write-buffer number to merge: 2
```

I use RocksDBStateBackend class, but I recorded the same result by using 
configuration parameter state.backend.incremental: true.

Il giorno lun 1 lug 2019 alle ore 14:41 Yun Tang 
<myas...@live.com<mailto:myas...@live.com><mailto:myas...@live.com<mailto:myas...@live.com>>>
 ha scritto:
Hi Andrea

The error happens when Flink try to verify whether your local backup directory 
existed[1]. If you could reproduce this, would you please share your 
configuration to RocksDBStateBackend, and what `fs.default-scheme` have you 
configured. Taskmanager log with more details is also very welcome.


[1] 
https://github.com/apache/flink/blob/6f4148180ba372a2c12c1d54bea8579350af6c98/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java#L2568

Best
Yun Tang
________________________________
From: Andrea Spina 
<andrea.sp...@radicalbit.io<mailto:andrea.sp...@radicalbit.io><mailto:andrea.sp...@radicalbit.io<mailto:andrea.sp...@radicalbit.io>>>
Sent: Monday, July 1, 2019 20:06
To: 
dev@flink.apache.org<mailto:dev@flink.apache.org><mailto:dev@flink.apache.org<mailto:dev@flink.apache.org>>
Subject: Fwd: Flink 1.6.4 Issue on RocksDB incremental checkpoints and 
fs.default-scheme

Dear community, I am running through the following issue. whenever I use
rocksdb as state backend along with incremental checkpoints, I get the
following error:
















*Caused by: java.lang.Exception: Could not materialize checkpoint 1 for
operator Service Join SuperService (6/8).        at
org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.handleExecutionException(StreamTask.java:942)
      ... 6 moreCaused by: java.util.concurrent.ExecutionException:
java.lang.IllegalStateException        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:53)
    at
org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:47)
      at
org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:853)
      ... 5 moreCaused by: java.lang.IllegalStateException        at
org.apache.flink.util.Preconditions.checkState(Preconditions.java:179)
  at
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBIncrementalSnapshotOperation.runSnapshot(RocksDBKeyedStateBackend.java:2568)
      at java.util.concurrent.FutureTask.run(FutureTask.java:266)        at
org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:50)
    ... 7 more*

In my case, I am able to use incremental checkopints with rocksdb as long
as I disable *fs.default-scheme* property; in any other case, I get the
above error. Is this a known issue?

Hope this can help,
--
*Andrea Spina*
Head of R&D @ Radicalbit Srl
Via Giovanni Battista Pirelli 11, 20124, Milano - IT


--
Andrea Spina
Head of R&D @ Radicalbit Srl
Via Giovanni Battista Pirelli 11, 20124, Milano - IT


--
Andrea Spina
Head of R&D @ Radicalbit Srl
Via Giovanni Battista Pirelli 11, 20124, Milano - IT

Reply via email to