Re: Incremental checkpointing performance

2018-03-29 Thread Stephan Ewen
I think what happens is the following:

  - For full checkpoints, Flink iterates asynchronously over the data. That
means the whole checkpoint is a compact asynchronous operation.

  - For incremental checkpoints, RocksDB has to flush the write buffer and
create a new SSTable. That flush is synchronous, but should be very brief.
Then there is an asynchronous materialization of the SSTables that are
different from the previous checkpoint.

Because of that, you see that
  - Full checkpoints have a shorter synchronous duration than incremental
checkpoints
  - For small state, full checkpoints may actually be faster end-to-end
  - For large state, the asynchronous part of incremental checkpoints
should be faster, and with that, the end-to-end duration as well

Stephan

On Fri, Mar 23, 2018 at 5:25 PM, Nico Kruber  wrote:

> Hi Miyuru,
> regarding "state.backend", I was looking at version 1.5 docs and some
> things changed compared to 1.3. The "Asynchronous RocksDB snapshot ..."
> messages only occur with full snapshots, i.e. non-incremental, and I
> verified this for your program as well.
>
> There are some issues with your project though:
> 1) your Flink dependencies should all have the same version
> 2) your source does not acquire the checkpoint lock before emitting
> events (see the docs around the SourceFunction you are implementing)
>
>
> Regarding the checkpoint sizes: you can rely on the web interface
> reporting correct metrics. However, the "average" values may not be too
> much useful for you since you are using a sliding count window and thus
> during ramp-up (until you get your 1 windows of the slide size) you
> will have smaller states than after than. Since you only have 2 keys,
> you will eventually have 2 window states to store and from then on
> stay with this number. So rather look at the "History" column of the web
> interface or into the JobManager log.
>
>
> Regarding the original issue: I was recently made aware of another thing
> which may influence the speed of an incremental snapshot: if done
> incrementally, we need to close and flush RocksDB's sst file so that it
> continues with a new file and we can hardlink and copy a consistent
> snapshot. For full snapshots, we simple iterate over all items to copy.
> Now this close-and-flush may be more costly (hence the higher duration)
> and since this cannot be done asynchronously (as a full snapshot) we
> also may not process as many records.
> -> Therefore, you probably did not run your program long enough to
> create the full set of windows and I'm guessing, you will eventually get
> to the same checkpoint sizes.
>
>
> TLDR; incremental snapshots are worth only (and are designed for...) if
> you have a lot of operator state (not just a few MB!) while only few
> parts are actually changing between checkpoints. In these scenarios, the
> added latency for transferring such a snapshot to the checkpoint store
> over network would cover the additional cost during snapshot creation.
>
>
> Nico
>
>
> On 21/03/18 06:01, Miyuru Dayarathna wrote:
> > Hi,
> >
> > Since we could not observe log messages such as "Asynchronous RocksDB
> > snapshot" in the Flink's log files, we ran the application with Flink
> > 1.3.3 as well. But it also did not print the log message. Hence I am
> > wondering whether we ran Flink's incremental checkpointing in the
> > correct manner. I have attached the complete application with this
> > email. Could you please run this in your setup and let me know whether
> > you get the incremental checkpoint related logs printed in your Flink
> setup?
> >
> > Thanks,
> > Miyuru
> >
> >
> >
> >
> > On Monday, 19 March 2018, 23:33:37 GMT+5:30, Miyuru Dayarathna
> >  wrote:
> >
> >
> > Hi Nico,
> >
> > Thanks for the detailed explanation. The only change I have made in my
> > flink-conf.yaml file is the following.
> >
> > state.backend.fs.checkpointdir: file:///home/ubuntu/tmp-flink-rocksdb
> >
> > The default "state.backend" value is set to filesystem. Removing the
> > env.setStateBackend() method code or changing the "state.backend"
> > property to rocksdb does not change the state backend to RocksDB. I got
> > this verified by looking at the Flink log files. I have mentioned a
> > sample of the log file for your reference.
> >
> > ---
> > carbon-5th:38631/user/taskmanager) as 1ac63dfb481eab3d3165a965084115f3.
> > Current number of registered hosts is 1. Current number of alive task
> > slots is 1.
> > 2018-03-19 23:10:11,606 INFO
> > org.apache.flink.runtime.client.JobClient - Checking
> > and uploading JAR files
> > 2018-03-19 23:10:11,618 INFO
> > org.apache.flink.runtime.jobmanager.JobManager-
> > Submitting job 7c19a14f4e75149ffaa064fac7e2bf29 (Flink application.).
> > 2018-03-19 23:10:11,623 INFO
> > org.apache.flink.runtime.jobmanager.JobManager- Using
> > restart 

