If checkpointing is disabled RocksDB will only store state locally on disk but
it will not be checkpointed to a DFS. This means that in case of failure you
lose state.
> On 5. Jan 2018, at 14:38, Jinhua Luo wrote:
>
> Thanks, and I will read the codes to get more understanding.
>
> Let me rep
Thanks, and I will read the codes to get more understanding.
Let me repeat another question, what happen if the checkpoing is
disabled (by default, as known)? Does the state still saved?
2018-01-04 22:48 GMT+08:00 Aljoscha Krettek :
> TaskManagers don't do any checkpointing but Operators that run
TaskManagers don't do any checkpointing but Operators that run in TaskManagers
do.
Each operator, of which there are multiple running on multiple TMs in the
cluster will write to a unique DFS directory. Something like:
/checkpoints/job-xyz/checkpoint-1/operator-a/1
These individual checkpoints
One task manager would create one rocksdb instance on its local
temporary dir, correct?
Since there is likely multiple task managers for one cluster, so how
they handle directory conflict, because one rocksdb instance is one
directory, that is, what I mentioned at first, how they merge rocksdb
inst
Ah I see. Currently the RocksDB backend will use one column in RocksDB per
state that is registered. The states for different keys of one state are stored
in one column.
> On 4. Jan 2018, at 14:56, Jinhua Luo wrote:
>
> ok, I see.
>
> But as known, one rocksdb instance occupy one directory, s
ok, I see.
But as known, one rocksdb instance occupy one directory, so I am still
wondering what's the relationship between the states and rocksdb
instances.
2018-01-04 21:50 GMT+08:00 Aljoscha Krettek :
> Each operator (which run in a TaskManager) will write its state to some
> location in HDFS
Each operator (which run in a TaskManager) will write its state to some
location in HDFS (or any other DFS) and send a handle to this to the
CheckpointCoordinator (which is running on the JobManager). The
CheckpointCoordinator is collecting all the handles and creating one
Uber-Handle, which de
OK, I think I get the point.
But another question raises: how task managers merge their rocksdb
snapshot on a global single path?
2018-01-04 21:30 GMT+08:00 Aljoscha Krettek :
> Hi,
>
> The path you give to the constructor must be a path on some distributed
> filesystem, otherwise the data will
Hi,
The path you give to the constructor must be a path on some distributed
filesystem, otherwise the data will be lost when the local machine crashes. As
you mentioned correctly.
RocksDB will keep files in a local directory (you can specify this using
setDbStoragePath()) and when checkpointin
I still do not understand the relationship between rocksdb backend and
the filesystem (here I refer to any filesystem impl, including local,
hdfs, s3).
For example, when I specify the path to rocksdb backend:
env.setStateBackend(new RocksDBStateBackend("file:///data1/flinkapp"));
What does it mea
Hi Jinhua,
I will try to answer your questions:
Flink checkpoints the state of each operator. For a Kafka consumer
operator this is only the offset. For other operators (such as Windows
or a ProcessFunction) the values/list/maps stored in the state are
checkpointed. If you are interested in t
Hi,
> I have two questions:
>
> a) does the records/elements themselves would be checkpointed? or just
> record offset checkpointed? That is, what data included in the
> checkpoint except for states?
No, just offsets (or something similar, depending on the source), which are
part of the state o
Hi All,
I have two questions:
a) does the records/elements themselves would be checkpointed? or just
record offset checkpointed? That is, what data included in the
checkpoint except for states?
b) where flink stores the state globally? so that the job manager
could restore them on each task mang
13 matches
Mail list logo