You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/12/07 21:02:13 UTC

[GitHub] [flink] XComp commented on a change in pull request #14287: [FLINK-20352][docs] Updated CLI documentation.

XComp commented on a change in pull request #14287:
URL: https://github.com/apache/flink/pull/14287#discussion_r537830044



##########
File path: docs/deployment/cli.md
##########
@@ -23,664 +23,254 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-Flink provides a Command-Line Interface (CLI) to run programs that are packaged
-as JAR files, and control their execution.  The CLI is part
-of any Flink setup, available in local single node setups and in
-distributed setups. It is located under `<flink-home>/bin/flink`
-and connects by default to the running JobManager that was
-started from the same installation directory.
-
-The command line can be used to
-
-- submit jobs for execution,
-- cancel a running job,
-- provide information about a job,
-- list running and waiting jobs,
-- trigger and dispose savepoints
-
-A prerequisite to using the command line interface is that the Flink
-master (JobManager) has been started (via
-`<flink-home>/bin/start-cluster.sh`) or that another deployment target such as YARN or Kubernetes is
-available.
-
-### Deployment targets
-
-Flink has the concept of executors for defining available deployment targets. You can see the
-available executors in the output of `bin/flink --help`, for example:
-
-```
-Options for Generic CLI mode:
-     -D <property=value>   Generic configuration options for
-                           execution/deployment and for the configured executor.
-                           The available options can be found at
-                           https://ci.apache.org/projects/flink/flink-docs-stabl
-                           e/deployment/config.html
-     -t,--target <arg>     The deployment target for the given application,
-                           which is equivalent to the "execution.target" config
-                           option. The currently available targets are:
-                           "remote", "local", "kubernetes-session", "yarn-per-job",
-                           "yarn-session", "yarn-application" and "kubernetes-application".
-```
-
-When running one of the `bin/flink` actions, the executor is specified using the `--executor`
-option.                  
-
-
+Flink provides a Command-Line Interface (CLI) `bin/flink` to run programs that 
+are packaged as JAR files and to control their execution. The CLI is part of any 
+Flink setup, available in local single node setups and in distributed setups. 
+It connects to the running JobManager specified in `conf/flink-config.yaml`.
 
 * This will be replaced by the TOC
 {:toc}
 