Re: Incremental checkpointing performance

2018-03-28 Thread Miyuru Dayarathna
 Hi Nico,

Thanks for the detailed explanation. I corrected the two issues you mentioned 
on my application and I was able to observe the behavior you mentioned with 
Flink 1.4.1. As you said the "Asynchronous RocksDB snapshot ..." message 
appears only for full snapshots. The incremental snapshot version of the 
application does not print that message. It should be noted that these logs 
appear only on the taskmanager log file.

Based on the description you provided on the checkpoint sizes, I think it is 
better to lookup the JobManager log file for the entries such as "Completed 
checkpoint 25 (290061 bytes in 60 ms)" and then start calculating the average 
latencies for taking a checkpoint after the window has been completely filled 
(i.e., after the application reaches the steady state).

Thanks for the explanation on the reason why it takes more time to take an 
incremental checkpoint compared to taking a full checkpoint in this setup. As 
you said there should be considerable overhead of flushing the data and closing 
the RocksDB sst file. Hence, repeatedly doing this with few MBs of state should 
have resulted in larger average elapsed time for taking incremental snapshots 
compared to full snapshots. Also As you said, I will double check whether the 
full windows were created before I measure the checkpoint sizes and checkpoint 
durations.

Thanks,
Miyuru

On Friday, 23 March 2018, 21:56:02 GMT+5:30, Nico Kruber 
 wrote:  
 
 Hi Miyuru,
regarding "state.backend", I was looking at version 1.5 docs and some
things changed compared to 1.3. The "Asynchronous RocksDB snapshot ..."
messages only occur with full snapshots, i.e. non-incremental, and I
verified this for your program as well.

There are some issues with your project though:
1) your Flink dependencies should all have the same version
2) your source does not acquire the checkpoint lock before emitting
events (see the docs around the SourceFunction you are implementing)


Regarding the checkpoint sizes: you can rely on the web interface
reporting correct metrics. However, the "average" values may not be too
much useful for you since you are using a sliding count window and thus
during ramp-up (until you get your 1 windows of the slide size) you
will have smaller states than after than. Since you only have 2 keys,
you will eventually have 2 window states to store and from then on
stay with this number. So rather look at the "History" column of the web
interface or into the JobManager log.


Regarding the original issue: I was recently made aware of another thing
which may influence the speed of an incremental snapshot: if done
incrementally, we need to close and flush RocksDB's sst file so that it
continues with a new file and we can hardlink and copy a consistent
snapshot. For full snapshots, we simple iterate over all items to copy.
Now this close-and-flush may be more costly (hence the higher duration)
and since this cannot be done asynchronously (as a full snapshot) we
also may not process as many records.
-> Therefore, you probably did not run your program long enough to
create the full set of windows and I'm guessing, you will eventually get
to the same checkpoint sizes.


TLDR; incremental snapshots are worth only (and are designed for...) if
you have a lot of operator state (not just a few MB!) while only few
parts are actually changing between checkpoints. In these scenarios, the
added latency for transferring such a snapshot to the checkpoint store
over network would cover the additional cost during snapshot creation.


Nico


