Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 253B1200CDE for ; Tue, 8 Aug 2017 14:06:55 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 23A59167251; Tue, 8 Aug 2017 12:06:55 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id BE83A167250 for ; Tue, 8 Aug 2017 14:06:53 +0200 (CEST) Received: (qmail 5522 invoked by uid 500); 8 Aug 2017 12:06:53 -0000 Mailing-List: contact commits-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list commits@flink.apache.org Received: (qmail 5491 invoked by uid 99); 8 Aug 2017 12:06:52 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 08 Aug 2017 12:06:52 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id F10BCDFA44; Tue, 8 Aug 2017 12:06:50 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: twalthr@apache.org To: commits@flink.apache.org Date: Tue, 08 Aug 2017 12:06:56 -0000 Message-Id: <7709fd0c77064d84bcde3be08d61bd3d@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [7/9] flink git commit: [FLINK-7301] [docs] Rework state documentation archived-at: Tue, 08 Aug 2017 12:06:55 -0000 http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/monitoring/large_state_tuning.md ---------------------------------------------------------------------- diff --git a/docs/monitoring/large_state_tuning.md b/docs/monitoring/large_state_tuning.md deleted file mode 100644 index d8b52ee..0000000 --- a/docs/monitoring/large_state_tuning.md +++ /dev/null @@ -1,237 +0,0 @@ ---- -title: "Debugging and Tuning Checkpoints and Large State" -nav-parent_id: monitoring -nav-pos: 12 ---- - - -This page gives a guide how to configure and tune applications that use large state. - -* ToC -{:toc} - -## Overview - -For Flink applications to run reliably at large scale, two conditions must be fulfilled: - - - The application needs to be able to take checkpoints reliably - - - The resources need to be sufficient catch up with the input data streams after a failure - -The first sections discuss how to get well performing checkpoints at scale. -The last section explains some best practices concerning planning how many resources to use. - - -## Monitoring State and Checkpoints - -The easiest way to monitor checkpoint behavior is via the UI's checkpoint section. The documentation -for [checkpoint monitoring](checkpoint_monitoring.html) shows how to access the available checkpoint -metrics. - -The two numbers that are of particular interest when scaling up checkpoints are: - - - The time until operators start their checkpoint: This time is currently not exposed directly, but corresponds - to: - - `checkpoint_start_delay = end_to_end_duration - synchronous_duration - asynchronous_duration` - - When the time to trigger the checkpoint is constantly very high, it means that the *checkpoint barriers* need a long - time to travel from the source to the operators. That typically indicates that the system is operating under a - constant backpressure. - - - The amount of data buffered during alignments. For exactly-once semantics, Flink *aligns* the streams at - operators that receive multiple input streams, buffering some data for that alignment. - The buffered data volume is ideally low - higher amounts means that checkpoint barriers are reveived at - very different times from the different input streams. - -Note that when the here indicated numbers can be occasionally high in the presence of transient backpressure, data skew, -or network issues. However, if the numbers are constantly very high, it means that Flink puts many resources into checkpointing. - - -## Tuning Checkpointing - -Checkpoints are triggered at regular intervals that applications can configure. When a checkpoint takes longer -to complete than the checkpoint interval, the next checkpoint is not triggered before the in-progress checkpoint -completes. By default the next checkpoint will then be triggered immediately once the ongoing checkpoint completes. - -When checkpoints end up frequently taking longer than the base interval (for example because state -grew larger than planned, or the storage where checkpoints are stored is temporarily slow), -the system is constantly taking checkpoints (new ones are started immediately once ongoing once finish). -That can mean that too many resources are constantly tied up in checkpointing and that the operators make too -little progress. This behavior has less impact on streaming applications that use asynchronously checkpointed state, -but may still have an impact on overall application performance. - -To prevent such a situation, applications can define a *minimum duration between checkpoints*: - -`StreamExecutionEnvironment.getCheckpointConfig().setMinPauseBetweenCheckpoints(milliseconds)` - -This duration is the minimum time interval that must pass between the end of the latest checkpoint and the beginning -of the next. The figure below illustrates how this impacts checkpointing. - -Illustration how the minimum-time-between-checkpoints parameter affects checkpointing behavior. - -*Note:* Applications can be configured (via the `CheckpointConfig`) to allow multiple checkpoints to be in progress at -the same time. For applications with large state in Flink, this often ties up too many resources into the checkpointing. -When a savepoint is manually triggered, it may be in process concurrently with an ongoing checkpoint. - - -## Tuning Network Buffers - -Before Flink 1.3, an increased number of network buffers also caused increased checkpointing times since -keeping more in-flight data meant that checkpoint barriers got delayed. Since Flink 1.3, the -number of network buffers used per outgoing/incoming channel is limited and thus network buffers -may be configured without affecting checkpoint times -(see [network buffer configuration](../setup/config.html#configuring-the-network-buffers)). - -## Make state checkpointing Asynchronous where possible - -When state is *asynchronously* snapshotted, the checkpoints scale better than when the state is *synchronously* snapshotted. -Especially in more complex streaming applications with multiple joins, Co-functions, or windows, this may have a profound -impact. - -To get state to be snapshotted asynchronously, applications have to do two things: - - 1. Use state that is [managed by Flink](../dev/stream/state.html): Managed state means that Flink provides the data - structure in which the state is stored. Currently, this is true for *keyed state*, which is abstracted behind the - interfaces like `ValueState`, `ListState`, `ReducingState`, ... - - 2. Use a state backend that supports asynchronous snapshots. In Flink 1.2, only the RocksDB state backend uses - fully asynchronous snapshots. - -The above two points imply that (in Flink 1.2) large state should generally be kept as keyed state, not as operator state. -This is subject to change with the planned introduction of *managed operator state*. - - -## Tuning RocksDB - -The state storage workhorse of many large scale Flink streaming applications is the *RocksDB State Backend*. -The backend scales well beyond main memory and reliably stores large [keyed state](../dev/stream/state.html). - -Unfortunately, RocksDB's performance can vary with configuration, and there is little documentation on how to tune -RocksDB properly. For example, the default configuration is tailored towards SSDs and performs suboptimal -on spinning disks. - -**Incremental Checkpoints** - -Incremental checkpoints can dramatically reduce the checkpointing time in comparison to full checkpoints, at the cost of a (potentially) longer -recovery time. The core idea is that incremental checkpoints only record all changes to the previous completed checkpoint, instead of -producing a full, self-contained backup of the state backend. Like this, incremental checkpoints build upon previous checkpoints. Flink leverages -RocksDB's internal backup mechanism in a way that is self-consolidating over time. As a result, the incremental checkpoint history in Flink -does not grow indefinitely, and old checkpoints are eventually subsumed and pruned automatically. ` - -While we strongly encourage the use of incremental checkpoints for large state, please note that this is a new feature and currently not enabled -by default. To enable this feature, users can instantiate a `RocksDBStateBackend` with the corresponding boolean flag in the constructor set to `true`, e.g.: - -{% highlight java %} - RocksDBStateBackend backend = - new RocksDBStateBackend(filebackend, true); -{% endhighlight %} - -**Passing Options to RocksDB** - -{% highlight java %} -RocksDBStateBackend.setOptions(new MyOptions()); - -public class MyOptions implements OptionsFactory { - - @Override - public DBOptions createDBOptions() { - return new DBOptions() - .setIncreaseParallelism(4) - .setUseFsync(false) - .setDisableDataSync(true); - } - - @Override - public ColumnFamilyOptions createColumnOptions() { - - return new ColumnFamilyOptions() - .setTableFormatConfig( - new BlockBasedTableConfig() - .setBlockCacheSize(256 * 1024 * 1024) // 256 MB - .setBlockSize(128 * 1024)); // 128 KB - } -} -{% endhighlight %} - -**Predefined Options** - -Flink provides some predefined collections of option for RocksDB for different settings, which can be set for example via -`RocksDBStateBacked.setPredefinedOptions(PredefinedOptions.SPINNING_DISK_OPTIMIZED_HIGH_MEM)`. - -We expect to accumulate more such profiles over time. Feel free to contribute such predefined option profiles when you -found a set of options that work well and seem representative for certain workloads. - -**Important:** RocksDB is a native library, whose allocated memory not from the JVM, but directly from the process' -native memory. Any memory you assign to RocksDB will have to be accounted for, typically by decreasing the JVM heap size -of the TaskManagers by the same amount. Not doing that may result in YARN/Mesos/etc terminating the JVM processes for -allocating more memory than configures. - - -## Capacity Planning - -This section discusses how to decide how many resources should be used for a Flink job to run reliably. -The basic rules of thumb for capacity planning are: - - - Normal operation should have enough capacity to not operate under constant *back pressure*. - See [back pressure monitoring](back_pressure.html) for details on how to check whether the application runs under back pressure. - - - Provision some extra resources on top of the resources needed to run the program back-pressure-free during failure-free time. - These resources are needed to "catch up" with the input data that accumulated during the time the application - was recovering. - How much that should be depends on how long recovery operations usually take (which depends on the size of the state - that needs to be loaded into the new TaskManagers on a failover) and how fast the scenario requires failures to recover. - - *Important*: The base line should to be established with checkpointing activated, because checkpointing ties up - some amount of resources (such as network bandwidth). - - - Temporary back pressure is usually okay, and an essential part of execution flow control during load spikes, - during catch-up phases, or when external systems (that are written to in a sink) exhibit temporary slowdown. - - - Certain operations (like large windows) result in a spiky load for their downstream operators: - In the case of windows, the downstream operators may have little to do while the window is being built, - and have a load to do when the windows are emitted. - The planning for the downstream parallelism needs to take into account how much the windows emit and how - fast such a spike needs to be processed. - -**Important:** In order to allow for adding resources later, make sure to set the *maximum parallelism* of the -data stream program to a reasonable number. The maximum parallelism defines how high you can set the programs -parallelism when re-scaling the program (via a savepoint). - -Flink's internal bookkeeping tracks parallel state in the granularity of max-parallelism-many *key groups*. -Flink's design strives to make it efficient to have a very high value for the maximum parallelism, even if -executing the program with a low parallelism. - -## Compression - -Flink offers optional compression (default: off) for all checkpoints and savepoints. Currently, compression always uses -the [snappy compression algorithm (version 1.1.4)](https://github.com/xerial/snappy-java) but we are planning to support -custom compression algorithms in the future. Compression works on the granularity of key-groups in keyed state, i.e. -each key-group can be decompressed individually, which is important for rescaling. - -Compression can be activated through the `ExecutionConfig`: - -{% highlight java %} - ExecutionConfig executionConfig = new ExecutionConfig(); - executionConfig.setUseSnapshotCompression(true); -{% endhighlight %} - -**Notice:** The compression option has no impact on incremental snapshots, because they are using RocksDB's internal -format which is always using snappy compression out of the box. http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/ops/README.md ---------------------------------------------------------------------- diff --git a/docs/ops/README.md b/docs/ops/README.md deleted file mode 100644 index 5fe6568..0000000 --- a/docs/ops/README.md +++ /dev/null @@ -1,21 +0,0 @@ - - -This folder contains the documentation in the category -**Deployment & Operations**. http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/ops/cli.md ---------------------------------------------------------------------- diff --git a/docs/ops/cli.md b/docs/ops/cli.md new file mode 100644 index 0000000..7b36177 --- /dev/null +++ b/docs/ops/cli.md @@ -0,0 +1,355 @@ +--- +title: "Command-Line Interface" +nav-title: CLI +nav-parent_id: ops +nav-pos: 6 +--- + + +Flink provides a command-line interface to run programs that are packaged +as JAR files, and control their execution. The command line interface is part +of any Flink setup, available in local single node setups and in +distributed setups. It is located under `/bin/flink` +and connects by default to the running Flink master (JobManager) that was +started from the same installation directory. + +A prerequisite to using the command line interface is that the Flink +master (JobManager) has been started (via +`/bin/start-local.sh` or +`/bin/start-cluster.sh`) or that a YARN environment is +available. + +The command line can be used to + +- submit jobs for execution, +- cancel a running job, +- provide information about a job, and +- list running and waiting jobs. + +* This will be replaced by the TOC +{:toc} + +## Examples + +- Run example program with no arguments. + + ./bin/flink run ./examples/batch/WordCount.jar + +- Run example program with arguments for input and result files + + ./bin/flink run ./examples/batch/WordCount.jar \ + --input file:///home/user/hamlet.txt --output file:///home/user/wordcount_out + +- Run example program with parallelism 16 and arguments for input and result files + + ./bin/flink run -p 16 ./examples/batch/WordCount.jar \ + --input file:///home/user/hamlet.txt --output file:///home/user/wordcount_out + +- Run example program with flink log output disabled + + ./bin/flink run -q ./examples/batch/WordCount.jar + +- Run example program in detached mode + + ./bin/flink run -d ./examples/batch/WordCount.jar + +- Run example program on a specific JobManager: + + ./bin/flink run -m myJMHost:6123 \ + ./examples/batch/WordCount.jar \ + --input file:///home/user/hamlet.txt --output file:///home/user/wordcount_out + +- Run example program with a specific class as an entry point: + + ./bin/flink run -c org.apache.flink.examples.java.wordcount.WordCount \ + ./examples/batch/WordCount.jar \ + --input file:///home/user/hamlet.txt --output file:///home/user/wordcount_out + +- Run example program using a [per-job YARN cluster]({{site.baseurl}}/ops/deployment/yarn_setup.html#run-a-single-flink-job-on-hadoop-yarn) with 2 TaskManagers: + + ./bin/flink run -m yarn-cluster -yn 2 \ + ./examples/batch/WordCount.jar \ + --input hdfs:///user/hamlet.txt --output hdfs:///user/wordcount_out + +- Display the optimized execution plan for the WordCount example program as JSON: + + ./bin/flink info ./examples/batch/WordCount.jar \ + --input file:///home/user/hamlet.txt --output file:///home/user/wordcount_out + +- List scheduled and running jobs (including their JobIDs): + + ./bin/flink list + +- List scheduled jobs (including their JobIDs): + + ./bin/flink list -s + +- List running jobs (including their JobIDs): + + ./bin/flink list -r + +- List running Flink jobs inside Flink YARN session: + + ./bin/flink list -m yarn-cluster -yid -r + +- Cancel a job: + + ./bin/flink cancel + +- Cancel a job with a savepoint: + + ./bin/flink cancel -s [targetDirectory] + +- Stop a job (streaming jobs only): + + ./bin/flink stop + + +The difference between cancelling and stopping a (streaming) job is the following: + +On a cancel call, the operators in a job immediately receive a `cancel()` method call to cancel them as +soon as possible. +If operators are not not stopping after the cancel call, Flink will start interrupting the thread periodically +until it stops. + +A "stop" call is a more graceful way of stopping a running streaming job. Stop is only available for jobs +which use sources that implement the `StoppableFunction` interface. When the user requests to stop a job, +all sources will receive a `stop()` method call. The job will keep running until all sources properly shut down. +This allows the job to finish processing all inflight data. + +### Savepoints + +[Savepoints]({{site.baseurl}}/ops/state/savepoints.html) are controlled via the command line client: + +#### Trigger a savepoint + +{% highlight bash %} +./bin/flink savepoint [savepointDirectory] +{% endhighlight %} + +Returns the path of the created savepoint. You need this path to restore and dispose savepoints. + +You can optionally specify a `savepointDirectory` when triggering the savepoint. If you don't specify one here, you need to configure a default savepoint directory for the Flink installation (see [Savepoints]({{site.baseurl}}/ops/state/savepoints.html#configuration)). + +##### Cancel with a savepoint + +You can atomically trigger a savepoint and cancel a job. + +{% highlight bash %} +./bin/flink cancel -s [savepointDirectory] +{% endhighlight %} + +If no savepoint directory is configured, you need to configure a default savepoint directory for the Flink installation (see [Savepoints]({{site.baseurl}}/ops/state/savepoints.html#configuration)). + +The job will only be cancelled if the savepoint succeeds. + +#### Restore a savepoint + +{% highlight bash %} +./bin/flink run -s ... +{% endhighlight %} + +The run command has a savepoint flag to submit a job, which restores its state from a savepoint. The savepoint path is returned by the savepoint trigger command. + +By default, we try to match all savepoint state to the job being submitted. If you want to allow to skip savepoint state that cannot be restored with the new job you can set the `allowNonRestoredState` flag. You need to allow this if you removed an operator from your program that was part of the program when the savepoint was triggered and you still want to use the savepoint. + +{% highlight bash %} +./bin/flink run -s -n ... +{% endhighlight %} + +This is useful if your program dropped an operator that was part of the savepoint. + +#### Dispose a savepoint + +{% highlight bash %} +./bin/flink savepoint -d +{% endhighlight %} + +Disposes the savepoint at the given path. The savepoint path is returned by the savepoint trigger command. + +If you use custom state instances (for example custom reducing state or RocksDB state), you have to specify the path to the program JAR with which the savepoint was triggered in order to dispose the savepoint with the user code class loader: + +{% highlight bash %} +./bin/flink savepoint -d -j +{% endhighlight %} + +Otherwise, you will run into a `ClassNotFoundException`. + +## Usage + +The command line syntax is as follows: + +~~~ +./flink [OPTIONS] [ARGUMENTS] + +The following actions are available: + +Action "run" compiles and runs a program. + + Syntax: run [OPTIONS] + "run" action options: + -c,--class Class with the program entry + point ("main" method or + "getPlan()" method. Only + needed if the JAR file does + not specify the class in its + manifest. + -C,--classpath Adds a URL to each user code + classloader on all nodes in + the cluster. The paths must + specify a protocol (e.g. + file://) and be accessible + on all nodes (e.g. by means + of a NFS share). You can use + this option multiple times + for specifying more than one + URL. The protocol must be + supported by the {@link + java.net.URLClassLoader}. + -d,--detached If present, runs the job in + detached mode + -m,--jobmanager Address of the JobManager + (master) to which to + connect. Use this flag to + connect to a different + JobManager than the one + specified in the + configuration. + -n,--allowNonRestoredState Allow non restored savepoint + state in case an operator has + been removed from the job. + -p,--parallelism The parallelism with which + to run the program. Optional + flag to override the default + value specified in the + configuration. + -q,--sysoutLogging If present, suppress logging + output to standard out. + -s,--fromSavepoint Path to a savepoint to + restore the job from (for + example + hdfs:///flink/savepoint-1537 + ). + -z,--zookeeperNamespace Namespace to create the + Zookeeper sub-paths for high + availability mode + + Options for yarn-cluster mode: + -yD Dynamic properties + -yd,--yarndetached Start detached + -yid,--yarnapplicationId Attach to running YARN session + -yj,--yarnjar Path to Flink jar file + -yjm,--yarnjobManagerMemory Memory for JobManager Container [in + MB] + -yn,--yarncontainer Number of YARN container to allocate + (=Number of Task Managers) + -ynm,--yarnname Set a custom name for the application + on YARN + -yq,--yarnquery Display available YARN resources + (memory, cores) + -yqu,--yarnqueue Specify YARN queue. + -ys,--yarnslots Number of slots per TaskManager + -yst,--yarnstreaming Start Flink in streaming mode + -yt,--yarnship Ship files in the specified directory + (t for transfer) + -ytm,--yarntaskManagerMemory Memory per TaskManager Container [in + MB] + -yz,--yarnzookeeperNamespace Namespace to create the Zookeeper + sub-paths for high availability mode + + + +Action "info" shows the optimized execution plan of the program (JSON). + + Syntax: info [OPTIONS] + "info" action options: + -c,--class Class with the program entry point ("main" + method or "getPlan()" method. Only needed + if the JAR file does not specify the class + in its manifest. + -p,--parallelism The parallelism with which to run the + program. Optional flag to override the + default value specified in the + configuration. + Options for yarn-cluster mode: + -yid,--yarnapplicationId Attach to running YARN session + + + +Action "list" lists running and scheduled programs. + + Syntax: list [OPTIONS] + "list" action options: + -m,--jobmanager Address of the JobManager (master) to which + to connect. Use this flag to connect to a + different JobManager than the one specified + in the configuration. + -r,--running Show only running programs and their JobIDs + -s,--scheduled Show only scheduled programs and their JobIDs + Options for yarn-cluster mode: + -yid,--yarnapplicationId Attach to running YARN session + + + +Action "stop" stops a running program (streaming jobs only). + + Syntax: stop [OPTIONS] + "stop" action options: + -m,--jobmanager Address of the JobManager (master) to which + to connect. Use this flag to connect to a + different JobManager than the one specified + in the configuration. + Options for yarn-cluster mode: + -yid,--yarnapplicationId Attach to running YARN session + + + +Action "cancel" cancels a running program. + + Syntax: cancel [OPTIONS] + "cancel" action options: + -m,--jobmanager Address of the JobManager (master) + to which to connect. Use this flag + to connect to a different JobManager + than the one specified in the + configuration. + -s,--withSavepoint Trigger savepoint and cancel job. + The target directory is optional. If + no directory is specified, the + configured default directory + (state.savepoints.dir) is used. + Options for yarn-cluster mode: + -yid,--yarnapplicationId Attach to running YARN session + + + +Action "savepoint" triggers savepoints for a running job or disposes existing ones. + + Syntax: savepoint [OPTIONS] [] + "savepoint" action options: + -d,--dispose Path of savepoint to dispose. + -j,--jarfile Flink program JAR file. + -m,--jobmanager Address of the JobManager (master) to which + to connect. Use this flag to connect to a + different JobManager than the one specified + in the configuration. + Options for yarn-cluster mode: + -yid,--yarnapplicationId Attach to running YARN session +~~~