-## Examples
-### Job Submission Examples
------------------------------
-
-These examples about how to submit a job in CLI.
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-
--   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 in detached mode:
-
-            ./bin/flink run -d ./examples/batch/WordCount.jar
-
--   Run example program on a specific JobManager:
-
-        ./bin/flink run -m myJMHost:8081 \
-                               ./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]({% link deployment/resource-providers/yarn.md %}#run-a-single-flink-job-on-hadoop-yarn) with 2 TaskManagers:
-
-        ./bin/flink run -m yarn-cluster \
-                               ./examples/batch/WordCount.jar \
-                               --input hdfs:///user/hamlet.txt --output hdfs:///user/wordcount_out
-
-</div>
+## Job Lifecycle Management
 
-<div data-lang="python" markdown="1">
-
-<span class="label label-info">Note</span> When submitting Python job via `flink run`, Flink will run the command “python”. Please run the following command to confirm that the command “python” in current environment points to a specified Python version 3.5, 3.6, 3.7 or 3.8:
+A prerequisite for the commands listed in this section to work is to have a running Flink deployment 
+like [Kubernetes]({% link deployment/resource-providers/native_kubernetes.md %}), 
+[YARN]({% link deployment/resource-providers/yarn.md %}) or any other option available. Feel free to 
+[start a Flink cluster locally]({% link deployment/resource-providers/standalone/local.md %}#start-a-local-flink-cluster) 
+to try the commands on your own machine.
+ 
+### Submitting a Job
 
+Submitting a job means uploading the job's JAR and related dependencies to the Flink cluster and 
+initiating the job execution. For the sake of this example, we select a long-running job like 
+`examples/streaming/StateMachineExample.jar`. Feel free to select any other JAR archive from the 
+`examples/` folder or deploy your own job.
 {% highlight bash %}
-$ python --version
-# the version printed here must be 3.5, 3.6, 3.7 or 3.8
+./bin/flink run --detached ./examples/streaming/StateMachineExample.jar
 {% endhighlight %}
-
--   Run Python Table program:
-
-        ./bin/flink run -py examples/python/table/batch/word_count.py
-
--   Run Python Table program with pyFiles:
-
-        ./bin/flink run -py examples/python/table/batch/word_count.py \
-                                -pyfs file:///user.txt,hdfs:///$namenode_address/username.txt
-
--   Run Python Table program with a JAR file:
-
-        ./bin/flink run -py examples/python/table/batch/word_count.py -j <jarFile>
-
--   Run Python Table program with pyFiles and pyModule:
-
-        ./bin/flink run -pym batch.word_count -pyfs examples/python/table/batch
-
--   Run Python Table program with parallelism 16:
-
-        ./bin/flink run -p 16 -py examples/python/table/batch/word_count.py
-
--   Run Python Table program with flink log output disabled:
-
-        ./bin/flink run -q -py examples/python/table/batch/word_count.py
-
--   Run Python Table program in detached mode:
-
-        ./bin/flink run -d -py examples/python/table/batch/word_count.py
-
--   Run Python Table program on a specific JobManager:
-
-        ./bin/flink run -m myJMHost:8081 \
-                               -py examples/python/table/batch/word_count.py
-
--   Run Python Table program using a [per-job YARN cluster]({% link deployment/resource-providers/yarn.md %}#run-a-single-flink-job-on-hadoop-yarn) with 2 TaskManagers:
-
-        ./bin/flink run -m yarn-cluster \
-                               -py examples/python/table/batch/word_count.py
-</div>
-
-### Job Management Examples
------------------------------
-
-These examples about how to manage a job in CLI.
-
--   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 all existing jobs (including their JobIDs):
-
-        ./bin/flink list -a
-
--   List running Flink jobs inside Flink YARN session:
-
-        ./bin/flink list -m yarn-cluster -yid <yarnApplicationID> -r
-
--   Cancel a job:
-
-        ./bin/flink cancel <jobID>
-
--   Cancel a job with a savepoint (deprecated; use "stop" instead):
-
-        ./bin/flink cancel -s [targetDirectory] <jobID>
-
--   Gracefully stop a job with a savepoint (streaming jobs only):
-
-        ./bin/flink stop [-p targetDirectory] [-d] <jobID>
-
-### Savepoints
-
-[Savepoints]({% link ops/state/savepoints.md %}) are controlled via the command line client:
-
-#### Trigger a Savepoint
-
+Submitting the job using `--detached` will make the command return after the submission is done.
+The output contains (besides other things) the ID of the newly submitted job.
+```
+Usage with built-in data generator: StateMachineExample [--error-rate <probability-of-invalid-transition>] [--sleep <sleep-per-record-in-ms>]
+Usage with Kafka: StateMachineExample --kafka-topic <topic> [--brokers <brokers>]
+Options for both the above setups:
+        [--backend <file|rocks>]
+        [--checkpoint-dir <filepath>]
+        [--async-checkpoints <true|false>]
+        [--incremental-checkpoints <true|false>]
+        [--output <filepath> OR null for stdout]
+
+Using standalone source with error rate 0.000000 and sleep delay 1 millis
+
+Job has been submitted with JobID cca7bc1061d61cf15238e92312c2fc20
+```
+The usage information printed lists job-related parameters that can be added to the end of the job 
+submission command if necessary. For the purpose of readability, we assume that the returned JobID is 
+stored in a variable `JOB_ID` for the commands below:
 {% highlight bash %}
-./bin/flink savepoint <jobId> [savepointDirectory]
+export JOB_ID="cca7bc1061d61cf15238e92312c2fc20"
 {% endhighlight %}
 
-This will trigger a savepoint for the job with ID `jobId`, and returns the path of the created savepoint. You need this path to restore and dispose savepoints.
-
+There is another action called `run-application` available to run the job in 
+[Application Mode]({% link deployment/index.md %}#deployment-modes). This documentation does not address
+this action individually as it works similarly to the `run` action in terms of the CLI frontend.
 
-Furthermore, you can optionally specify a target file system directory to store the savepoint in. The directory needs to be accessible by the JobManager.
-
-If you don't specify a target directory, you need to have [configured a default directory]({% link ops/state/savepoints.md %}#configuration). Otherwise, triggering the savepoint will fail.
-
-#### Trigger a Savepoint with YARN
+### Job Monitoring
 
+You can monitor any running jobs using the `list` action:
 {% highlight bash %}
-./bin/flink savepoint <jobId> [savepointDirectory] -yid <yarnAppId>
+./bin/flink list
 {% endhighlight %}
+```
+Waiting for response...
+------------------ Running/Restarting Jobs -------------------
+30.11.2020 16:02:29 : cca7bc1061d61cf15238e92312c2fc20 : State machine job (RUNNING)
+--------------------------------------------------------------
+No scheduled jobs.
+```
+Jobs that were submitted but not started, yet, would be listed under "Scheduled Jobs".
 
-This will trigger a savepoint for the job with ID `jobId` and YARN application ID `yarnAppId`, and returns the path of the created savepoint.
-
-Everything else is the same as described in the above **Trigger a Savepoint** section.
-
-#### Stop
-
-Use the `stop` to gracefully stop a running streaming job with a savepoint.
+### Creating a Savepoint
 
+Savepoints can be created to save the current state a job is in. All that's needed is the JobID:
 {% highlight bash %}
-./bin/flink stop [-p targetDirectory] [-d] <jobID>
+./bin/flink savepoint $JOB_ID /tmp/flink-savepoints
 {% endhighlight %}
+```
+Triggering savepoint for job cca7bc1061d61cf15238e92312c2fc20.
+Waiting for response...
+Savepoint completed. Path: file:/tmp/flink-savepoints/savepoint-cca7bc-bb1e257f0dab
+You can resume your program from this savepoint with the run command.
+```
+The savepoint folder is optional and needs to be specified if 
+[state.savepoints.dir]({% link deployment/config.md %}#state-savepoints-dir) isn't set.
 
-A "stop" call is a more graceful way of stopping a running streaming job, as the "stop" signal flows from
-source to sink. When the user requests to stop a job, all sources will be requested to send the last checkpoint barrier
-that will trigger a savepoint, and after the successful completion of that savepoint, they will finish by calling their
-`cancel()` method. If the `-d` flag is specified, then a `MAX_WATERMARK` will be emitted before the last checkpoint
-barrier. This will result all registered event-time timers to fire, thus flushing out any state that is waiting for
-a specific watermark, e.g. windows. The job will keep running until all sources properly shut down. This allows the
- job to finish processing all in-flight data.
-
-
-#### Cancel with a savepoint (deprecated)
+The path to the savepoint can be used later on to [restart the Flink job](#starting-a-job-from-a-savepoint).
 
-You can atomically trigger a savepoint and cancel a job.
+#### Disposing a Savepoint
 
+The `savepoint` action can be also used to remove savepoints. `--dispose` with the corresponding 
+savepoint path needs to be added:
 {% highlight bash %}
-./bin/flink cancel -s [savepointDirectory] <jobID>
+./bin/flink savepoint --dispose /tmp/flink-savepoints/savepoint-cca7bc-bb1e257f0dab $JOB_ID
 {% endhighlight %}
+```
+Disposing savepoint '/tmp/flink-savepoints/savepoint-cca7bc-bb1e257f0dab'.
+Waiting for response...
+Savepoint '/tmp/flink-savepoints/savepoint-cca7bc-bb1e257f0dab' disposed.
+```
+Triggering the savepoint disposal through the `savepoint` action does not only remove the data from 
+the storage but makes Flink clean up the savepoint-related metadata as well.
 
-If no savepoint directory is configured, you need to configure a default savepoint directory for the Flink installation (see [Savepoints]({% link ops/state/savepoints.md %}#configuration)).
-
-The job will only be cancelled if the savepoint succeeds.
-
-<p style="border-radius: 5px; padding: 5px" class="bg-danger">
-    <b>Note</b>: Cancelling a job with savepoint is deprecated. Use "stop" instead.
-</p>
-
-#### Restore a savepoint
+### Cancelling a Job
 
+Cancelling a job can be achieved through the `cancel` action:
 {% highlight bash %}
-./bin/flink run -s <savepointPath> ...
+./bin/flink cancel $JOB_ID
 {% endhighlight %}
+```
+Cancelling job cca7bc1061d61cf15238e92312c2fc20.
+Cancelled job cca7bc1061d61cf15238e92312c2fc20.
+```
+The corresponding job's state will be transitioned from `Running` to `Cancelled`. Any computations 
+will be stopped.
 
-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.
+#### Stop: Cancelling a Job but creating a final Savepoint
 
+Another action for stopping a job is `stop`. It function in the same way as `cancel` but creates a 
+savepoint as part of the cancellation process:
 {% highlight bash %}
-./bin/flink run -s <savepointPath> -n ...
+./bin/flink stop --savepointPath /tmp-flink-savepoints $JOB_ID
 {% endhighlight %}
+```
+Suspending job "cca7bc1061d61cf15238e92312c2fc20" with a savepoint.
+Savepoint completed. Path: file:/tmp/flink-savepoints/savepoint-cca7bc-bb1e257f0dab
+```
+We have to use `--savepointPath` to specify the savepoint folder if 
+[state.savepoints.dir]({% link deployment/config.md %}#state-savepoints-dir) isn't set.
 
-This is useful if your program dropped an operator that was part of the savepoint.
-
-#### Dispose a savepoint
+### Starting a Job from a Savepoint
 
+Starting a job from a savepoint can be achieved using the `run` (and `run-application`) action.
 {% highlight bash %}
-./bin/flink savepoint -d <savepointPath>
+./bin/flink run --detached --fromSavepoint /tmp/flink-savepoints/savepoint-cca7bc-bb1e257f0dab ./examples/streaming/StateMachineExample.jar
 {% endhighlight %}
+```
+Usage with built-in data generator: StateMachineExample [--error-rate <probability-of-invalid-transition>] [--sleep <sleep-per-record-in-ms>]
+Usage with Kafka: StateMachineExample --kafka-topic <topic> [--brokers <brokers>]
+Options for both the above setups:
+        [--backend <file|rocks>]
+        [--checkpoint-dir <filepath>]
+        [--async-checkpoints <true|false>]
+        [--incremental-checkpoints <true|false>]
+        [--output <filepath> OR null for stdout]
+
+Using standalone source with error rate 0.000000 and sleep delay 1 millis
+
+Job has been submitted with JobID 97b20a0a8ffd5c1d656328b0cd6436a6
+```
 
-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:
+See how the command is equal to the [initial run command](#submitting-a-job) except for the 
+`--fromSavepoint` parameter which is used to refer to the state of the 
+[previously cancelled job](#cancelling-a-job).  A new JobID is generated that can be used to 
+maintain the job.
 
-{% highlight bash %}
-./bin/flink savepoint -d <savepointPath> -j <jarFile>
-{% endhighlight %}
-
-Otherwise, you will run into a `ClassNotFoundException`.
+{% top %}
 
-## Usage
+## CLI Actions
+
+Here's an overview of actions supported by Flink's CLI tool:
+<table class="table table-bordered">
+    <thead>
+        <tr>
+          <th class="text-left" style="width: 25%">Action</th>
+          <th class="text-left" style="width: 50%">Purpose</th>
+        </tr>
+    </thead>
+    <tbody>
+        <tr>
+            <td><code class="highlighter-rouge">run</code></td>
+            <td>
+                This action executes jobs. It requires at least the jar containing the job. Flink-
+                or job-related arguments can be passed if necessary.
+            </td>
+        </tr>
+        <tr>
+            <td><code class="highlighter-rouge">run-application</code></td>
+            <td>
+                This action executes jobs in <a href="{% link deployment/index.md %}#deployment-modes">
+                Application Mode</a>. Other than that, it requires the same parameters as the 
+                <code class="highlighter-rouge">run</code> action.
+            </td>
+        </tr>
+        <tr>
+            <td><code class="highlighter-rouge">info</code></td>
+            <td>
+                This action can be used to print an optimized execution graph of the passed job. Again,
+                the jar containing the job needs to be passed.
+            </td>
+        </tr>
+        <tr>
+            <td><code class="highlighter-rouge">list</code></td>
+            <td>
+                This action lists all running or scheduled jobs.
+            </td>
+        </tr>
+        <tr>
+            <td><code class="highlighter-rouge">savepoint</code></td>
+            <td>
+                This action can be used to create or disposing savepoints for a given job. It might be
+                necessary to specify a savepoint directory besides the JobID, if the 
+                <a href="{% link deployment/config.md %}#state-savepoints-dir">state.savepoints.dir</a> 
+                parameter was not specified in <code class="highlighter-rouge">conf/flink-config.yaml</code>.
+            </td>
+        </tr>
+        <tr>
+            <td><code class="highlighter-rouge">cancel</code></td>
+            <td>
+                This action can be used to cancel running jobs based on their JobID.
+            </td>
+        </tr>
+        <tr>
+            <td><code class="highlighter-rouge">stop</code></td>
+            <td>
+                This action combines the <code class="highlighter-rouge">cancel</code> and 
+                <code class="highlighter-rouge">savepoint</code> actions to stop a running job 
+                but also create a savepoint to start from again.
+            </td>
+        </tr>
+    </tbody>
+</table>
+
+A more fine-grained description of all actions and their parameters can be accessed through `bin/flink --help` 
+or the usage information of each individual action `bin/flink <action> --help`.
 
-The command line syntax is as follows:
+{% top %}
 
-{% highlight bash %}
-./flink <ACTION> [OPTIONS] [ARGUMENTS]
-
-The following actions are available:
-
-Action "run" compiles and runs a program.
-
-  Syntax: run [OPTIONS] <jar-file> <arguments>
-  "run" action options:
-     -c,--class <classname>               Class with the program entry point
-                                          ("main()" method). Only needed if the
-                                          JAR file does not specify the class in
-                                          its manifest.
-     -C,--classpath <url>                 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
-     -n,--allowNonRestoredState           Allow to skip savepoint state that
-                                          cannot be restored. You need to allow
-                                          this if you removed an operator from
-                                          your program that was part of the
-                                          program when the savepoint was
-                                          triggered.
-     -p,--parallelism <parallelism>       The parallelism with which to run the
-                                          program. Optional flag to override the
-                                          default value specified in the
-                                          configuration.
-     -py,--python <pythonFile>            Python script with the program entry
-                                          point. The dependent resources can be
-                                          configured with the `--pyFiles`
-                                          option.
-     -pyarch,--pyArchives <arg>           Add python archive files for job. The
-                                          archive files will be extracted to the
-                                          working directory of python UDF
-                                          worker. Currently only zip-format is
-                                          supported. For each archive file, a
-                                          target directory be specified. If the
-                                          target directory name is specified,
-                                          the archive file will be extracted to
-                                          a name can directory with the
-                                          specified name. Otherwise, the archive
-                                          file will be extracted to a directory
-                                          with the same name of the archive
-                                          file. The files uploaded via this
-                                          option are accessible via relative
-                                          path. '#' could be used as the
-                                          separator of the archive file path and
-                                          the target directory name. Comma (',')
-                                          could be used as the separator to
-                                          specify multiple archive files. This
-                                          option can be used to upload the
-                                          virtual environment, the data files
-                                          used in Python UDF (e.g.: --pyArchives
-                                          file:///tmp/py37.zip,file:///tmp/data.
-                                          zip#data --pyExecutable
-                                          py37.zip/py37/bin/python). The data
-                                          files could be accessed in Python UDF,
-                                          e.g.: f = open('data/data.txt', 'r').
-     -pyexec,--pyExecutable <arg>         Specify the path of the python
-                                          interpreter used to execute the python
-                                          UDF worker (e.g.: --pyExecutable
-                                          /usr/local/bin/python3). The python
-                                          UDF worker depends on a specified Python
-                                          version 3.5, 3.6 3.7 or 3.8, Apache Beam
-                                          (version == 2.23.0), Pip (version >= 7.1.0)
-                                          and SetupTools (version >= 37.0.0).
-                                          Please ensure that the specified environment
-                                          meets the above requirements.
-     -pyfs,--pyFiles <pythonFiles>        Attach custom python files for job.
-                                          These files will be added to the
-                                          PYTHONPATH of both the local client
-                                          and the remote python UDF worker. The
-                                          standard python resource file suffixes
-                                          such as .py/.egg/.zip or directory are
-                                          all supported. Comma (',') could be
-                                          used as the separator to specify
-                                          multiple files (e.g.: --pyFiles
-                                          file:///tmp/myresource.zip,hdfs:///$na
-                                          menode_address/myresource2.zip).
-     -pym,--pyModule <pythonModule>       Python module with the program entry
-                                          point. This option must be used in
-                                          conjunction with `--pyFiles`.
-     -pyreq,--pyRequirements <arg>        Specify a requirements.txt file which
-                                          defines the third-party dependencies.
-                                          These dependencies will be installed
-                                          and added to the PYTHONPATH of the
-                                          python UDF worker. A directory which
-                                          contains the installation packages of
-                                          these dependencies could be specified
-                                          optionally. Use '#' as the separator
-                                          if the optional parameter exists
-                                          (e.g.: --pyRequirements
-                                          file:///tmp/requirements.txt#file:///t
-                                          mp/cached_dir).
-     -s,--fromSavepoint <savepointPath>   Path to a savepoint to restore the job
-                                          from (for example
-                                          hdfs:///flink/savepoint-1537).
-     -sae,--shutdownOnAttachedExit        If the job is submitted in attached
-                                          mode, perform a best-effort cluster
-                                          shutdown when the CLI is terminated
-                                          abruptly, e.g., in response to a user
-                                          interrupt, such as typing Ctrl + C.
-  Options for Generic CLI mode:
-     -D <property=value>   Generic configuration options for
-                           execution/deployment and for the configured executor.
-                           The available options can be found at
-                           https://ci.apache.org/projects/flink/flink-docs-stabl
-                           e/deployment/config.html
-     -e,--executor <arg>   DEPRECATED: Please use the -t option instead which is
-                           also available with the "Application Mode".
-                           The name of the executor to be used for executing the
-                           given job, which is equivalent to the
-                           "execution.target" config option. The currently
-                           available executors are: "remote", "local",
-                           "kubernetes-session", "yarn-per-job", "yarn-session".
-     -t,--target <arg>     The deployment target for the given application,
-                           which is equivalent to the "execution.target" config
-                           option. The currently available targets are:
-                           "remote", "local", "kubernetes-session",
-                           "yarn-per-job", "yarn-session", "yarn-application"
-                           and "kubernetes-application".
-
-  Options for yarn-cluster mode:
-     -d,--detached                        If present, runs the job in detached
-                                          mode
-     -m,--jobmanager <arg>                Set to yarn-cluster to use YARN
-                                          execution mode.
-     -yat,--yarnapplicationType <arg>     Set a custom application type for the
-                                          application on YARN
-     -yD <property=value>                 use value for given property
-     -yd,--yarndetached                   If present, runs the job in detached
-                                          mode (deprecated; use non-YARN
-                                          specific option instead)
-     -yh,--yarnhelp                       Help for the Yarn session CLI.
-     -yid,--yarnapplicationId <arg>       Attach to running YARN session
-     -yj,--yarnjar <arg>                  Path to Flink jar file
-     -yjm,--yarnjobManagerMemory <arg>    Memory for JobManager Container with
-                                          optional unit (default: MB)
-     -ynl,--yarnnodeLabel <arg>           Specify YARN node label for the YARN
-                                          application
-     -ynm,--yarnname <arg>                Set a custom name for the application
-                                          on YARN
-     -yq,--yarnquery                      Display available YARN resources
-                                          (memory, cores)
-     -yqu,--yarnqueue <arg>               Specify YARN queue.
-     -ys,--yarnslots <arg>                Number of slots per TaskManager
-     -yt,--yarnship <arg>                 Ship files in the specified directory
-                                          (t for transfer)
-     -ytm,--yarntaskManagerMemory <arg>   Memory per TaskManager Container with
-                                          optional unit (default: MB)
-     -yz,--yarnzookeeperNamespace <arg>   Namespace to create the Zookeeper
-                                          sub-paths for high availability mode
-     -z,--zookeeperNamespace <arg>        Namespace to create the Zookeeper
-                                          sub-paths for high availability mode
-
-  Options for default mode:
-     -m,--jobmanager <arg>           Address of the JobManager to which to
-                                     connect. Use this flag to connect to a
-                                     different JobManager than the one specified
-                                     in the configuration. Attention: This
-                                     option is respected only if the
-                                     high-availability configuration is NONE.
-     -z,--zookeeperNamespace <arg>   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] <jar-file> <arguments>
-  "info" action options:
-     -c,--class <classname>           Class with the program entry point
-                                      ("main()" method). Only needed if the JAR
-                                      file does not specify the class in its
-                                      manifest.
-     -p,--parallelism <parallelism>   The parallelism with which to run the
-                                      program. Optional flag to override the
-                                      default value specified in the
-                                      configuration.
-
-
-Action "list" lists running and scheduled programs.
-
-  Syntax: list [OPTIONS]
-  "list" action options:
-     -a,--all         Show all programs and their JobIDs
-     -r,--running     Show only running programs and their JobIDs
-     -s,--scheduled   Show only scheduled programs and their JobIDs
-  Options for Generic CLI mode:
-     -D <property=value>   Generic configuration options for
-                           execution/deployment and for the configured executor.
-                           The available options can be found at
-                           https://ci.apache.org/projects/flink/flink-docs-stabl
-                           e/deployment/config.html
-     -e,--executor <arg>   DEPRECATED: Please use the -t option instead which is
-                           also available with the "Application Mode".
-                           The name of the executor to be used for executing the
-                           given job, which is equivalent to the
-                           "execution.target" config option. The currently
-                           available executors are: "remote", "local",
-                           "kubernetes-session", "yarn-per-job", "yarn-session".
-     -t,--target <arg>     The deployment target for the given application,
-                           which is equivalent to the "execution.target" config
-                           option. The currently available targets are:
-                           "remote", "local", "kubernetes-session",
-                           "yarn-per-job", "yarn-session", "yarn-application"
-                           and "kubernetes-application".
-
-  Options for yarn-cluster mode:
-     -m,--jobmanager <arg>            Set to yarn-cluster to use YARN execution
-                                      mode.
-     -yid,--yarnapplicationId <arg>   Attach to running YARN session
-     -z,--zookeeperNamespace <arg>    Namespace to create the Zookeeper
-                                      sub-paths for high availability mode
-
-  Options for default mode:
-     -m,--jobmanager <arg>           Address of the JobManager to which to
-                                     connect. Use this flag to connect to a
-                                     different JobManager than the one specified
-                                     in the configuration. Attention: This
-                                     option is respected only if the
-                                     high-availability configuration is NONE.
-     -z,--zookeeperNamespace <arg>   Namespace to create the Zookeeper sub-paths
-                                     for high availability mode
-
-
-
-Action "stop" stops a running program with a savepoint (streaming jobs only).
-
-  Syntax: stop [OPTIONS] <Job ID>
-  "stop" action options:
-     -d,--drain                           Send MAX_WATERMARK before taking the
-                                          savepoint and stopping the pipelne.
-     -p,--savepointPath <savepointPath>   Path to the savepoint (for example
-                                          hdfs:///flink/savepoint-1537). If no
-                                          directory is specified, the configured
-                                          default will be used
-                                          ("state.savepoints.dir").
-  Options for Generic CLI mode:
-     -D <property=value>   Generic configuration options for
-                           execution/deployment and for the configured executor.
-                           The available options can be found at
-                           https://ci.apache.org/projects/flink/flink-docs-stabl
-                           e/deployment/config.html
-     -e,--executor <arg>   DEPRECATED: Please use the -t option instead which is
-                           also available with the "Application Mode".
-                           The name of the executor to be used for executing the
-                           given job, which is equivalent to the
-                           "execution.target" config option. The currently
-                           available executors are: "remote", "local",
-                           "kubernetes-session", "yarn-per-job", "yarn-session".
-     -t,--target <arg>     The deployment target for the given application,
-                           which is equivalent to the "execution.target" config
-                           option. The currently available targets are:
-                           "remote", "local", "kubernetes-session",
-                           "yarn-per-job", "yarn-session", "yarn-application"
-                           and "kubernetes-application".
-
-  Options for yarn-cluster mode:
-     -m,--jobmanager <arg>            Set to yarn-cluster to use YARN execution
-                                      mode.
-     -yid,--yarnapplicationId <arg>   Attach to running YARN session
-     -z,--zookeeperNamespace <arg>    Namespace to create the Zookeeper
-                                      sub-paths for high availability mode
-
-  Options for default mode:
-     -m,--jobmanager <arg>           Address of the JobManager to which to
-                                     connect. Use this flag to connect to a
-                                     different JobManager than the one specified
-                                     in the configuration. Attention: This
-                                     option is respected only if the
-                                     high-availability configuration is NONE.
-     -z,--zookeeperNamespace <arg>   Namespace to create the Zookeeper sub-paths
-                                     for high availability mode
-
-
-
-Action "cancel" cancels a running program.
-
-  Syntax: cancel [OPTIONS] <Job ID>
-  "cancel" action options:
-     -s,--withSavepoint <targetDirectory>   **DEPRECATION WARNING**: Cancelling
-                                            a job with savepoint is deprecated.
-                                            Use "stop" instead.
-                                            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 Generic CLI mode:
-     -D <property=value>   Generic configuration options for
-                           execution/deployment and for the configured executor.
-                           The available options can be found at
-                           https://ci.apache.org/projects/flink/flink-docs-stabl
-                           e/deployment/config.html
-     -e,--executor <arg>   DEPRECATED: Please use the -t option instead which is
-                           also available with the "Application Mode".
-                           The name of the executor to be used for executing the
-                           given job, which is equivalent to the
-                           "execution.target" config option. The currently
-                           available executors are: "remote", "local",
-                           "kubernetes-session", "yarn-per-job", "yarn-session".
-     -t,--target <arg>     The deployment target for the given application,
-                           which is equivalent to the "execution.target" config
-                           option. The currently available targets are:
-                           "remote", "local", "kubernetes-session",
-                           "yarn-per-job", "yarn-session", "yarn-application"
-                           and "kubernetes-application".
-
-  Options for yarn-cluster mode:
-     -m,--jobmanager <arg>            Set to yarn-cluster to use YARN execution
-                                      mode.
-     -yid,--yarnapplicationId <arg>   Attach to running YARN session
-     -z,--zookeeperNamespace <arg>    Namespace to create the Zookeeper
-                                      sub-paths for high availability mode
-
-  Options for default mode:
-     -m,--jobmanager <arg>           Address of the JobManager to which to
-                                     connect. Use this flag to connect to a
-                                     different JobManager than the one specified
-                                     in the configuration. Attention: This
-                                     option is respected only if the
-                                     high-availability configuration is NONE.
-     -z,--zookeeperNamespace <arg>   Namespace to create the Zookeeper sub-paths
-                                     for high availability mode
-
-
-
-Action "savepoint" triggers savepoints for a running job or disposes existing ones.
-
-  Syntax: savepoint [OPTIONS] <Job ID> [<target directory>]
-  "savepoint" action options:
-     -d,--dispose <arg>       Path of savepoint to dispose.
-     -j,--jarfile <jarfile>   Flink program JAR file.
-  Options for Generic CLI mode:
-     -D <property=value>   Generic configuration options for
-                           execution/deployment and for the configured executor.
-                           The available options can be found at
-                           https://ci.apache.org/projects/flink/flink-docs-stabl
-                           e/deployment/config.html
-     -e,--executor <arg>   DEPRECATED: Please use the -t option instead which is
-                           also available with the "Application Mode".
-                           The name of the executor to be used for executing the
-                           given job, which is equivalent to the
-                           "execution.target" config option. The currently
-                           available executors are: "remote", "local",
-                           "kubernetes-session", "yarn-per-job", "yarn-session".
-     -t,--target <arg>     The deployment target for the given application,
-                           which is equivalent to the "execution.target" config
-                           option. The currently available targets are:
-                           "remote", "local", "kubernetes-session",
-                           "yarn-per-job", "yarn-session", "yarn-application"
-                           and "kubernetes-application".
-
-  Options for yarn-cluster mode:
-     -m,--jobmanager <arg>            Set to yarn-cluster to use YARN execution
-                                      mode.
-     -yid,--yarnapplicationId <arg>   Attach to running YARN session
-     -z,--zookeeperNamespace <arg>    Namespace to create the Zookeeper
-                                      sub-paths for high availability mode
-
-  Options for default mode:
-     -m,--jobmanager <arg>           Address of the JobManager to which to
-                                     connect. Use this flag to connect to a
-                                     different JobManager than the one specified
-                                     in the configuration. Attention: This
-                                     option is respected only if the
-                                     high-availability configuration is NONE.
-     -z,--zookeeperNamespace <arg>   Namespace to create the Zookeeper sub-paths
-                                     for high availability mode
-{% endhighlight %}
+## Advanced CLI
+ 
+### REST API
+
+The Flink cluster can be also managed using the [REST API]({% link ops/rest_api.md %}). The commands 
+described in previous sections are a subset of what is offered by Flink's REST endpoints. Therefore, 
+tools like `curl` can be used to get even more out of Flink.
+
+### Selecting Deployment Targets
+
+Flink supports different deployment backends like [Kubernetes]({% link deployment/resource-providers/native_kubernetes.md %}) 
+or [YARN]({% link deployment/resource-providers/yarn.md %}) which are described in more detail in the 
+Resource Provider section. You can specify the mode you're Flink cluster is running in through the 

Review comment:
       We don't have a dedicated resource provider section. That's why I mentioned two examples to be able to at least reference to their subsections.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org