On 21/03/18 06:01, Miyuru Dayarathna wrote:
> Hi,
> 
> Since we could not observe log messages such as "Asynchronous RocksDB
> snapshot" in the Flink's log files, we ran the application with Flink
> 1.3.3 as well. But it also did not print the log message. Hence I am
> wondering whether we ran Flink's incremental checkpointing in the
> correct manner. I have attached the complete application with this
> email. Could you please run this in your setup and let me know whether
> you get the incremental checkpoint related logs printed in your Flink setup?
> 
> Thanks,
> Miyuru
> 
> 
> 
> 
> On Monday, 19 March 2018, 23:33:37 GMT+5:30, Miyuru Dayarathna
>  wrote:
> 
> 
> Hi Nico,
> 
> Thanks for the detailed explanation. The only change I have made in my
> flink-conf.yaml file is the following.
> 
> state.backend.fs.checkpointdir: file:///home/ubuntu/tmp-flink-rocksdb
> 
> The default "state.backend" value is set to filesystem. Removing the
> env.setStateBackend() method code or changing the "state.backend"
> property to rocksdb does not change the state backend to RocksDB. I got
> this verified by looking at the Flink log files. I have mentioned a
> sample of the log file for your reference.
> 
> ---
> carbon-5th:38631/user/taskmanager) as 1ac63dfb481eab3d3165a965084115f3.
> Current number of registered hosts 

Re: Incremental checkpointing performance

2018-03-23 Thread Nico Kruber
Hi Miyuru,
regarding "state.backend", I was looking at version 1.5 docs and some
things changed compared to 1.3. The "Asynchronous RocksDB snapshot ..."
messages only occur with full snapshots, i.e. non-incremental, and I
verified this for your program as well.

There are some issues with your project though:
1) your Flink dependencies should all have the same version
2) your source does not acquire the checkpoint lock before emitting
events (see the docs around the SourceFunction you are implementing)


Regarding the checkpoint sizes: you can rely on the web interface
reporting correct metrics. However, the "average" values may not be too
much useful for you since you are using a sliding count window and thus
during ramp-up (until you get your 1 windows of the slide size) you
will have smaller states than after than. Since you only have 2 keys,
you will eventually have 2 window states to store and from then on
stay with this number. So rather look at the "History" column of the web
interface or into the JobManager log.


Regarding the original issue: I was recently made aware of another thing
which may influence the speed of an incremental snapshot: if done
incrementally, we need to close and flush RocksDB's sst file so that it
continues with a new file and we can hardlink and copy a consistent
snapshot. For full snapshots, we simple iterate over all items to copy.
Now this close-and-flush may be more costly (hence the higher duration)
and since this cannot be done asynchronously (as a full snapshot) we
also may not process as many records.
-> Therefore, you probably did not run your program long enough to
create the full set of windows and I'm guessing, you will eventually get
to the same checkpoint sizes.


TLDR; incremental snapshots are worth only (and are designed for...) if
you have a lot of operator state (not just a few MB!) while only few
parts are actually changing between checkpoints. In these scenarios, the
added latency for transferring such a snapshot to the checkpoint store
over network would cover the additional cost during snapshot creation.


Nico


On 21/03/18 06:01, Miyuru Dayarathna wrote:
> Hi,
> 
> Since we could not observe log messages such as "Asynchronous RocksDB
> snapshot" in the Flink's log files, we ran the application with Flink
> 1.3.3 as well. But it also did not print the log message. Hence I am
> wondering whether we ran Flink's incremental checkpointing in the
> correct manner. I have attached the complete application with this
> email. Could you please run this in your setup and let me know whether
> you get the incremental checkpoint related logs printed in your Flink setup?
> 
> Thanks,
> Miyuru
> 
> 
> 
> 
> On Monday, 19 March 2018, 23:33:37 GMT+5:30, Miyuru Dayarathna
>  wrote:
> 
> 
> Hi Nico,
> 
> Thanks for the detailed explanation. The only change I have made in my
> flink-conf.yaml file is the following.
> 
> state.backend.fs.checkpointdir: file:///home/ubuntu/tmp-flink-rocksdb
> 
> The default "state.backend" value is set to filesystem. Removing the
> env.setStateBackend() method code or changing the "state.backend"
> property to rocksdb does not change the state backend to RocksDB. I got
> this verified by looking at the Flink log files. I have mentioned a
> sample of the log file for your reference.
> 
> ---
> carbon-5th:38631/user/taskmanager) as 1ac63dfb481eab3d3165a965084115f3.
> Current number of registered hosts is 1. Current number of alive task
> slots is 1.
> 2018-03-19 23:10:11,606 INFO 
> org.apache.flink.runtime.client.JobClient - Checking
> and uploading JAR files
> 2018-03-19 23:10:11,618 INFO 
> org.apache.flink.runtime.jobmanager.JobManager    -
> Submitting job 7c19a14f4e75149ffaa064fac7e2bf29 (Flink application.).
> 2018-03-19 23:10:11,623 INFO 
> org.apache.flink.runtime.jobmanager.JobManager    - Using
> restart strategy
> FixedDelayRestartStrategy(maxNumberRestartAttempts=2147483647,
> delayBetweenRestartAttempts=1) for 7c19a14f4e75149ffaa064fac7e2bf29.
> 2018-03-19 23:10:11,636 INFO 
> org.apache.flink.runtime.executiongraph.ExecutionGraph    - Job
> recovers via failover strategy: full graph restart
> 2018-03-19 23:10:11,648 INFO 
> org.apache.flink.runtime.jobmanager.JobManager    - Running
> initialization on master for job Flink application.
> (7c19a14f4e75149ffaa064fac7e2bf29).
> 2018-03-19 23:10:11,648 INFO 
> org.apache.flink.runtime.jobmanager.JobManager    -
> Successfully ran initialization on master in 0 ms.
> 2018-03-19 23:10:11,664 INFO 
> org.apache.flink.runtime.jobmanager.JobManager    - Using
> application-defined state backend for checkpoint/savepoint metadata:
> RocksDB State Backend {isInitialized=false, configuredDbBasePaths=null,
> initializedDbBasePaths=null, checkpointStreamBackend=File State Backend
> @ file:/home/ubuntu/tmp-flink-rocksdb}.
> 

Re: Incremental checkpointing performance

2018-03-19 Thread Miyuru Dayarathna
Hi Nico,
Thanks for the detailed explanation. The only change I have made in my 
flink-conf.yaml file is the following.
state.backend.fs.checkpointdir: file:///home/ubuntu/tmp-flink-rocksdb
The default "state.backend" value is set to filesystem. Removing the 
env.setStateBackend() method code or changing the "state.backend" property to 
rocksdb does not change the state backend to RocksDB. I got this verified by 
looking at the Flink log files. I have mentioned a sample of the log file for 
your reference.
---carbon-5th:38631/user/taskmanager)
 as 1ac63dfb481eab3d3165a965084115f3. Current number of registered hosts is 1. 
Current number of alive task slots is 1.
2018-03-19 23:10:11,606 INFO  org.apache.flink.runtime.client.JobClient 
    - Checking and uploading JAR files
2018-03-19 23:10:11,618 INFO  org.apache.flink.runtime.jobmanager.JobManager
    - Submitting job 7c19a14f4e75149ffaa064fac7e2bf29 (Flink 
application.).
2018-03-19 23:10:11,623 INFO  org.apache.flink.runtime.jobmanager.JobManager
    - Using restart strategy 
FixedDelayRestartStrategy(maxNumberRestartAttempts=2147483647, 
delayBetweenRestartAttempts=1) for 7c19a14f4e75149ffaa064fac7e2bf29.
2018-03-19 23:10:11,636 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph    - Job recovers 
via failover strategy: full graph restart
2018-03-19 23:10:11,648 INFO  org.apache.flink.runtime.jobmanager.JobManager
    - Running initialization on master for job Flink application. 
(7c19a14f4e75149ffaa064fac7e2bf29).
2018-03-19 23:10:11,648 INFO  org.apache.flink.runtime.jobmanager.JobManager
    - Successfully ran initialization on master in 0 ms.
2018-03-19 23:10:11,664 INFO  org.apache.flink.runtime.jobmanager.JobManager
    - Using application-defined state backend for checkpoint/savepoint 
metadata: RocksDB State Backend {isInitialized=false, 
configuredDbBasePaths=null, initializedDbBasePaths=null, 
checkpointStreamBackend=File State Backend @ 
file:/home/ubuntu/tmp-flink-rocksdb}.
2018-03-19 23:10:11,685 INFO  org.apache.flink.runtime.jobmanager.JobManager
    - Scheduling job 7c19a14f4e75149ffaa064fac7e2bf29 (Flink 
application.).
2018-03-19 23:10:11,685 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph    - Job Flink 
application. (7c19a14f4e75149ffaa064fac7e2bf29) switched from state CREATED to 
RUNNING.
2018-03-19 23:10:11,692 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph    - Source: 
inputStream -> Filter (1/1) (ebf95b067eb7624edeb151bcba3d55f8) switched from 
CREATED to SCHEDULED.
2018-03-19 23:10:11,698 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph    - 
TriggerWindow(GlobalWindows(), 
ListStateDescriptor{serializer=org.apache.flink.api.common.typeutils.base.ListSerializer@43d5ff75},
 CountTrigger(1), 
org.apache.flink.streaming.api.windowing.evictors.CountEvictor@337a4b0a, 
WindowedStream.reduce(WindowedStream.java:241)) -> Sink: Unnamed (1/1) 
(796fcd9c38c87b6efb6f512e78e626e9) switched from CREATED to SCHEDULED.
2018-03-19 23:10:11,706 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph    - Source: 
inputStream -> Filter (1/1) (ebf95b067eb7624edeb151bcba3d55f8) switched from 
SCHEDULED to DEPLOYING.
2018-03-19 23:10:11,707 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph    - Deploying 
Source: inputStream -> Filter (1/1) (attempt #0) to computer1
2018-03-19 23:10:11,712 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph    - 
TriggerWindow(GlobalWindows(), 
ListStateDescriptor{serializer=org.apache.flink.api.common.typeutils.base.ListSerializer@43d5ff75},
 CountTrigger(1), 
org.apache.flink.streaming.api.windowing.evictors.CountEvictor@337a4b0a, 
WindowedStream.reduce(WindowedStream.java:241)) -> Sink: Unnamed (1/1) 
(796fcd9c38c87b6efb6f512e78e626e9) switched from SCHEDULED to DEPLOYING.
2018-03-19 23:10:11,712 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph    - Deploying 
TriggerWindow(GlobalWindows(), 
ListStateDescriptor{serializer=org.apache.flink.api.common.typeutils.base.ListSerializer@43d5ff75},
 CountTrigger(1), 
org.apache.flink.streaming.api.windowing.evictors.CountEvictor@337a4b0a, 
WindowedStream.reduce(WindowedStream.java:241)) -> Sink: Unnamed (1/1) (attempt 
#0) to computer1
2018-03-19 23:10:12,004 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph    - Source: 
inputStream -> Filter (1/1) (ebf95b067eb7624edeb151bcba3d55f8) switched from 
DEPLOYING to RUNNING.
2018-03-19 23:10:12,011 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph    - 
TriggerWindow(GlobalWindows(), 
ListStateDescriptor{serializer=org.apache.flink.api.common.typeutils.base.ListSerializer@43d5ff75},
 CountTrigger(1), 
org.apache.flink.streaming.api.windowing.evictors.CountEvictor@337a4b0a, 
WindowedStream.reduce(WindowedStream.java:241)) -> 

Re: Incremental checkpointing performance

2018-03-19 Thread Nico Kruber
Hi Miyuru,
Indeed, the behaviour you observed sounds strange and kind of go against
the results Stefan presented in [1]. To see what is going on, can you
also share your changes to Flink's configuration, i.e. flink-conf.yaml?

Let's first make sure you're really comparing RocksDBStateBackend with
vs without incremental checkpoints:
- if you remove this from the code:
env.setStateBackend(new RocksDBStateBackend(
   new FsStateBackend("file:///home/ubuntu/tmp-flink-rocksdb"),
   true));
then you will end up with the state backend configured via the
"state.backend" property. Was this set to "rocksdb"? Alternatively, you
can set the second parameter to the RocksDBStateBackend constructor to
false to get the right back-end.

You can also verify the values you see from the web interface by looking
into the logs (at INFO level). There, you should see reports like this:
"Asynchronous RocksDB snapshot (..., asynchronous part) in thread ...
took ... ms." and "Asynchronous RocksDB snapshot (..., synchronous part)
in thread ... took ... ms."

Other than that, from what I know about it (Stefan (cc'd), correct me if
I'm wrong), incremental checkpoints only do hard links locally to the
changed sst files and then copy the data in there to the checkpoint
store (the path you gave). A full checkpoint must copy all current data.
If, between two checkpoints, you write more data than the contents of
the database, e.g. by updating a key multiple times, you may indeed have
more data to store. Judging from the state sizes you gave, this is
probably not the case.


Let's get started with this and see whether there is anything unusual.


Regards,
Nico


[1]
https://berlin.flink-forward.org/kb_sessions/a-look-at-flinks-internal-data-structures-and-algorithms-for-efficient-checkpointing/

On 19/03/18 05:25, Miyuru Dayarathna wrote:
> Hi,
> 
> We did a performance test of Flink's incremental checkpointing to
> measure the average time it takes to create a checkpoint and the average
> checkpoint file size. We did this test on a single computer in order to
> avoid the latencies introduced by network communication. The computer
> had Intel® Core™ i7-7600U CPU @ 2.80GHz, 4 cores, 16GB RAM, 450GB HDD,
> 101GB free SSD space. The computer was running on Ubuntu 16.04
> LTS, Apache Flink 1.4.1, Java version "1.8.0_152", Java(TM) SE Runtime
> Environment (build 1.8.0_152-b16), Java HotSpot(TM) 64-Bit Server VM
> (build 25.152-b16, mixed mode). We used the JVM flags -Xmx8g -Xms8g. The
> test was run for 40 minutes.
> 
> The Flink application we used is as follows,
> //-
> public class LengthWindowIncrementalCheckpointing {
>     private static DataStream>
> inputStream = null;
>     private static final int PARALLELISM = 1;
>     private static final int timeoutMillis = 10;
>     private static final int WINDOWLENGTH = 1;
>     private static final int SLIDELENGTH = 1;
>     private static Logger logger =
> LoggerFactory.getLogger(LengthWindowIncrementalCheckpointing.class);
> 
>     public static void main(String[] args) {
>     StreamExecutionEnvironment env =
> StreamExecutionEnvironment.getExecutionEnvironment();
> 
>     // start a checkpoint every 1000 ms
>     env.enableCheckpointing(1000);
>     try {
>     env.setStateBackend(new RocksDBStateBackend(
>     new
> FsStateBackend("file:///home/ubuntu/tmp-flink-rocksdb"),
>     true));
>     } catch (IOException e) {
>     e.printStackTrace();
>     }
> 
>     env.setBufferTimeout(timeoutMillis);
>     inputStream = env.addSource(new
> MicrobenchSourceFunction()).setParallelism(PARALLELISM).name("inputStream");
> 
>     DataStream>
> incrementStream2 =
>     inputStream.filter(new FilterFunction Float, Integer, String>>() {
>     @Override
>     public boolean filter(Tuple4 String> tuple) throws Exception {
>     if (tuple.f1 > 10) {
>     return true;
>     }
>     return false;
>     }
>     }).keyBy(1).countWindow(WINDOWLENGTH, SLIDELENGTH).sum(2);
>     incrementStream2.writeUsingOutputFormat(new
> DiscardingOutputFormat     String>>());
> 
>     try {
>     env.execute("Flink application.");
>     } catch (Exception e) {
>     logger.error("Error in starting the Flink stream
> application: " + e.getMessage(), e);
>     }
>     }
> }
> 
> //-
> 
> I have attached two charts (Average_latencies.jpg and
> Average_state_sizes.jpg)