diff --git a/docs/content.zh/docs/concepts/flink-architecture.md b/docs/content.zh/docs/concepts/flink-architecture.md index d0cd8cfcb60b21..e57b06fcd13089 100644 --- a/docs/content.zh/docs/concepts/flink-architecture.md +++ b/docs/content.zh/docs/concepts/flink-architecture.md @@ -97,7 +97,7 @@ _JobManager_ 具有许多与协调 Flink 应用程序的分布式执行有关的 _Flink 应用程序_ 是从其 ``main()`` 方法产生的一个或多个 Flink 作业的任何用户程序。这些作业的执行可以在本地 JVM(`LocalEnvironment`)中进行,或具有多台机器的集群的远程设置(``RemoteEnvironment``)中进行。对于每个程序,`ExecutionEnvironment` 提供了一些方法来控制作业执行(例如设置并行度)并与外界交互(请参考 [Flink 程序剖析]({{< ref "docs/dev/datastream/overview" >}}#anatomy-of-a-flink-program) )。 -Flink 应用程序的作业可以被提交到长期运行的 [Flink Session 集群]({{< ref "docs/concepts/glossary" >}}#flink-session-cluster)、专用的 [Flink Job 集群]({{< ref "docs/concepts/glossary" >}}#flink-job-cluster) 或 [Flink Application 集群]({{< ref "docs/concepts/glossary" >}}#flink-application-cluster)。这些选项之间的差异主要与集群的生命周期和资源隔离保证有关。 +Flink 应用程序的作业可以被提交到长期运行的 [Flink Session 集群]({{< ref "docs/concepts/glossary" >}}#flink-session-cluster) 或 [Flink Application 集群]({{< ref "docs/concepts/glossary" >}}#flink-application-cluster)。这些选项之间的差异主要与集群的生命周期和资源隔离保证有关。 ### Flink Session 集群 @@ -111,21 +111,6 @@ Flink 应用程序的作业可以被提交到长期运行的 [Flink Session 集 以前,Flink Session 集群也被称为 session 模式下的 Flink 集群。 {{< /hint >}} -### Flink Job 集群 - -* **集群生命周期**:在 Flink Job 集群中,可用的集群管理器(例如 YARN)用于为每个提交的作业启动一个集群,并且该集群仅可用于该作业。在这里,客户端首先从集群管理器请求资源启动 JobManager,然后将作业提交给在这个进程中运行的 Dispatcher。然后根据作业的资源请求惰性的分配 TaskManager。一旦作业完成,Flink Job 集群将被拆除。 - -* **资源隔离**:JobManager 中的致命错误仅影响在 Flink Job 集群中运行的一个作业。 - -* **其他注意事项**:由于 ResourceManager 必须应用并等待外部资源管理组件来启动 TaskManager 进程和分配资源,因此 Flink Job 集群更适合长期运行、具有高稳定性要求且对较长的启动时间不敏感的大型作业。 - -{{< hint info >}} -以前,Flink Job 集群也被称为 job (or per-job) 模式下的 Flink 集群。 -{{< /hint >}} -{{< hint info >}} -Kubernetes 不支持 Flink Job 集群。 请参考 [Standalone Kubernetes]({{< ref "docs/deployment/resource-providers/standalone/kubernetes" >}}#per-job-cluster-mode) 和 [Native Kubernetes]({{< ref "docs/deployment/resource-providers/native_kubernetes" >}}#per-job-cluster-mode)。 -{{< /hint >}} - ### Flink Application 集群 * **集群生命周期**:Flink Application 集群是专用的 Flink 集群,仅从 Flink 应用程序执行作业,并且 ``main()``方法在集群上而不是客户端上运行。提交作业是一个单步骤过程:无需先启动 Flink 集群,然后将作业提交到现有的 session 集群;相反,将应用程序逻辑和依赖打包成一个可执行的作业 JAR 中,并且集群入口(``ApplicationClusterEntryPoint``)负责调用 ``main()``方法来提取 JobGraph。例如,这允许你像在 Kubernetes 上部署任何其他应用程序一样部署 Flink 应用程序。因此,Flink Application 集群的寿命与 Flink 应用程序的寿命有关。 diff --git a/docs/content.zh/docs/deployment/cli.md b/docs/content.zh/docs/deployment/cli.md index 8acbbbfb7d1f91..8d2f079af71303 100644 --- a/docs/content.zh/docs/deployment/cli.md +++ b/docs/content.zh/docs/deployment/cli.md @@ -79,7 +79,7 @@ $ export JOB_ID="cca7bc1061d61cf15238e92312c2fc20" The `run` command support passing additional configuration parameters via the `-D` argument. For example setting the [maximum parallelism]({{< ref "docs/deployment/config#pipeline-max-parallelism" >}}#application-mode) for a job can be done by setting `-Dpipeline.max-parallelism=120`. This argument is very useful for -configuring per-job or application mode clusters, because you can pass any configuration parameter +configuring application mode clusters, because you can pass any configuration parameter to the cluster, without changing the configuration file. When submitting a job to an existing session cluster, only [execution configuration parameters]({{< ref "docs/deployment/config#execution" >}}) are supported. @@ -403,13 +403,11 @@ Resource Provider section. Jobs can be submitted in different [Deployment Modes] The parameterization of a job submission differs based on the underlying framework and Deployment Mode. `bin/flink` offers a parameter `--target` to handle the different options. In addition to that, jobs -have to be submitted using `run` (for [Session]({{< ref "docs/deployment/overview" >}}#session-mode), -[Per-Job Mode]({{< ref "docs/deployment/overview" >}}#per-job-mode) and -[Application Mode]({{< ref "docs/deployment/overview" >}}#application-mode)). See the following summary of +have to be submitted using `run` (for [Session]({{< ref "docs/deployment/overview" >}}#session-mode) +and [Application Mode]({{< ref "docs/deployment/overview" >}}#application-mode)). See the following summary of parameter combinations: * YARN * `./bin/flink run --target yarn-session`: Submission to an already running Flink on YARN cluster - * `./bin/flink run --target yarn-per-job`: Submission spinning up a Flink on YARN cluster in Per-Job Mode * `./bin/flink run --target yarn-application`: Submission spinning up Flink on YARN cluster in Application Mode * Kubernetes * `./bin/flink run --target kubernetes-session`: Submission to an already running Flink on Kubernetes cluster @@ -474,13 +472,6 @@ $ ./bin/flink run \ --python examples/python/table/word_count.py ``` -- Run a PyFlink job using a [YARN cluster in Per-Job Mode]({{< ref "docs/deployment/resource-providers/yarn" >}}#per-job-cluster-mode): -```bash -$ ./bin/flink run \ - --target yarn-per-job - --python examples/python/table/word_count.py -``` - - Run a PyFlink job using a [YARN cluster in Application Mode]({{< ref "docs/deployment/resource-providers/yarn" >}}#application-mode): ```bash $ ./bin/flink run -t yarn-application \ diff --git a/docs/content.zh/docs/deployment/overview.md b/docs/content.zh/docs/deployment/overview.md index 9be46a0b60198b..e3d5e0168e9578 100644 --- a/docs/content.zh/docs/deployment/overview.md +++ b/docs/content.zh/docs/deployment/overview.md @@ -74,7 +74,6 @@ When deploying Flink, there are often multiple options available for each buildi JobManager modes for job submissions: @@ -173,7 +172,6 @@ covered by [FLINK-26606](https://issues.apache.org/jira/browse/FLINK-26606). Flink can execute applications in one of three ways: - in Application Mode, -- in a Per-Job Mode, - in Session Mode. The above modes differ in: @@ -182,7 +180,7 @@ Flink can execute applications in one of three ways: -{{< img class="img-fluid" width="80%" style="margin: 15px" src="/fig/deployment_modes.svg" alt="Figure for Deployment Modes" >}} +{{< img class="img-fluid" width="70%" style="margin: 15px" src="/fig/deployment_modes.png" alt="Figure for Deployment Modes" >}} #### Application Mode @@ -196,8 +194,8 @@ network bandwidth to download dependencies and ship binaries to the cluster, and Building on this observation, the *Application Mode* creates a cluster per submitted application, but this time, the `main()` method of the application is executed on the JobManager. Creating a cluster per application can be seen as creating a session cluster shared only among the jobs of a particular application, and torn down when -the application finishes. With this architecture, the *Application Mode* provides the same resource isolation -and load balancing guarantees as the *Per-Job* mode, but at the granularity of a whole application. Executing +the application finishes. With this architecture, the *Application Mode* provides the application granularity resource isolation +and load balancing guarantees. Executing the `main()` on the JobManager allows for saving the CPU cycles required, but also save the bandwidth required for downloading the dependencies locally. Furthermore, it allows for more even spread of the network load for downloading the dependencies of the applications in the cluster, as there is one JobManager per application. @@ -208,7 +206,7 @@ as in the other modes. This may have implications for your code as, for example, your environment using the `registerCachedFile()` must be accessible by the JobManager of your application. {{< /hint >}} -Compared to the *Per-Job* mode, the *Application Mode* allows the submission of applications consisting of +The *Application Mode* allows the submission of applications consisting of multiple jobs. The order of job execution is not affected by the deployment mode but by the call used to launch the job. Using `execute()`, which is blocking, establishes an order and it will lead to the execution of the "next" job being postponed until "this" job finishes. Using `executeAsync()`, which is @@ -224,16 +222,6 @@ Additionally, when any of multiple running jobs in Application Mode (submitted f Regular job completions (by the sources shutting down) are supported. {{< /hint >}} -#### Per-Job Mode - -Aiming at providing better resource isolation guarantees, the *Per-Job* mode uses the available resource provider -framework (e.g. YARN, Kubernetes) to spin up a cluster for each submitted job. This cluster is available to -that job only. When the job finishes, the cluster is torn down and any lingering resources (files, etc) are -cleared up. This provides better resource isolation, as a misbehaving job can only bring down its own -TaskManagers. In addition, it spreads the load of book-keeping across multiple JobManagers, as there is -one per job. For these reasons, the *Per-Job* resource allocation model is the preferred mode by many -production reasons. - #### Session Mode *Session mode* assumes an already running cluster and uses the resources of that cluster to execute any @@ -250,9 +238,7 @@ is responsible for the book-keeping of all the jobs in the cluster. #### Summary In *Session Mode*, the cluster lifecycle is independent of that of any job running on the cluster -and the resources are shared across all jobs. The *Per-Job* mode pays the price of spinning up a cluster -for every submitted job, but this comes with better isolation guarantees as the resources are not shared -across jobs. In this case, the lifecycle of the cluster is bound to that of the job. Finally, the +and the resources are shared across all jobs. The *Application Mode* creates a session cluster per application and executes the application's `main()` method on the cluster. diff --git a/docs/content.zh/docs/deployment/resource-providers/native_kubernetes.md b/docs/content.zh/docs/deployment/resource-providers/native_kubernetes.md index 2a19c84f47c354..92d5199d559abd 100644 --- a/docs/content.zh/docs/deployment/resource-providers/native_kubernetes.md +++ b/docs/content.zh/docs/deployment/resource-providers/native_kubernetes.md @@ -179,10 +179,6 @@ $ ./bin/flink cancel --target kubernetes-application -Dkubernetes.cluster-id=my- You can override configurations set in [Flink configuration file]({{< ref "docs/deployment/config#flink-配置文件" >}}) by passing key-value pairs `-Dkey=value` to `bin/flink`. -### Per-Job Cluster Mode - -Flink on Kubernetes does not support Per-Job Cluster Mode. - ### Session Mode You have seen the deployment of a Session cluster in the [Getting Started](#getting-started) guide at the top of this page. diff --git a/docs/content.zh/docs/deployment/resource-providers/standalone/kubernetes.md b/docs/content.zh/docs/deployment/resource-providers/standalone/kubernetes.md index c357acd0a0228d..9a4d9dd37c7dec 100644 --- a/docs/content.zh/docs/deployment/resource-providers/standalone/kubernetes.md +++ b/docs/content.zh/docs/deployment/resource-providers/standalone/kubernetes.md @@ -136,12 +136,6 @@ $ ./bin/flink run -m localhost:8081 ./examples/streaming/TopSpeedWindowing.jar $ kubectl delete -f jobmanager-application-non-ha.yaml ``` - - -### Per-Job 集群模式 - -在 Kubernetes 上部署 Standalone 集群时不支持 Per-Job 集群模式。 - ### Session 集群模式 diff --git a/docs/content.zh/docs/deployment/resource-providers/yarn.md b/docs/content.zh/docs/deployment/resource-providers/yarn.md index 21b24055558ce8..46f4af2600f54e 100644 --- a/docs/content.zh/docs/deployment/resource-providers/yarn.md +++ b/docs/content.zh/docs/deployment/resource-providers/yarn.md @@ -83,7 +83,7 @@ Congratulations! You have successfully run a Flink application by deploying Flin ## Deployment Modes Supported by Flink on YARN -For production use, we recommend deploying Flink Applications in the [Per-job or Application Mode]({{< ref "docs/deployment/overview" >}}#deployment-modes), as these modes provide a better isolation for the Applications. +For production use, we recommend deploying Flink Applications in the [Application Mode]({{< ref "docs/deployment/overview" >}}#deployment-modes), as these modes provide a better isolation for the Applications. ### Application Mode @@ -120,28 +120,6 @@ The above will allow the job submission to be extra lightweight as the needed Fl are going to be picked up by the specified remote locations rather than be shipped to the cluster by the client. -### Per-Job Cluster Mode - -The Per-job Cluster mode will launch a Flink cluster on YARN, then run the provided application jar locally and finally submit the JobGraph to the JobManager on YARN. If you pass the `--detached` argument, the client will stop once the submission is accepted. - -The YARN cluster will stop once the job has stopped. - -```bash -./bin/flink run -t yarn-per-job --detached ./examples/streaming/TopSpeedWindowing.jar -``` - -Once a Per-Job Cluster is deployed, you can interact with it for operations like cancelling or taking a savepoint. - -```bash -# List running job on the cluster -./bin/flink list -t yarn-per-job -Dyarn.application.id=application_XXXX_YY -# Cancel running job -./bin/flink cancel -t yarn-per-job -Dyarn.application.id=application_XXXX_YY -``` - -Note that cancelling your job on an Per-Job Cluster will stop the cluster. - - ### Session Mode We describe deployment with the Session Mode in the [Getting Started](#getting-started) guide at the top of the page. @@ -241,9 +219,9 @@ The configuration parameter for specifying the REST endpoint port is [rest.bind- When deploying Flink with Session Mode on Yarn, only the JAR file specified in startup command will be recognized as user-jars and included into user classpath. -**PerJob Mode & Application Mode** +**Application Mode** -When deploying Flink with PerJob/Application Mode on Yarn, the JAR file specified in startup command and all JAR files in Flink's `usrlib` folder will be recognized as user-jars. +When deploying Flink with Application Mode on Yarn, the JAR file specified in startup command and all JAR files in Flink's `usrlib` folder will be recognized as user-jars. By default Flink will include the user-jars into the system classpath. This behavior can be controlled with the [yarn.classpath.include-user-jar]({{< ref "docs/deployment/config" >}}#yarn-classpath-include-user-jar) parameter. When setting this to `DISABLED` Flink will include the jar in the user classpath instead. diff --git a/docs/content.zh/docs/ops/debugging/debugging_classloading.md b/docs/content.zh/docs/ops/debugging/debugging_classloading.md index df55db4c2c5166..cc36a2449e104a 100644 --- a/docs/content.zh/docs/ops/debugging/debugging_classloading.md +++ b/docs/content.zh/docs/ops/debugging/debugging_classloading.md @@ -58,19 +58,11 @@ created for an job/application and will contain the job/application's jar files. --> -**Per-Job模式(已弃用)(Yarn)** - -当前只有Yarn支持Per-Job模式。默认情况下,Flink集群运行在Per-Job模式下时会将用户的jar文件包含在系统的classpath中。 -这种模式可以由[yarn.classpath.include-user-jar]({{< ref "docs/deployment/config" >}}#yarn-classpath-include-user-jar) 参数控制。 -当该参数设定为`DISABLED`时,Flink会将用户jar文件含在用户的classpath中,并由*FlinkUserCodeClassLoader*进行动态加载。 - -详细信息参见[Flink on Yarn]({{< ref "docs/deployment/resource-providers/yarn" >}})。 - **Application模式(Standalone/Yarn/Kubernetes)** 当Application模式的Flink集群基于Standalone或Kubernetes方式运行时,用户jar文件(启动命令指定的jar文件和Flink的`usrlib`目录中的jar包)会由*FlinkUserCodeClassLoader*进行动态加载。 -当Flink集群以Application模式运行时,用户jar文件(启动命令指定的jar文件和Flink的`usrlib`目录中的jar包)默认情况下会包含在系统classpath(*AppClassLoader*)。与Per-Job模式相同,当[yarn.classpath.include-user-jar]({{< ref "docs/deployment/config" >}}#yarn-classpath-include-user-jar)设置为`DISABLED`时,Flink会将用户jar文件含在用户的classpath中,并由*FlinkUserCodeClassLoader*进行动态加载。 +当Flink集群以Application模式运行时,用户jar文件(启动命令指定的jar文件和Flink的`usrlib`目录中的jar包)默认情况下会包含在系统classpath(*AppClassLoader*)。当[yarn.classpath.include-user-jar]({{< ref "docs/deployment/config" >}}#yarn-classpath-include-user-jar)设置为`DISABLED`时,Flink会将用户jar文件含在用户的classpath中,并由*FlinkUserCodeClassLoader*进行动态加载。 ## 倒置类加载(Inverted Class Loading)和ClassLoader解析顺序 diff --git a/docs/content/docs/concepts/flink-architecture.md b/docs/content/docs/concepts/flink-architecture.md index 722bffe1d72ad7..458f0b03038ee0 100644 --- a/docs/content/docs/concepts/flink-architecture.md +++ b/docs/content/docs/concepts/flink-architecture.md @@ -217,35 +217,4 @@ isolation guarantees. Formerly, a Flink Session Cluster was also known as a Flink Cluster in `session mode`. {{< /hint >}} -### Flink Job Cluster (deprecated) - -{{< hint danger >}} -Per-job mode is only supported by YARN and has been deprecated in Flink 1.15. -It will be dropped in [FLINK-26000](https://issues.apache.org/jira/browse/FLINK-26000). -Please consider application mode to launch a dedicated cluster per-job on YARN. -{{< /hint >}} - -* **Cluster Lifecycle**: in a Flink Job Cluster, the available cluster manager - (like YARN) is used to spin up a cluster for each submitted job - and this cluster is available to that job only. Here, the client first - requests resources from the cluster manager to start the JobManager and - submits the job to the Dispatcher running inside this process. TaskManagers - are then lazily allocated based on the resource requirements of the job. Once - the job is finished, the Flink Job Cluster is torn down. - -* **Resource Isolation**: a fatal error in the JobManager only affects the one job running in that Flink Job Cluster. - -* **Other considerations**: because the ResourceManager has to apply and wait - for external resource management components to start the TaskManager - processes and allocate resources, Flink Job Clusters are more suited to large - jobs that are long-running, have high-stability requirements and are not - sensitive to longer startup times. - -{{< hint info >}} -Formerly, a Flink Job Cluster was also known as a Flink Cluster in `job (or per-job) mode`. -{{< /hint >}} -{{< hint info >}} -Flink Job Clusters are only supported with YARN. -{{< /hint >}} - {{< top >}} diff --git a/docs/content/docs/deployment/cli.md b/docs/content/docs/deployment/cli.md index 25f08e89387fb5..9250704ef4174b 100644 --- a/docs/content/docs/deployment/cli.md +++ b/docs/content/docs/deployment/cli.md @@ -400,10 +400,9 @@ Flink is compatible with multiple cluster management frameworks like Resource Provider section. Jobs can be submitted in different [Deployment Modes]({{< ref "docs/deployment/overview" >}}#deployment-modes). The parameterization of a job submission differs based on the underlying framework and Deployment Mode. -`bin/flink` offers a parameter `--target` to handle the different options. In addition to that, jobs -have to be submitted using either `run` (for [Session]({{< ref "docs/deployment/overview" >}}#session-mode) -and [Per-Job Mode (deprecated)]({{< ref "docs/deployment/overview" >}}#per-job-mode)) or `run-application` (for -[Application Mode]({{< ref "docs/deployment/overview" >}}#application-mode)). See the following summary of +`bin/flink` offers a parameter `--target` to handle the different options. In addition to that, jobs +have to be submitted using `run` (for [Session]({{< ref "docs/deployment/overview" >}}#session-mode) +and [Application Mode]({{< ref "docs/deployment/overview" >}}#application-mode)). See the following summary of parameter combinations: * YARN * `./bin/flink run --target yarn-session`: Submission to an already running Flink on YARN cluster @@ -471,13 +470,6 @@ $ ./bin/flink run \ --python examples/python/table/word_count.py ``` -- Run a PyFlink job using a [YARN cluster in Per-Job Mode]({{< ref "docs/deployment/resource-providers/yarn" >}}#per-job-cluster-mode): -```bash -$ ./bin/flink run \ - --target yarn-per-job - --python examples/python/table/word_count.py -``` - - Run a PyFlink job using a [YARN cluster in Application Mode]({{< ref "docs/deployment/resource-providers/yarn" >}}#application-mode): ```bash $ ./bin/flink run -t yarn-application \ diff --git a/docs/content/docs/deployment/overview.md b/docs/content/docs/deployment/overview.md index f87264d0b00cab..79e22a25c9b254 100644 --- a/docs/content/docs/deployment/overview.md +++ b/docs/content/docs/deployment/overview.md @@ -74,7 +74,6 @@ When deploying Flink, there are often multiple options available for each buildi JobManager modes for job submissions: @@ -175,15 +174,13 @@ covered by [FLINK-26606](https://issues.apache.org/jira/browse/FLINK-26606). Flink can execute applications in one of three ways: - in Application Mode, - in Session Mode, -- in a Per-Job Mode (deprecated). The above modes differ in: - the cluster lifecycle and resource isolation guarantees - whether the application's `main()` method is executed on the client or on the cluster. - -{{< img class="img-fluid" width="100%" style="margin: 15px" src="/fig/deployment_modes.svg" alt="Figure for Deployment Modes" >}} +{{< img class="img-fluid" width="70%" style="margin: 10px" src="/fig/deployment_modes.png" alt="Figure for Deployment Modes" >}} ### Application Mode @@ -197,8 +194,8 @@ network bandwidth to download dependencies and ship binaries to the cluster, and Building on this observation, the *Application Mode* creates a cluster per submitted application, but this time, the `main()` method of the application is executed by the *JobManager*. Creating a cluster per application can be seen as creating a session cluster shared only among the jobs of a particular application, and turning down when -the application finishes. With this architecture, the *Application Mode* provides the same resource isolation -and load balancing guarantees as the *Per-Job* mode, but at the granularity of a whole application. +the application finishes. With this architecture, the *Application Mode* provides the application granularity resource isolation +and load balancing guarantees. The *Application Mode* builds on an assumption that the user jars are already available on the classpath (`usrlib` folder) of all Flink components that needs access to it (*JobManager*, *TaskManager*). In other words, your application comes @@ -212,7 +209,7 @@ Executing the `main()` method on the cluster may have other implications for you in your environment using the `registerCachedFile()` must be accessible by the JobManager of your application. {{< /hint >}} -Compared to the *Per-Job (deprecated)* mode, the *Application Mode* allows the submission of applications consisting of +The *Application Mode* allows the submission of applications consisting of multiple jobs. The order of job execution is not affected by the deployment mode but by the call used to launch the job. Using `execute()`, which is blocking, establishes an order and it will lead to the execution of the "next" job being postponed until "this" job finishes. Using `executeAsync()`, which is @@ -240,21 +237,6 @@ restarting jobs accessing the filesystem concurrently and making it unavailable Additionally, having a single cluster running multiple jobs implies more load for the JobManager, who is responsible for the book-keeping of all the jobs in the cluster. -### Per-Job Mode (deprecated) - -{{< hint danger >}} -Per-job mode is only supported by YARN and has been deprecated in Flink 1.15. -It will be dropped in [FLINK-26000](https://issues.apache.org/jira/browse/FLINK-26000). -Please consider application mode to launch a dedicated cluster per-job on YARN. -{{< /hint >}} - -Aiming at providing better resource isolation guarantees, the *Per-Job* mode uses the available resource provider -framework (e.g. YARN) to spin up a cluster for each submitted job. This cluster is available to -that job only. When the job finishes, the cluster is torn down and any lingering resources (files, etc) are -cleared up. This provides better resource isolation, as a misbehaving job can only bring down its own -TaskManagers. In addition, it spreads the load of book-keeping across multiple JobManagers, as there is -one per job. - ### Summary In *Session Mode*, the cluster lifecycle is independent of that of any job running on the cluster diff --git a/docs/content/docs/deployment/resource-providers/yarn.md b/docs/content/docs/deployment/resource-providers/yarn.md index 7f8e3ddb0d2605..20f45c539720cb 100644 --- a/docs/content/docs/deployment/resource-providers/yarn.md +++ b/docs/content/docs/deployment/resource-providers/yarn.md @@ -157,37 +157,6 @@ The YARN session client also has a few "shortcut arguments" for commonly used se {{< top >}} -### Per-Job Mode (deprecated) - -{{< hint danger >}} -Per-job mode is only supported by YARN and has been deprecated in Flink 1.15. -It will be dropped in [FLINK-26000](https://issues.apache.org/jira/browse/FLINK-26000). -Please consider application mode to launch a dedicated cluster per-job on YARN. -{{< /hint >}} - -{{< hint info >}} -For high-level intuition behind the per-job mode, please refer to the [deployment mode overview]({{< ref "docs/deployment/overview#per-job-mode" >}}). -{{< /hint >}} - -The Per-job Cluster mode will launch a Flink cluster on YARN, then run the provided application jar locally and finally submit the JobGraph to the JobManager on YARN. If you pass the `--detached` argument, the client will stop once the submission is accepted. - -The YARN cluster will stop once the job has stopped. - -```bash -./bin/flink run -t yarn-per-job --detached ./examples/streaming/TopSpeedWindowing.jar -``` - -Once a Per-Job Cluster is deployed, you can interact with it for operations like cancelling or taking a savepoint. - -```bash -# List running job on the cluster -./bin/flink list -t yarn-per-job -Dyarn.application.id=application_XXXX_YY -# Cancel running job -./bin/flink cancel -t yarn-per-job -Dyarn.application.id=application_XXXX_YY -``` - -Note that cancelling your job on an Per-Job Cluster will stop the cluster. - ## Flink on YARN Reference ### Configuring Flink on YARN @@ -257,9 +226,9 @@ The configuration parameter for specifying the REST endpoint port is [rest.bind- When deploying Flink with Session Mode on Yarn, only the JAR file specified in startup command will be recognized as user-jars and included into user classpath. -**PerJob Mode & Application Mode** +**Application Mode** -When deploying Flink with PerJob/Application Mode on Yarn, the JAR file specified in startup command and all JAR files in Flink's `usrlib` folder will be recognized as user-jars. +When deploying Flink with Application Mode on Yarn, the JAR file specified in startup command and all JAR files in Flink's `usrlib` folder will be recognized as user-jars. By default Flink will include the user-jars into the system classpath. This behavior can be controlled with the [yarn.classpath.include-user-jar]({{< ref "docs/deployment/config" >}}#yarn-classpath-include-user-jar) parameter. When setting this to `DISABLED` Flink will include the jar in the user classpath instead. diff --git a/docs/content/docs/ops/debugging/debugging_classloading.md b/docs/content/docs/ops/debugging/debugging_classloading.md index 27d25de8e4eec1..99da741e92b6aa 100644 --- a/docs/content/docs/ops/debugging/debugging_classloading.md +++ b/docs/content/docs/ops/debugging/debugging_classloading.md @@ -61,21 +61,13 @@ created for an job/application and will contain the job/application's jar files. --> -**Per-Job Mode (deprecated) (Yarn)** - -Currently, only Yarn supports Per-Job mode. By default, running a Flink cluster in Per-Job mode will include the user jars -(the JAR file specified in startup command and all JAR files in Flink's `usrlib` folder) into the system classpath (the *AppClassLoader*). -This behavior can be controlled with the [yarn.classpath.include-user-jar]({{< ref "docs/deployment/config" >}}#yarn-classpath-include-user-jar) config option. -When setting it to `DISABLED`, Flink will include the user jars in the user classpath and load them dynamically by *FlinkUserCodeClassLoader*. -See [Flink on Yarn]({{< ref "docs/deployment/resource-providers/yarn" >}}) for more details. - **Application Mode (Standalone/Yarn/Kubernetes)** When run a Standalone/Kubernetes Flink cluster in Application Mode, the user jars (the JAR file specified in startup command and all JAR files in Flink's `usrlib` folder) will be loaded dynamically by *FlinkUserCodeClassLoader*. When run a Yarn Flink cluster in Application Mode, the user jars (the JAR file specified in startup command and all JAR files in Flink's `usrlib` folder) -will be included into the system classpath (the *AppClassLoader*) by default. Same as Per-Job mode, when setting the [yarn.classpath.include-user-jar]({{< ref "docs/deployment/config" >}}#yarn-classpath-include-user-jar) +will be included into the system classpath (the *AppClassLoader*) by default. When setting the [yarn.classpath.include-user-jar]({{< ref "docs/deployment/config" >}}#yarn-classpath-include-user-jar) to `DISABLED`, Flink will include the user jars in the user classpath and load them dynamically by *FlinkUserCodeClassLoader*. ## Inverted Class Loading and ClassLoader Resolution Order diff --git a/docs/layouts/shortcodes/generated/deployment_configuration.html b/docs/layouts/shortcodes/generated/deployment_configuration.html index eef1e42672b29d..bfae1eb18666a3 100644 --- a/docs/layouts/shortcodes/generated/deployment_configuration.html +++ b/docs/layouts/shortcodes/generated/deployment_configuration.html @@ -60,7 +60,7 @@
execution.target
(none) String - The deployment target for the execution. This can take one of the following values when calling bin/flink run: + The deployment target for the execution. This can take one of the following values when calling bin/flink run: diff --git a/docs/static/fig/deployment_modes.svg b/docs/static/fig/deployment_modes.svg deleted file mode 100644 index 7ed73edca10847..00000000000000 --- a/docs/static/fig/deployment_modes.svg +++ /dev/null @@ -1,19 +0,0 @@ - - diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/GenericCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/GenericCLI.java index 3a11d5078755f8..2a8f76f26d88a5 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/GenericCLI.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/GenericCLI.java @@ -132,20 +132,12 @@ private static String getTargetNames() { final Stream executorNames = new DefaultExecutorServiceLoader() .getExecutorNames() - .map(name -> String.format("\"%s\"", name)) - .map(GenericCLI::addDeprecationNoticeToYarnPerJobMode); + .map(name -> String.format("\"%s\"", name)); return Stream.concat(executorNames, getApplicationModeTargetNames().stream()) .collect(Collectors.joining(DELIMITER)); } - private static String addDeprecationNoticeToYarnPerJobMode(String name) { - if (name.contains("yarn-per-job")) { - return name + " (deprecated)"; - } - return name; - } - private static List getApplicationModeTargetNames() { return new DefaultClusterClientServiceLoader() .getApplicationModeTargetNames() diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java index fa741e1f7f14f1..f001c3805b11e6 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java @@ -20,7 +20,6 @@ import org.apache.flink.client.deployment.application.ApplicationConfiguration; import org.apache.flink.client.program.ClusterClientProvider; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.util.FlinkException; /** @@ -70,26 +69,6 @@ ClusterClientProvider deployApplicationCluster( final ApplicationConfiguration applicationConfiguration) throws ClusterDeploymentException; - /** - * Deploys a per-job cluster with the given job on the cluster. - * - * @param clusterSpecification Initial cluster specification with which the Flink cluster is - * launched - * @param jobGraph JobGraph with which the job cluster is started - * @param detached true if the cluster should be stopped after the job completion without - * serving the result, otherwise false - * @return Cluster client to talk to the Flink cluster - * @deprecated Per-job mode has been deprecated in Flink 1.15 and will be removed in the future. - * Please use application mode instead. - * @throws ClusterDeploymentException if the cluster could not be deployed - */ - @Deprecated - ClusterClientProvider deployJobCluster( - final ClusterSpecification clusterSpecification, - final JobGraph jobGraph, - final boolean detached) - throws ClusterDeploymentException; - /** * Terminates the cluster with the given cluster id. * diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java index ce041704974cc4..87f2abce8e9e74 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java @@ -23,7 +23,6 @@ import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; @@ -69,13 +68,6 @@ public ClusterClientProvider deployApplicationCluster( "Application Mode not supported by standalone deployments."); } - @Override - public ClusterClientProvider deployJobCluster( - ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) { - throw new UnsupportedOperationException( - "Per-Job Mode not supported by standalone deployments."); - } - @Override public void killCluster(StandaloneClusterId clusterId) throws FlinkException { throw new UnsupportedOperationException("Cannot terminate a standalone cluster."); diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractJobClusterExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractJobClusterExecutor.java deleted file mode 100644 index f43e9ef46f9755..00000000000000 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractJobClusterExecutor.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.client.deployment.executors; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.dag.Pipeline; -import org.apache.flink.client.cli.ExecutionConfigAccessor; -import org.apache.flink.client.deployment.ClusterClientFactory; -import org.apache.flink.client.deployment.ClusterClientJobClientAdapter; -import org.apache.flink.client.deployment.ClusterDescriptor; -import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.client.program.ClusterClientProvider; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.core.execution.PipelineExecutor; -import org.apache.flink.runtime.jobgraph.JobGraph; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nonnull; - -import java.util.concurrent.CompletableFuture; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * An abstract {@link PipelineExecutor} used to execute {@link Pipeline pipelines} on dedicated - * (per-job) clusters. - * - * @param the type of the id of the cluster. - * @param the type of the {@link ClusterClientFactory} used to create/retrieve a - * client to the target cluster. - */ -@Internal -@Deprecated -public class AbstractJobClusterExecutor< - ClusterID, ClientFactory extends ClusterClientFactory> - implements PipelineExecutor { - - private static final Logger LOG = LoggerFactory.getLogger(AbstractJobClusterExecutor.class); - - private final ClientFactory clusterClientFactory; - - public AbstractJobClusterExecutor(@Nonnull final ClientFactory clusterClientFactory) { - this.clusterClientFactory = checkNotNull(clusterClientFactory); - } - - @Override - public CompletableFuture execute( - @Nonnull final Pipeline pipeline, - @Nonnull final Configuration configuration, - @Nonnull final ClassLoader userCodeClassloader) - throws Exception { - final JobGraph jobGraph = - PipelineExecutorUtils.getJobGraph(pipeline, configuration, userCodeClassloader); - - try (final ClusterDescriptor clusterDescriptor = - clusterClientFactory.createClusterDescriptor(configuration)) { - final ExecutionConfigAccessor configAccessor = - ExecutionConfigAccessor.fromConfiguration(configuration); - - final ClusterSpecification clusterSpecification = - clusterClientFactory.getClusterSpecification(configuration); - - final ClusterClientProvider clusterClientProvider = - clusterDescriptor.deployJobCluster( - clusterSpecification, jobGraph, configAccessor.getDetachedMode()); - LOG.info("Job has been submitted with JobID " + jobGraph.getJobID()); - - return CompletableFuture.completedFuture( - new ClusterClientJobClientAdapter<>( - clusterClientProvider, jobGraph.getJobID(), userCodeClassloader)); - } - } -} diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java index 0eff4127927ad7..2143199628f3ae 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java @@ -23,7 +23,6 @@ import org.apache.flink.client.deployment.application.ApplicationConfiguration; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.ClusterClientProvider; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.util.Preconditions; /** Dummy {@link ClusterDescriptor} implementation for testing purposes. */ @@ -58,12 +57,6 @@ public ClusterClientProvider deployApplicationCluster( return () -> clusterClient; } - @Override - public ClusterClientProvider deployJobCluster( - ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) { - return () -> clusterClient; - } - @Override public void killCluster(T clusterId) { throw new UnsupportedOperationException("Cannot terminate a dummy cluster."); diff --git a/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java index ff273d82695907..bd422a629ade31 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java @@ -46,7 +46,6 @@ public class DeploymentOptions { text("remote"), text("local"), text("yarn-application"), - text("yarn-per-job (deprecated)"), text("yarn-session"), text("kubernetes-application"), text("kubernetes-session")) diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 963903c3257e5c..6cff66e293f49f 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -144,8 +144,6 @@ function run_group_1 { # Hadoop YARN doesn't support aarch64 at this moment. See: https://issues.apache.org/jira/browse/HADOOP-16723 # These tests are known to fail on JDK11. See FLINK-13719 # if [[ ${PROFILE} != *"jdk11"* ]]; then - # run_test "Running Kerberized YARN per-job on Docker test (default input)" "$END_TO_END_DIR/test-scripts/test_yarn_job_kerberos_docker.sh" - # run_test "Running Kerberized YARN per-job on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_yarn_job_kerberos_docker.sh dummy-fs" # run_test "Running Kerberized YARN application on Docker test (default input)" "$END_TO_END_DIR/test-scripts/test_yarn_application_kerberos_docker.sh" # run_test "Running Kerberized YARN application on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_yarn_application_kerberos_docker.sh dummy-fs" # fi @@ -220,7 +218,7 @@ function run_group_2 { # Disable the test as we use JDK11 by default. We should enable it once we use the yarn docker image with JDK 11. # These tests are known to fail on JDK11. See FLINK-13719 # if [[ ${PROFILE} != *"jdk11"* ]] && [[ `uname -i` != 'aarch64' ]]; then - # run_test "PyFlink YARN per-job on Docker test" "$END_TO_END_DIR/test-scripts/test_pyflink_yarn.sh" "skip_check_exceptions" + # run_test "PyFlink YARN application on Docker test" "$END_TO_END_DIR/test-scripts/test_pyflink_yarn.sh" "skip_check_exceptions" # fi ################################################################################ diff --git a/flink-end-to-end-tests/test-scripts/test_pyflink_yarn.sh b/flink-end-to-end-tests/test-scripts/test_pyflink_yarn.sh index 2a5e8375b7d349..7cdbf46b838bcc 100755 --- a/flink-end-to-end-tests/test-scripts/test_pyflink_yarn.sh +++ b/flink-end-to-end-tests/test-scripts/test_pyflink_yarn.sh @@ -59,7 +59,9 @@ zip -q -r /tmp/venv.zip .conda docker exec master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && \ export PYFLINK_CLIENT_EXECUTABLE=/tmp/.conda/bin/python && \ - /home/hadoop-user/$FLINK_DIRNAME/bin/flink run -m yarn-cluster -ytm 1500 -yjm 1000 \ + /home/hadoop-user/$FLINK_DIRNAME/bin/flink run -t yarn-application \ + -Djobmanager.memory.process.size=1500m \ + -Dtaskmanager.memory.process.size=1000m \ -pyfs /tmp/add_one.py \ -pyreq /tmp/requirements.txt \ -pyarch /tmp/venv.zip \ @@ -68,7 +70,9 @@ docker exec master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && \ docker exec master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && \ export PYFLINK_CLIENT_EXECUTABLE=/tmp/.conda/bin/python && \ - /home/hadoop-user/$FLINK_DIRNAME/bin/flink run -m yarn-cluster -ytm 1500 -yjm 1000 \ + /home/hadoop-user/$FLINK_DIRNAME/bin/flink run -t yarn-application \ + -Djobmanager.memory.process.size=1500m \ + -Dtaskmanager.memory.process.size=1000m \ -pyfs /tmp/add_one.py \ -pyreq /tmp/requirements.txt \ -pyarch /tmp/venv.zip \ diff --git a/flink-end-to-end-tests/test-scripts/test_yarn_job_kerberos_docker.sh b/flink-end-to-end-tests/test-scripts/test_yarn_job_kerberos_docker.sh deleted file mode 100644 index 37e839103a9dcd..00000000000000 --- a/flink-end-to-end-tests/test-scripts/test_yarn_job_kerberos_docker.sh +++ /dev/null @@ -1,85 +0,0 @@ -#!/usr/bin/env bash -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -source "$(dirname "$0")"/common.sh -source "$(dirname "$0")"/common_yarn_docker.sh - -# Configure Flink dir before making tarball. -INPUT_TYPE=${1:-default-input} -case $INPUT_TYPE in - (default-input) - INPUT_ARGS="" - ;; - (dummy-fs) - source "$(dirname "$0")"/common_dummy_fs.sh - dummy_fs_setup - INPUT_ARGS="--input dummy://localhost/words --input anotherDummy://localhost/words" - ;; - (*) - echo "Unknown input type ${INPUT_TYPE}" - exit 1 - ;; -esac - -start_hadoop_cluster_and_prepare_flink - -# make the output path random, just in case it already exists, for example if we -# had cached docker containers -OUTPUT_PATH=hdfs:///user/hadoop-user/wc-out-$RANDOM - -# it's important to run this with higher parallelism, otherwise we might risk that -# JM and TM are on the same YARN node and that we therefore don't test the keytab shipping -if docker exec master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && \ - /home/hadoop-user/${FLINK_DIRNAME}/bin/flink run -m yarn-cluster -ys 1 -ytm 1000 -yjm 1000 -p 3 \ - -yD taskmanager.memory.jvm-metaspace.size=128m \ - /home/hadoop-user/${FLINK_DIRNAME}/examples/streaming/WordCount.jar ${INPUT_ARGS} --output ${OUTPUT_PATH}"; -then - OUTPUT=$(get_output "${OUTPUT_PATH}/*/*") - echo "==== OUTPUT_BEGIN ====" - echo "$OUTPUT" - echo "==== OUTPUT_END ====" - - YARN_APPLICATION_LOGS=$(get_yarn_application_logs) - if [[ ! "${YARN_APPLICATION_LOGS}" =~ "Receive initial delegation tokens from resource manager" ]]; then - echo "YARN logs does not contain delegation token usage message as required" - exit 1 - fi -else - echo "Running the job failed." - exit 1 -fi - -echo "Running Job without configured keytab, the exception you see below is expected" -docker exec master bash -c "echo \"\" > /home/hadoop-user/${FLINK_DIRNAME}/conf/config.yaml" -# verify that it doesn't work if we don't configure a keytab -docker exec master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && \ - /home/hadoop-user/${FLINK_DIRNAME}/bin/flink run \ - -m yarn-cluster -ys 1 -ytm 1000 -yjm 1000 -p 3 \ - -yD taskmanager.memory.jvm-metaspace.size=128m \ - /home/hadoop-user/${FLINK_DIRNAME}/examples/streaming/WordCount.jar --output ${OUTPUT_PATH}" > stderrAndstdoutFile 2>&1 -STD_ERR_AND_STD_OUT=$(cat stderrAndstdoutFile) -rm stderrAndstdoutFile -echo "==== STD_ERR_AND_STD_OUT_BEGIN ====" -echo "${STD_ERR_AND_STD_OUT}" -echo "==== STD_ERR_AND_STD_OUT_END ====" - -if [[ ! "$STD_ERR_AND_STD_OUT" =~ "Hadoop security with Kerberos is enabled but the login user does not have Kerberos credentials" ]]; then - echo "Output does not contain the Kerberos error message as required" - exit 1 -fi diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java index 25547a1193d21f..865dd7d219c047 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java @@ -52,7 +52,6 @@ import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneClientHAServices; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.rpc.AddressResolution; import org.apache.flink.util.FlinkException; @@ -245,14 +244,6 @@ public ClusterClientProvider deployApplicationCluster( return clusterClientProvider; } - @Override - public ClusterClientProvider deployJobCluster( - ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) - throws ClusterDeploymentException { - throw new ClusterDeploymentException( - "Per-Job Mode not supported by Active Kubernetes deployments."); - } - private ClusterClientProvider deployClusterInternal( String entryPoint, ClusterSpecification clusterSpecification, boolean detached) throws ClusterDeploymentException { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java deleted file mode 100644 index 5a3baa170b3496..00000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunnerFactory; -import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; -import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.streaming.api.graph.ExecutionPlan; -import org.apache.flink.util.Preconditions; - -import org.apache.flink.shaded.guava32.com.google.common.collect.Iterables; - -import java.util.Collection; - -import static org.apache.flink.runtime.entrypoint.ClusterEntrypoint.INTERNAL_CLUSTER_EXECUTION_MODE; - -/** {@link DispatcherFactory} which creates a {@link MiniDispatcher}. */ -public enum JobDispatcherFactory implements DispatcherFactory { - INSTANCE; - - @Override - public MiniDispatcher createDispatcher( - RpcService rpcService, - DispatcherId fencingToken, - Collection recoveredJobs, - Collection recoveredDirtyJobResults, - DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobPersistenceComponents - partialDispatcherServicesWithJobPersistenceComponents) - throws Exception { - final ExecutionPlan recoveredExecutionPlan = Iterables.getOnlyElement(recoveredJobs, null); - final JobResult recoveredDirtyJob = - Iterables.getOnlyElement(recoveredDirtyJobResults, null); - - Preconditions.checkArgument( - recoveredExecutionPlan == null ^ recoveredDirtyJob == null, - "Either the ExecutionPlan or the recovered JobResult needs to be specified."); - - final Configuration configuration = - partialDispatcherServicesWithJobPersistenceComponents.getConfiguration(); - final String executionModeValue = configuration.get(INTERNAL_CLUSTER_EXECUTION_MODE); - final ClusterEntrypoint.ExecutionMode executionMode = - ClusterEntrypoint.ExecutionMode.valueOf(executionModeValue); - - return new MiniDispatcher( - rpcService, - fencingToken, - DispatcherServices.from( - partialDispatcherServicesWithJobPersistenceComponents, - JobMasterServiceLeadershipRunnerFactory.INSTANCE, - CheckpointResourcesCleanupRunnerFactory.INSTANCE), - recoveredExecutionPlan, - recoveredDirtyJob, - dispatcherBootstrapFactory, - executionMode); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java deleted file mode 100644 index e74b12632eacd4..00000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java +++ /dev/null @@ -1,170 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher; - -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleanerFactory; -import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; -import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; -import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.streaming.api.graph.ExecutionPlan; -import org.apache.flink.util.CollectionUtil; -import org.apache.flink.util.FlinkException; - -import javax.annotation.Nullable; - -import java.time.Duration; -import java.util.concurrent.CompletableFuture; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Mini Dispatcher which is instantiated as the dispatcher component by the {@link - * JobClusterEntrypoint}. - * - *

The mini dispatcher is initialized with a single {@link ExecutionPlan} which it runs. - * - *

Depending on the {@link ClusterEntrypoint.ExecutionMode}, the mini dispatcher will directly - * terminate after job completion if its execution mode is {@link - * ClusterEntrypoint.ExecutionMode#DETACHED}. - */ -public class MiniDispatcher extends Dispatcher { - - private final JobClusterEntrypoint.ExecutionMode executionMode; - private boolean jobCancelled = false; - - public MiniDispatcher( - RpcService rpcService, - DispatcherId fencingToken, - DispatcherServices dispatcherServices, - @Nullable ExecutionPlan executionPlan, - @Nullable JobResult recoveredDirtyJob, - DispatcherBootstrapFactory dispatcherBootstrapFactory, - JobClusterEntrypoint.ExecutionMode executionMode) - throws Exception { - super( - rpcService, - fencingToken, - CollectionUtil.ofNullable(executionPlan), - CollectionUtil.ofNullable(recoveredDirtyJob), - dispatcherBootstrapFactory, - dispatcherServices); - - this.executionMode = checkNotNull(executionMode); - } - - @VisibleForTesting - public MiniDispatcher( - RpcService rpcService, - DispatcherId fencingToken, - DispatcherServices dispatcherServices, - @Nullable ExecutionPlan executionPlan, - @Nullable JobResult recoveredDirtyJob, - DispatcherBootstrapFactory dispatcherBootstrapFactory, - JobManagerRunnerRegistry jobManagerRunnerRegistry, - ResourceCleanerFactory resourceCleanerFactory, - JobClusterEntrypoint.ExecutionMode executionMode) - throws Exception { - super( - rpcService, - fencingToken, - CollectionUtil.ofNullable(executionPlan), - CollectionUtil.ofNullable(recoveredDirtyJob), - dispatcherBootstrapFactory, - dispatcherServices, - jobManagerRunnerRegistry, - resourceCleanerFactory); - - this.executionMode = checkNotNull(executionMode); - } - - @Override - public CompletableFuture submitJob(ExecutionPlan executionPlan, Duration timeout) { - final CompletableFuture acknowledgeCompletableFuture = - super.submitJob(executionPlan, timeout); - - acknowledgeCompletableFuture.whenComplete( - (Acknowledge ignored, Throwable throwable) -> { - if (throwable != null) { - onFatalError( - new FlinkException( - "Failed to submit job " - + executionPlan.getJobID() - + " in job mode.", - throwable)); - } - }); - - return acknowledgeCompletableFuture; - } - - @Override - public CompletableFuture requestJobResult(JobID jobId, Duration timeout) { - final CompletableFuture jobResultFuture = super.requestJobResult(jobId, timeout); - - if (executionMode == ClusterEntrypoint.ExecutionMode.NORMAL) { - // terminate the MiniDispatcher once we served the first JobResult successfully - jobResultFuture.thenAccept( - (JobResult result) -> { - ApplicationStatus status = - result.getSerializedThrowable().isPresent() - ? ApplicationStatus.FAILED - : ApplicationStatus.SUCCEEDED; - - if (!ApplicationStatus.UNKNOWN.equals(result.getApplicationStatus())) { - log.info( - "Shutting down cluster because someone retrieved the job result" - + " and the status is globally terminal."); - shutDownFuture.complete(status); - } - }); - } else { - log.info("Not shutting down cluster after someone retrieved the job result."); - } - - return jobResultFuture; - } - - @Override - public CompletableFuture cancelJob(JobID jobId, Duration timeout) { - jobCancelled = true; - return super.cancelJob(jobId, timeout); - } - - @Override - protected void runPostJobGloballyTerminated(JobID jobId, JobStatus jobStatus) { - super.runPostJobGloballyTerminated(jobId, jobStatus); - - if (jobCancelled || executionMode == ClusterEntrypoint.ExecutionMode.DETACHED) { - // shut down if job is cancelled or we don't have to wait for the execution - // result retrieval - log.info( - "Shutting down cluster after job with state {}, jobCancelled: {}, executionMode: {}", - jobStatus, - jobCancelled, - executionMode); - shutDownFuture.complete(ApplicationStatus.fromJobStatus(jobStatus)); - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java index a490e01f86b2a9..fdd9333df4740a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java @@ -20,7 +20,6 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.leaderelection.LeaderElection; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -67,10 +66,4 @@ public static DefaultDispatcherRunnerFactory createSessionRunner( return new DefaultDispatcherRunnerFactory( SessionDispatcherLeaderProcessFactoryFactory.create(dispatcherFactory)); } - - public static DefaultDispatcherRunnerFactory createJobRunner( - JobGraphRetriever jobGraphRetriever) { - return new DefaultDispatcherRunnerFactory( - JobDispatcherLeaderProcessFactoryFactory.create(jobGraphRetriever)); - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java deleted file mode 100644 index 4a48f821ec06a1..00000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher.runner; - -import org.apache.flink.runtime.dispatcher.DispatcherId; -import org.apache.flink.runtime.highavailability.JobResultStore; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.ThrowingExecutionPlanWriter; -import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.util.CollectionUtil; -import org.apache.flink.util.Preconditions; - -import javax.annotation.Nullable; - -import java.util.UUID; - -/** {@link DispatcherLeaderProcess} implementation for the per-job mode. */ -public class JobDispatcherLeaderProcess extends AbstractDispatcherLeaderProcess { - - private final DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; - - @Nullable private final JobGraph jobGraph; - @Nullable private final JobResult recoveredDirtyJobResult; - - private final JobResultStore jobResultStore; - - JobDispatcherLeaderProcess( - UUID leaderSessionId, - DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, - @Nullable JobGraph jobGraph, - @Nullable JobResult recoveredDirtyJobResult, - JobResultStore jobResultStore, - FatalErrorHandler fatalErrorHandler) { - super(leaderSessionId, fatalErrorHandler); - this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; - this.jobGraph = jobGraph; - this.recoveredDirtyJobResult = recoveredDirtyJobResult; - this.jobResultStore = Preconditions.checkNotNull(jobResultStore); - } - - @Override - protected void onStart() { - final DispatcherGatewayService dispatcherService = - dispatcherGatewayServiceFactory.create( - DispatcherId.fromUuid(getLeaderSessionId()), - CollectionUtil.ofNullable(jobGraph), - CollectionUtil.ofNullable(recoveredDirtyJobResult), - ThrowingExecutionPlanWriter.INSTANCE, - jobResultStore); - - completeDispatcherSetup(dispatcherService); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java deleted file mode 100644 index 1716a1e961a191..00000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher.runner; - -import org.apache.flink.runtime.highavailability.JobResultStore; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.util.Preconditions; - -import javax.annotation.Nullable; - -import java.util.UUID; - -/** Factory for the {@link JobDispatcherLeaderProcess}. */ -public class JobDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFactory { - - private final AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory - dispatcherGatewayServiceFactory; - - @Nullable private final JobGraph jobGraph; - @Nullable private final JobResult recoveredDirtyJobResult; - - private final JobResultStore jobResultStore; - - private final FatalErrorHandler fatalErrorHandler; - - JobDispatcherLeaderProcessFactory( - AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory - dispatcherGatewayServiceFactory, - @Nullable JobGraph jobGraph, - @Nullable JobResult recoveredDirtyJobResult, - JobResultStore jobResultStore, - FatalErrorHandler fatalErrorHandler) { - this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; - this.jobGraph = jobGraph; - this.recoveredDirtyJobResult = recoveredDirtyJobResult; - this.jobResultStore = Preconditions.checkNotNull(jobResultStore); - this.fatalErrorHandler = fatalErrorHandler; - } - - @Override - public DispatcherLeaderProcess create(UUID leaderSessionID) { - return new JobDispatcherLeaderProcess( - leaderSessionID, - dispatcherGatewayServiceFactory, - jobGraph, - recoveredDirtyJobResult, - jobResultStore, - fatalErrorHandler); - } - - @Nullable - JobGraph getJobGraph() { - return this.jobGraph; - } - - @Nullable - JobResult getRecoveredDirtyJobResult() { - return this.recoveredDirtyJobResult; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java deleted file mode 100644 index 8d38825bc58d51..00000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher.runner; - -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.dispatcher.JobDispatcherFactory; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; -import org.apache.flink.runtime.highavailability.JobResultStore; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; -import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.FlinkRuntimeException; -import org.apache.flink.util.Preconditions; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Collection; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.Executor; -import java.util.stream.Collectors; - -/** Factory for the {@link JobDispatcherLeaderProcessFactory}. */ -public class JobDispatcherLeaderProcessFactoryFactory - implements DispatcherLeaderProcessFactoryFactory { - - private static final Logger LOG = - LoggerFactory.getLogger(JobDispatcherLeaderProcessFactoryFactory.class); - - private final JobGraphRetriever jobGraphRetriever; - - @VisibleForTesting - JobDispatcherLeaderProcessFactoryFactory(JobGraphRetriever jobGraphRetriever) { - this.jobGraphRetriever = jobGraphRetriever; - } - - @Override - public JobDispatcherLeaderProcessFactory createFactory( - JobPersistenceComponentFactory jobPersistenceComponentFactory, - Executor ioExecutor, - RpcService rpcService, - PartialDispatcherServices partialDispatcherServices, - FatalErrorHandler fatalErrorHandler) { - - final JobGraph jobGraph; - - try { - jobGraph = - Preconditions.checkNotNull( - jobGraphRetriever.retrieveJobGraph( - partialDispatcherServices.getConfiguration())); - } catch (FlinkException e) { - throw new FlinkRuntimeException("Could not retrieve the JobGraph.", e); - } - - final JobResultStore jobResultStore = jobPersistenceComponentFactory.createJobResultStore(); - final Collection recoveredDirtyJobResults = getDirtyJobResults(jobResultStore); - - final Optional maybeRecoveredDirtyJobResult = - extractDirtyJobResult(recoveredDirtyJobResults, jobGraph); - final Optional maybeJobGraph = - getJobGraphBasedOnDirtyJobResults(jobGraph, recoveredDirtyJobResults); - - final DefaultDispatcherGatewayServiceFactory defaultDispatcherServiceFactory = - new DefaultDispatcherGatewayServiceFactory( - JobDispatcherFactory.INSTANCE, rpcService, partialDispatcherServices); - - return new JobDispatcherLeaderProcessFactory( - defaultDispatcherServiceFactory, - maybeJobGraph.orElse(null), - maybeRecoveredDirtyJobResult.orElse(null), - jobResultStore, - fatalErrorHandler); - } - - public static JobDispatcherLeaderProcessFactoryFactory create( - JobGraphRetriever jobGraphRetriever) { - return new JobDispatcherLeaderProcessFactoryFactory(jobGraphRetriever); - } - - private static Collection getDirtyJobResults(JobResultStore jobResultStore) { - try { - return jobResultStore.getDirtyResults(); - } catch (IOException e) { - throw new FlinkRuntimeException( - "Could not retrieve the JobResults of dirty jobs from the underlying JobResultStore.", - e); - } - } - - private static Optional extractDirtyJobResult( - Collection dirtyJobResults, JobGraph jobGraph) { - Optional actualDirtyJobResult = Optional.empty(); - for (JobResult dirtyJobResult : dirtyJobResults) { - if (dirtyJobResult.getJobId().equals(jobGraph.getJobID())) { - actualDirtyJobResult = Optional.of(dirtyJobResult); - } else { - LOG.warn( - "Unexpected dirty JobResultStore entry: Job '{}' is listed as dirty, isn't part of this single-job cluster, though.", - dirtyJobResult.getJobId()); - } - } - - return actualDirtyJobResult; - } - - private static Optional getJobGraphBasedOnDirtyJobResults( - JobGraph jobGraph, Collection dirtyJobResults) { - final Set jobIdsOfFinishedJobs = - dirtyJobResults.stream().map(JobResult::getJobId).collect(Collectors.toSet()); - if (jobIdsOfFinishedJobs.contains(jobGraph.getJobID())) { - LOG.info( - "Skipping recovery of a job with job id {}, because it already reached a globally terminal state", - jobGraph.getJobID()); - return Optional.empty(); - } - return Optional.of(jobGraph); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index 122db8d639fc34..c311d76da73351 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -43,7 +43,6 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.dispatcher.ExecutionGraphInfoStore; -import org.apache.flink.runtime.dispatcher.MiniDispatcher; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.parser.CommandLineParser; @@ -743,7 +742,7 @@ public static void runClusterEntrypoint(ClusterEntrypoint clusterEntrypoint) { System.exit(returnCode); } - /** Execution mode of the {@link MiniDispatcher}. */ + /** Execution mode of the dispatcher. */ public enum ExecutionMode { /** Waits until the job result has been served. */ NORMAL, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java deleted file mode 100644 index 629fcc58896c54..00000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.entrypoint; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.dispatcher.ExecutionGraphInfoStore; -import org.apache.flink.runtime.dispatcher.MemoryExecutionGraphInfoStore; -import org.apache.flink.util.concurrent.ScheduledExecutor; - -/** - * Base class for per-job cluster entry points. - * - * @deprecated Per-job mode has been deprecated in Flink 1.15 and will be removed in the future. - * Please use application mode instead. - */ -@Deprecated -public abstract class JobClusterEntrypoint extends ClusterEntrypoint { - - public JobClusterEntrypoint(Configuration configuration) { - super(configuration); - } - - @Override - protected ExecutionGraphInfoStore createSerializableExecutionGraphStore( - Configuration configuration, ScheduledExecutor scheduledExecutor) { - return new MemoryExecutionGraphInfoStore(); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java index 094d37af3c2d51..c1178130c9f87f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java @@ -46,7 +46,6 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.ResourceManagerService; import org.apache.flink.runtime.resourcemanager.ResourceManagerServiceImpl; -import org.apache.flink.runtime.rest.JobRestEndpointFactory; import org.apache.flink.runtime.rest.RestEndpointFactory; import org.apache.flink.runtime.rest.SessionRestEndpointFactory; import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher; @@ -301,12 +300,4 @@ public static DefaultDispatcherResourceManagerComponentFactory createSessionComp resourceManagerFactory, SessionRestEndpointFactory.INSTANCE); } - - public static DefaultDispatcherResourceManagerComponentFactory createJobComponentFactory( - ResourceManagerFactory resourceManagerFactory, JobGraphRetriever jobGraphRetriever) { - return new DefaultDispatcherResourceManagerComponentFactory( - DefaultDispatcherRunnerFactory.createJobRunner(jobGraphRetriever), - resourceManagerFactory, - JobRestEndpointFactory.INSTANCE); - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java index ad258ab0d13474..30fe4ef2d12cc4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java @@ -20,7 +20,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.TransientBlobService; -import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; import org.apache.flink.runtime.leaderelection.LeaderElection; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; @@ -35,7 +34,7 @@ import java.io.IOException; import java.util.concurrent.ScheduledExecutorService; -/** REST endpoint for the {@link JobClusterEntrypoint}. */ +/** REST endpoint for the ApplicationClusterEntryPoint. */ public class MiniDispatcherRestEndpoint extends WebMonitorEndpoint { public MiniDispatcherRestEndpoint( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java deleted file mode 100644 index 05bd86a53df99f..00000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java +++ /dev/null @@ -1,226 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunnerFactory; -import org.apache.flink.runtime.dispatcher.runner.JobDispatcherLeaderProcessFactoryFactory; -import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; -import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; -import org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever; -import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobGraphBuilder; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; -import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; -import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; -import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.runtime.messages.FlinkJobNotFoundException; -import org.apache.flink.runtime.minicluster.MiniCluster; -import org.apache.flink.runtime.minicluster.TestingMiniCluster; -import org.apache.flink.runtime.minicluster.TestingMiniClusterConfiguration; -import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; -import org.apache.flink.runtime.rest.JobRestEndpointFactory; -import org.apache.flink.runtime.scheduler.adaptive.AdaptiveSchedulerClusterITCase; -import org.apache.flink.runtime.testutils.CommonTestUtils; -import org.apache.flink.testutils.TestingUtils; -import org.apache.flink.testutils.executor.TestExecutorExtension; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.TestLoggerExtension; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; - -import java.io.IOException; -import java.io.ObjectOutputStream; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.ScheduledExecutorService; -import java.util.function.Supplier; - -import static java.nio.file.StandardOpenOption.CREATE; -import static org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever.JOB_GRAPH_FILE_PATH; - -/** An integration test which recovers from checkpoint after regaining the leadership. */ -@ExtendWith(TestLoggerExtension.class) -public class JobDispatcherITCase { - @RegisterExtension - static final TestExecutorExtension EXECUTOR_RESOURCE = - TestingUtils.defaultExecutorExtension(); - - private Supplier - createJobModeDispatcherResourceManagerComponentFactorySupplier( - Configuration configuration) { - return () -> { - try { - return new DefaultDispatcherResourceManagerComponentFactory( - new DefaultDispatcherRunnerFactory( - JobDispatcherLeaderProcessFactoryFactory.create( - FileJobGraphRetriever.createFrom(configuration, null))), - StandaloneResourceManagerFactory.getInstance(), - JobRestEndpointFactory.INSTANCE); - } catch (IOException e) { - throw new RuntimeException(e); - } - }; - } - - @Test - public void testRecoverFromCheckpointAfterLosingAndRegainingLeadership(@TempDir Path tmpPath) - throws Exception { - final Configuration configuration = new Configuration(); - configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); - final TestingMiniClusterConfiguration clusterConfiguration = - TestingMiniClusterConfiguration.newBuilder() - .setConfiguration(configuration) - .build(); - final EmbeddedHaServicesWithLeadershipControl haServices = - new EmbeddedHaServicesWithLeadershipControl(EXECUTOR_RESOURCE.getExecutor()); - - final Configuration newConfiguration = - new Configuration(clusterConfiguration.getConfiguration()); - final long checkpointInterval = 100; - final JobID jobID = - generateAndPersistJobGraph(newConfiguration, checkpointInterval, tmpPath); - - final TestingMiniCluster.Builder clusterBuilder = - TestingMiniCluster.newBuilder(clusterConfiguration) - .setHighAvailabilityServicesSupplier(() -> haServices) - .setDispatcherResourceManagerComponentFactorySupplier( - createJobModeDispatcherResourceManagerComponentFactorySupplier( - newConfiguration)); - AtLeastOneCheckpointInvokable.reset(); - - try (final MiniCluster cluster = clusterBuilder.build()) { - // start mini cluster and submit the job - cluster.start(); - - AtLeastOneCheckpointInvokable.atLeastOneCheckpointCompleted.await(); - - final CompletableFuture firstJobResult = cluster.requestJobResult(jobID); - // make sure requestJobResult was processed by job master - cluster.getJobStatus(jobID).get(); - - haServices.revokeDispatcherLeadership(); - // make sure the leadership is revoked to avoid race conditions - Assertions.assertEquals( - ApplicationStatus.UNKNOWN, firstJobResult.get().getApplicationStatus()); - - haServices.grantDispatcherLeadership(); - - // job is suspended, wait until it's running - awaitJobStatus(cluster, jobID, JobStatus.RUNNING); - - CommonTestUtils.waitUntilCondition( - () -> - cluster.getArchivedExecutionGraph(jobID) - .get() - .getCheckpointStatsSnapshot() - .getLatestRestoredCheckpoint() - != null); - } - } - - private JobID generateAndPersistJobGraph( - Configuration configuration, long checkpointInterval, Path tmpPath) throws Exception { - final JobVertex jobVertex = new JobVertex("jobVertex"); - jobVertex.setInvokableClass(AtLeastOneCheckpointInvokable.class); - jobVertex.setParallelism(1); - - final CheckpointCoordinatorConfiguration checkpointCoordinatorConfiguration = - CheckpointCoordinatorConfiguration.builder() - .setCheckpointInterval(checkpointInterval) - .build(); - final JobCheckpointingSettings checkpointingSettings = - new JobCheckpointingSettings(checkpointCoordinatorConfiguration, null); - final JobGraph jobGraph = - JobGraphBuilder.newStreamingJobGraphBuilder() - .addJobVertex(jobVertex) - .setJobCheckpointingSettings(checkpointingSettings) - .build(); - - final Path jobGraphPath = tmpPath.resolve(JOB_GRAPH_FILE_PATH.defaultValue()); - try (ObjectOutputStream objectOutputStream = - new ObjectOutputStream(Files.newOutputStream(jobGraphPath, CREATE))) { - objectOutputStream.writeObject(jobGraph); - } - configuration.setString(JOB_GRAPH_FILE_PATH.key(), jobGraphPath.toString()); - return jobGraph.getJobID(); - } - - private static void awaitJobStatus(MiniCluster cluster, JobID jobId, JobStatus status) - throws Exception { - CommonTestUtils.waitUntilCondition( - () -> { - try { - return cluster.getJobStatus(jobId).get() == status; - } catch (ExecutionException e) { - if (ExceptionUtils.findThrowable(e, FlinkJobNotFoundException.class) - .isPresent()) { - // job may not be yet submitted - return false; - } - throw e; - } - }); - } - - /** - * An invokable that supports checkpointing and counts down when there is at least one - * checkpoint. - */ - public static class AtLeastOneCheckpointInvokable - extends AdaptiveSchedulerClusterITCase.CheckpointingNoOpInvokable { - - private static volatile CountDownLatch atLeastOneCheckpointCompleted; - - private static void reset() { - atLeastOneCheckpointCompleted = new CountDownLatch(1); - } - - public AtLeastOneCheckpointInvokable(Environment environment) { - super(environment); - } - - @Override - public Future notifyCheckpointCompleteAsync(long checkpointId) { - atLeastOneCheckpointCompleted.countDown(); - return CompletableFuture.completedFuture(null); - } - - @Override - public Future notifyCheckpointAbortAsync( - long checkpointId, long latestCompletedCheckpointId) { - return CompletableFuture.completedFuture(null); - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java deleted file mode 100644 index f0d1b86ffc8544..00000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java +++ /dev/null @@ -1,380 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.VoidBlobStore; -import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.dispatcher.cleanup.TestingCleanupRunnerFactory; -import org.apache.flink.runtime.dispatcher.cleanup.TestingResourceCleanerFactory; -import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; -import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.heartbeat.HeartbeatServicesImpl; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; -import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; -import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; -import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; -import org.apache.flink.runtime.rpc.RpcUtils; -import org.apache.flink.runtime.rpc.TestingRpcService; -import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; -import org.apache.flink.runtime.testutils.CommonTestUtils; -import org.apache.flink.runtime.testutils.TestingJobResultStore; -import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; -import org.apache.flink.util.TestLogger; -import org.apache.flink.util.concurrent.FutureUtils; - -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import javax.annotation.Nullable; - -import java.io.IOException; -import java.time.Duration; -import java.util.Collections; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ForkJoinPool; -import java.util.concurrent.TimeoutException; - -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; - -/** Tests for the {@link MiniDispatcher}. */ -public class MiniDispatcherTest extends TestLogger { - - private static final Duration timeout = Duration.ofSeconds(10L); - - @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); - - @Rule - public final TestingFatalErrorHandlerResource testingFatalErrorHandlerResource = - new TestingFatalErrorHandlerResource(); - - private static JobGraph jobGraph; - - private static ExecutionGraphInfo executionGraphInfo; - - private static TestingRpcService rpcService; - - private static Configuration configuration; - - private static BlobServer blobServer; - - private final TestingResourceManagerGateway resourceManagerGateway = - new TestingResourceManagerGateway(); - - private final HeartbeatServices heartbeatServices = new HeartbeatServicesImpl(1000L, 1000L); - - private final ExecutionGraphInfoStore executionGraphInfoStore = - new MemoryExecutionGraphInfoStore(); - - private TestingHighAvailabilityServices highAvailabilityServices; - - private TestingJobMasterServiceLeadershipRunnerFactory testingJobManagerRunnerFactory; - private TestingCleanupRunnerFactory testingCleanupRunnerFactory; - - private CompletableFuture localCleanupResultFuture; - private CompletableFuture globalCleanupResultFuture; - - @BeforeClass - public static void setupClass() throws IOException { - jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); - - executionGraphInfo = - new ExecutionGraphInfo( - new ArchivedExecutionGraphBuilder() - .setJobID(jobGraph.getJobID()) - .setState(JobStatus.FINISHED) - .build()); - - rpcService = new TestingRpcService(); - configuration = new Configuration(); - - blobServer = - new BlobServer(configuration, temporaryFolder.newFolder(), new VoidBlobStore()); - } - - @Before - public void setup() throws Exception { - highAvailabilityServices = new TestingHighAvailabilityServicesBuilder().build(); - - testingJobManagerRunnerFactory = new TestingJobMasterServiceLeadershipRunnerFactory(); - testingCleanupRunnerFactory = new TestingCleanupRunnerFactory(); - - // the default setting shouldn't block the cleanup - localCleanupResultFuture = FutureUtils.completedVoidFuture(); - globalCleanupResultFuture = FutureUtils.completedVoidFuture(); - } - - @AfterClass - public static void teardownClass() - throws IOException, InterruptedException, ExecutionException, TimeoutException { - if (blobServer != null) { - blobServer.close(); - } - - if (rpcService != null) { - RpcUtils.terminateRpcService(rpcService); - } - } - - /** Tests that the {@link MiniDispatcher} recovers the single job with which it was started. */ - @Test - public void testSingleJobRecovery() throws Exception { - final MiniDispatcher miniDispatcher = - createMiniDispatcher(ClusterEntrypoint.ExecutionMode.DETACHED); - - miniDispatcher.start(); - - try { - final TestingJobManagerRunner testingJobManagerRunner = - testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); - - assertThat(testingJobManagerRunner.getJobID(), is(jobGraph.getJobID())); - } finally { - RpcUtils.terminateRpcEndpoint(miniDispatcher); - } - } - - /** Tests that the {@link MiniDispatcher} recovers the single job with which it was started. */ - @Test - public void testDirtyJobResultCleanup() throws Exception { - final JobID jobId = new JobID(); - final MiniDispatcher miniDispatcher = - createMiniDispatcher( - ClusterEntrypoint.ExecutionMode.DETACHED, - null, - TestingJobResultStore.createSuccessfulJobResult(jobId)); - - miniDispatcher.start(); - - try { - final TestingJobManagerRunner testingCleanupRunner = - testingCleanupRunnerFactory.takeCreatedJobManagerRunner(); - assertThat(testingCleanupRunner.getJobID(), is(jobId)); - } finally { - RpcUtils.terminateRpcEndpoint(miniDispatcher); - } - } - - /** - * Tests that in detached mode, the {@link MiniDispatcher} will complete the future that signals - * job termination. - */ - @Test - public void testTerminationAfterJobCompletion() throws Exception { - globalCleanupResultFuture = new CompletableFuture<>(); - final MiniDispatcher miniDispatcher = - createMiniDispatcher(ClusterEntrypoint.ExecutionMode.DETACHED); - - miniDispatcher.start(); - - try { - // wait until we have submitted the job - final TestingJobManagerRunner testingJobManagerRunner = - testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); - - testingJobManagerRunner.completeResultFuture(executionGraphInfo); - - CommonTestUtils.waitUntilCondition( - () -> - !highAvailabilityServices - .getJobResultStore() - .getDirtyResults() - .isEmpty()); - - assertFalse( - "The shutdownFuture should not be completed before the cleanup is triggered.", - miniDispatcher.getShutDownFuture().isDone()); - - globalCleanupResultFuture.complete(null); - - miniDispatcher.getShutDownFuture().get(); - } finally { - // we have to complete the future to make the job and, as a consequence, the - // MiniDispatcher terminate - globalCleanupResultFuture.complete(null); - RpcUtils.terminateRpcEndpoint(miniDispatcher); - } - } - - /** - * Tests that in detached mode, the {@link MiniDispatcher} will not complete the future that - * signals job termination if the JobStatus is not globally terminal state. - */ - @Test - public void testNotTerminationWithoutGloballyTerminalState() throws Exception { - final MiniDispatcher miniDispatcher = - createMiniDispatcher(ClusterEntrypoint.ExecutionMode.DETACHED); - miniDispatcher.start(); - - try { - // wait until we have submitted the job - final TestingJobManagerRunner testingJobManagerRunner = - testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); - - testingJobManagerRunner.completeResultFuture( - new ExecutionGraphInfo( - new ArchivedExecutionGraphBuilder() - .setJobID(jobGraph.getJobID()) - .setState(JobStatus.SUSPENDED) - .build())); - - testingJobManagerRunner.getTerminationFuture().get(); - Assertions.assertThat(miniDispatcher.getShutDownFuture()).isNotDone(); - } finally { - RpcUtils.terminateRpcEndpoint(miniDispatcher); - } - } - - /** - * Tests that the {@link MiniDispatcher} only terminates in {@link - * ClusterEntrypoint.ExecutionMode#NORMAL} after it has served the {@link - * org.apache.flink.runtime.jobmaster.JobResult} once. - */ - @Test - public void testJobResultRetrieval() throws Exception { - final MiniDispatcher miniDispatcher = - createMiniDispatcher(ClusterEntrypoint.ExecutionMode.NORMAL); - - miniDispatcher.start(); - - try { - // wait until we have submitted the job - final TestingJobManagerRunner testingJobManagerRunner = - testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); - - testingJobManagerRunner.completeResultFuture(executionGraphInfo); - - assertFalse(miniDispatcher.getTerminationFuture().isDone()); - - final DispatcherGateway dispatcherGateway = - miniDispatcher.getSelfGateway(DispatcherGateway.class); - - final CompletableFuture jobResultFuture = - dispatcherGateway.requestJobResult(jobGraph.getJobID(), timeout); - - final JobResult jobResult = jobResultFuture.get(); - - assertThat(jobResult.getJobId(), is(jobGraph.getJobID())); - } finally { - RpcUtils.terminateRpcEndpoint(miniDispatcher); - } - } - - @Test - public void testShutdownIfJobCancelledInNormalMode() throws Exception { - final MiniDispatcher miniDispatcher = - createMiniDispatcher(ClusterEntrypoint.ExecutionMode.NORMAL); - miniDispatcher.start(); - - try { - // wait until we have submitted the job - final TestingJobManagerRunner testingJobManagerRunner = - testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); - - assertFalse(miniDispatcher.getTerminationFuture().isDone()); - - final DispatcherGateway dispatcherGateway = - miniDispatcher.getSelfGateway(DispatcherGateway.class); - - dispatcherGateway.cancelJob(jobGraph.getJobID(), Duration.ofSeconds(10L)); - testingJobManagerRunner.completeResultFuture( - new ExecutionGraphInfo( - new ArchivedExecutionGraphBuilder() - .setJobID(jobGraph.getJobID()) - .setState(JobStatus.CANCELED) - .build())); - - ApplicationStatus applicationStatus = miniDispatcher.getShutDownFuture().get(); - assertThat(applicationStatus, is(ApplicationStatus.CANCELED)); - } finally { - RpcUtils.terminateRpcEndpoint(miniDispatcher); - } - } - - // -------------------------------------------------------- - // Utilities - // -------------------------------------------------------- - - private MiniDispatcher createMiniDispatcher(ClusterEntrypoint.ExecutionMode executionMode) - throws Exception { - return createMiniDispatcher(executionMode, jobGraph, null); - } - - private MiniDispatcher createMiniDispatcher( - ClusterEntrypoint.ExecutionMode executionMode, - @Nullable JobGraph recoveredJobGraph, - @Nullable JobResult recoveredDirtyJob) - throws Exception { - final JobManagerRunnerRegistry jobManagerRunnerRegistry = - new DefaultJobManagerRunnerRegistry(2); - return new MiniDispatcher( - rpcService, - DispatcherId.generate(), - new DispatcherServices( - configuration, - highAvailabilityServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - heartbeatServices, - executionGraphInfoStore, - testingFatalErrorHandlerResource.getFatalErrorHandler(), - VoidHistoryServerArchivist.INSTANCE, - null, - new DispatcherOperationCaches(), - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - highAvailabilityServices.getExecutionPlanStore(), - highAvailabilityServices.getJobResultStore(), - testingJobManagerRunnerFactory, - testingCleanupRunnerFactory, - ForkJoinPool.commonPool(), - Collections.emptySet()), - recoveredJobGraph, - recoveredDirtyJob, - (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(), - jobManagerRunnerRegistry, - TestingResourceCleanerFactory.builder() - // JobManagerRunnerRegistry needs to be added explicitly - // because cleaning it will trigger the closeAsync latch - // provided by TestingJobManagerRunner - .withLocallyCleanableResource(jobManagerRunnerRegistry) - .withGloballyCleanableResource( - (jobId, ignoredExecutor) -> globalCleanupResultFuture) - .withLocallyCleanableResource( - (jobId, ignoredExecutor) -> localCleanupResultFuture) - .build(), - executionMode); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactoryTest.java deleted file mode 100644 index b09d5da3cbd29c..00000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactoryTest.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher.runner; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.dispatcher.TestingPartialDispatcherServices; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; -import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; -import org.apache.flink.runtime.jobmanager.StandaloneExecutionPlanStore; -import org.apache.flink.runtime.jobmanager.TestingJobPersistenceComponentFactory; -import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler; -import org.apache.flink.runtime.rpc.TestingRpcService; -import org.apache.flink.runtime.testutils.TestingJobResultStore; -import org.apache.flink.util.CollectionUtil; -import org.apache.flink.util.TestLoggerExtension; -import org.apache.flink.util.concurrent.Executors; - -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.io.TempDir; - -import javax.annotation.Nullable; - -import java.io.IOException; -import java.nio.file.Path; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -@ExtendWith(TestLoggerExtension.class) -class JobDispatcherLeaderProcessFactoryFactoryTest { - - @TempDir private Path temporaryFolder; - - @Test - public void testJobGraphWithoutDirtyJobResult() throws IOException { - final JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); - - final JobDispatcherLeaderProcessFactory factory = - createDispatcherLeaderProcessFactoryFromTestInstance( - jobGraph, null, temporaryFolder); - - assertThat(factory.getJobGraph()).isEqualTo(jobGraph); - assertThat(factory.getRecoveredDirtyJobResult()).isNull(); - } - - @Test - public void testJobGraphWithMatchingDirtyJobResult() throws IOException { - final JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); - final JobResult jobResult = - TestingJobResultStore.createSuccessfulJobResult(jobGraph.getJobID()); - - final JobDispatcherLeaderProcessFactory factory = - createDispatcherLeaderProcessFactoryFromTestInstance( - jobGraph, jobResult, temporaryFolder); - - assertThat(factory.getJobGraph()).isNull(); - assertThat(factory.getRecoveredDirtyJobResult()).isEqualTo(jobResult); - } - - @Test - public void testJobGraphWithNotMatchingDirtyJobResult() throws IOException { - final JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); - final JobResult jobResult = TestingJobResultStore.createSuccessfulJobResult(new JobID()); - - final JobDispatcherLeaderProcessFactory factory = - createDispatcherLeaderProcessFactoryFromTestInstance( - jobGraph, jobResult, temporaryFolder); - - assertThat(factory.getJobGraph()).isEqualTo(jobGraph); - assertThat(factory.getRecoveredDirtyJobResult()).isNull(); - } - - @Test - public void testMissingJobGraph() throws IOException { - assertThatThrownBy( - () -> - createDispatcherLeaderProcessFactoryFromTestInstance( - null, - TestingJobResultStore.createSuccessfulJobResult( - new JobID()), - temporaryFolder)) - .isInstanceOf(NullPointerException.class); - } - - private static JobDispatcherLeaderProcessFactory - createDispatcherLeaderProcessFactoryFromTestInstance( - @Nullable JobGraph jobGraph, - @Nullable JobResult dirtyJobResult, - Path storageDir) - throws IOException { - final JobDispatcherLeaderProcessFactoryFactory testInstance = - new JobDispatcherLeaderProcessFactoryFactory(ignoredConfig -> jobGraph); - - final TestingJobResultStore jobResultStore = - TestingJobResultStore.builder() - .withGetDirtyResultsSupplier( - () -> CollectionUtil.ofNullable(dirtyJobResult)) - .build(); - final ExecutionPlanStore executionPlanStore = new StandaloneExecutionPlanStore(); - return testInstance.createFactory( - new TestingJobPersistenceComponentFactory(executionPlanStore, jobResultStore), - Executors.directExecutor(), - new TestingRpcService(), - TestingPartialDispatcherServices.builder() - .withHighAvailabilityServices( - new TestingHighAvailabilityServicesBuilder() - .setExecutionPlanStore(executionPlanStore) - .setJobResultStore(jobResultStore) - .build()) - .build(storageDir.toFile(), new Configuration()), - NoOpFatalErrorHandler.INSTANCE); - } -} diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNFileReplicationITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNFileReplicationITCase.java deleted file mode 100644 index 387343ffc7cb1b..00000000000000 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNFileReplicationITCase.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.yarn; - -import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.configuration.RpcOptions; -import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; -import org.apache.flink.yarn.configuration.YarnConfigOptions; -import org.apache.flink.yarn.util.TestUtils; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DFSConfigKeys; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.time.Duration; -import java.util.Arrays; -import java.util.Objects; -import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; - -import static org.apache.flink.yarn.configuration.YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertTrue; - -/** - * Test cases for the deployment of Yarn Flink clusters with customized file replication numbers. - */ -class YARNFileReplicationITCase extends YarnTestBase { - - private static final Duration yarnAppTerminateTimeout = Duration.ofSeconds(10); - private static final int sleepIntervalInMS = 100; - - @BeforeAll - static void setup() { - YARN_CONFIGURATION.set(YarnTestBase.TEST_CLUSTER_NAME_KEY, "flink-yarn-tests-per-job"); - startYARNWithConfig(YARN_CONFIGURATION, true); - } - - @Test - void testPerJobModeWithCustomizedFileReplication() throws Exception { - final Configuration configuration = getDefaultConfiguration(); - configuration.set(YarnConfigOptions.FILE_REPLICATION, 4); - - runTest(() -> deployPerJob(configuration, getTestingJobGraph())); - } - - @Test - void testPerJobModeWithDefaultFileReplication() throws Exception { - runTest(() -> deployPerJob(getDefaultConfiguration(), getTestingJobGraph())); - } - - private void deployPerJob(Configuration configuration, JobGraph jobGraph) throws Exception { - try (final YarnClusterDescriptor yarnClusterDescriptor = - createYarnClusterDescriptor(configuration)) { - - yarnClusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); - yarnClusterDescriptor.addShipFiles( - Arrays.stream(Objects.requireNonNull(flinkLibFolder.listFiles())) - .map(file -> new Path(file.toURI())) - .collect(Collectors.toList())); - - final int masterMemory = - yarnClusterDescriptor - .getFlinkConfiguration() - .get(JobManagerOptions.TOTAL_PROCESS_MEMORY) - .getMebiBytes(); - final ClusterSpecification clusterSpecification = - new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(masterMemory) - .setTaskManagerMemoryMB(1024) - .setSlotsPerTaskManager(1) - .createClusterSpecification(); - - File testingJar = - TestUtils.findFile("..", new TestUtils.TestJarFinder("flink-yarn-tests")); - - jobGraph.addJar(new org.apache.flink.core.fs.Path(testingJar.toURI())); - try (ClusterClient clusterClient = - yarnClusterDescriptor - .deployJobCluster(clusterSpecification, jobGraph, false) - .getClusterClient()) { - - ApplicationId applicationId = clusterClient.getClusterId(); - - extraVerification(configuration, applicationId); - - final CompletableFuture jobResultCompletableFuture = - clusterClient.requestJobResult(jobGraph.getJobID()); - - final JobResult jobResult = jobResultCompletableFuture.get(); - - assertThat(jobResult).isNotNull(); - jobResult - .getSerializedThrowable() - .ifPresent( - serializedThrowable -> { - throw new AssertionError( - "Job failed", - serializedThrowable.deserializeError( - YARNFileReplicationITCase.class - .getClassLoader())); - }); - - waitApplicationFinishedElseKillIt( - applicationId, - yarnAppTerminateTimeout, - yarnClusterDescriptor, - sleepIntervalInMS); - } - } - } - - private JobGraph getTestingJobGraph() { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(2); - - env.addSource(new NoDataSource()).shuffle().sinkTo(new DiscardingSink<>()); - - return env.getStreamGraph().getJobGraph(); - } - - private Configuration getDefaultConfiguration() { - final Configuration configuration = new Configuration(); - configuration.set(JobManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(768)); - configuration.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse("1g")); - configuration.set(RpcOptions.ASK_TIMEOUT_DURATION, Duration.ofSeconds(30)); - configuration.set(CLASSPATH_INCLUDE_USER_JAR, YarnConfigOptions.UserJarInclusion.DISABLED); - - return configuration; - } - - private void extraVerification(Configuration configuration, ApplicationId applicationId) - throws Exception { - final FileSystem fs = FileSystem.get(getYarnConfiguration()); - - String suffix = ".flink/" + applicationId.toString() + "/" + flinkUberjar.getName(); - - Path uberJarHDFSPath = new Path(fs.getHomeDirectory(), suffix); - - assertTrue( - fs.exists(uberJarHDFSPath), - "The Flink uber jar needs to exist. If it does not exist, then this " - + "indicates that the Flink cluster has already terminated and Yarn has " - + "already deleted the working directory."); - - FileStatus fsStatus = fs.getFileStatus(uberJarHDFSPath); - - final int flinkFileReplication = configuration.get(YarnConfigOptions.FILE_REPLICATION); - final int replication = - YARN_CONFIGURATION.getInt( - DFSConfigKeys.DFS_REPLICATION_KEY, DFSConfigKeys.DFS_REPLICATION_DEFAULT); - - // If YarnConfigOptions.FILE_REPLICATION is not set. The replication number should equals to - // yarn configuration value. - int expectedReplication = flinkFileReplication > 0 ? flinkFileReplication : replication; - assertThat((int) fsStatus.getReplication()).isEqualTo(expectedReplication); - } -} diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java deleted file mode 100644 index d975b1fd27f86c..00000000000000 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java +++ /dev/null @@ -1,222 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.yarn; - -import org.apache.flink.api.common.cache.DistributedCache; -import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.configuration.RpcOptions; -import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobType; -import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; -import org.apache.flink.yarn.configuration.YarnConfigOptions; -import org.apache.flink.yarn.testjob.YarnTestArchiveJob; -import org.apache.flink.yarn.testjob.YarnTestCacheJob; -import org.apache.flink.yarn.util.TestUtils; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -import java.io.File; -import java.io.IOException; -import java.time.Duration; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.CompletableFuture; - -import static org.apache.flink.yarn.configuration.YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR; -import static org.assertj.core.api.Assertions.assertThat; - -/** Test cases for the deployment of Yarn Flink clusters. */ -class YARNITCase extends YarnTestBase { - - private static final Duration yarnAppTerminateTimeout = Duration.ofSeconds(10); - private static final int sleepIntervalInMS = 100; - - @BeforeAll - static void setup() { - YARN_CONFIGURATION.set(YarnTestBase.TEST_CLUSTER_NAME_KEY, "flink-yarn-tests-per-job"); - startYARNWithConfig(YARN_CONFIGURATION, true); - } - - @Test - void testPerJobModeWithEnableSystemClassPathIncludeUserJar() throws Exception { - runTest( - () -> - deployPerJob( - createDefaultConfiguration( - YarnConfigOptions.UserJarInclusion.FIRST), - getTestingJobGraph(), - true)); - } - - @Test - void testPerJobModeWithDisableSystemClassPathIncludeUserJar() throws Exception { - runTest( - () -> - deployPerJob( - createDefaultConfiguration( - YarnConfigOptions.UserJarInclusion.DISABLED), - getTestingJobGraph(), - true)); - } - - @Test - void testPerJobModeWithDistributedCache(@TempDir File tempDir) throws Exception { - runTest( - () -> - deployPerJob( - createDefaultConfiguration( - YarnConfigOptions.UserJarInclusion.DISABLED), - YarnTestCacheJob.getDistributedCacheJobGraph(tempDir), - true)); - } - - @Test - void testPerJobWithProvidedLibDirs() throws Exception { - final Path remoteLib = - new Path( - miniDFSCluster.getFileSystem().getUri().toString() + "/flink-provided-lib"); - miniDFSCluster - .getFileSystem() - .copyFromLocalFile(new Path(flinkLibFolder.toURI()), remoteLib); - miniDFSCluster.getFileSystem().setPermission(remoteLib, new FsPermission("755")); - - final Configuration flinkConfig = - createDefaultConfiguration(YarnConfigOptions.UserJarInclusion.DISABLED); - flinkConfig.set( - YarnConfigOptions.PROVIDED_LIB_DIRS, - Collections.singletonList(remoteLib.toString())); - runTest(() -> deployPerJob(flinkConfig, getTestingJobGraph(), false)); - } - - @Test - void testPerJobWithArchive(@TempDir File tempDir) throws Exception { - final Configuration flinkConfig = - createDefaultConfiguration(YarnConfigOptions.UserJarInclusion.DISABLED); - final JobGraph archiveJobGraph = - YarnTestArchiveJob.getArchiveJobGraph(tempDir, flinkConfig); - runTest(() -> deployPerJob(flinkConfig, archiveJobGraph, true)); - } - - private void deployPerJob(Configuration configuration, JobGraph jobGraph, boolean withDist) - throws Exception { - jobGraph.setJobType(JobType.STREAMING); - try (final YarnClusterDescriptor yarnClusterDescriptor = - withDist - ? createYarnClusterDescriptor(configuration) - : createYarnClusterDescriptorWithoutLibDir(configuration)) { - - final int masterMemory = - yarnClusterDescriptor - .getFlinkConfiguration() - .get(JobManagerOptions.TOTAL_PROCESS_MEMORY) - .getMebiBytes(); - final ClusterSpecification clusterSpecification = - new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(masterMemory) - .setTaskManagerMemoryMB(1024) - .setSlotsPerTaskManager(1) - .createClusterSpecification(); - - File testingJar = - TestUtils.findFile("..", new TestUtils.TestJarFinder("flink-yarn-tests")); - - jobGraph.addJar(new org.apache.flink.core.fs.Path(testingJar.toURI())); - try (ClusterClient clusterClient = - yarnClusterDescriptor - .deployJobCluster(clusterSpecification, jobGraph, false) - .getClusterClient()) { - - for (DistributedCache.DistributedCacheEntry entry : - jobGraph.getUserArtifacts().values()) { - assertThat(Utils.isRemotePath(entry.filePath)).isTrue(); - } - - ApplicationId applicationId = clusterClient.getClusterId(); - - final CompletableFuture jobResultCompletableFuture = - clusterClient.requestJobResult(jobGraph.getJobID()); - - final JobResult jobResult = jobResultCompletableFuture.get(); - - assertThat(jobResult).isNotNull(); - assertThat(jobResult.getSerializedThrowable()).isNotPresent(); - - checkStagingDirectory(configuration, applicationId); - - waitApplicationFinishedElseKillIt( - applicationId, - yarnAppTerminateTimeout, - yarnClusterDescriptor, - sleepIntervalInMS); - } - } - } - - private void checkStagingDirectory(Configuration flinkConfig, ApplicationId appId) - throws IOException { - final List providedLibDirs = flinkConfig.get(YarnConfigOptions.PROVIDED_LIB_DIRS); - final boolean isProvidedLibDirsConfigured = - providedLibDirs != null && !providedLibDirs.isEmpty(); - - try (final FileSystem fs = FileSystem.get(YARN_CONFIGURATION)) { - final Path stagingDirectory = - new Path(fs.getHomeDirectory(), ".flink/" + appId.toString()); - if (isProvidedLibDirsConfigured) { - assertThat(fs.exists(new Path(stagingDirectory, flinkLibFolder.getName()))) - .isFalse(); - } else { - assertThat(fs.exists(new Path(stagingDirectory, flinkLibFolder.getName()))) - .isTrue(); - } - } - } - - private JobGraph getTestingJobGraph() { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(2); - - env.addSource(new NoDataSource()).shuffle().sinkTo(new DiscardingSink<>()); - - return env.getStreamGraph().getJobGraph(); - } - - private Configuration createDefaultConfiguration( - YarnConfigOptions.UserJarInclusion userJarInclusion) { - Configuration configuration = new Configuration(); - configuration.set(JobManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(768)); - configuration.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse("1g")); - configuration.set(RpcOptions.ASK_TIMEOUT_DURATION, Duration.ofSeconds(30)); - configuration.set(CLASSPATH_INCLUDE_USER_JAR, userJarInclusion); - - return configuration; - } -} diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index 9af20b0d2e6c76..c10f947dc1f382 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -51,10 +51,8 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -95,6 +93,9 @@ class YARNSessionCapacitySchedulerITCase extends YarnTestBase { private static final Logger LOG = LoggerFactory.getLogger(YARNSessionCapacitySchedulerITCase.class); + private static final ApplicationId TEST_YARN_APPLICATION_ID = + ApplicationId.newInstance(System.currentTimeMillis(), 42); + /** RestClient to query Flink cluster. */ private static RestClient restClient; @@ -173,100 +174,6 @@ void testStartYarnSessionClusterInQaTeamQueue() throws Exception { 0)); } - /** - * Test per-job yarn cluster - * - *

This also tests the prefixed CliFrontend options for the YARN case We also test if the - * requested parallelism of 2 is passed through. The parallelism is requested at the YARN client - * (-ys). - */ - @Test - @Disabled("per-job mode is deprecated, we will be removed in 2.0 version.") - void perJobYarnCluster() throws Exception { - runTest( - () -> { - LOG.info("Starting perJobYarnCluster()"); - File exampleJarLocation = getTestJarPath("BatchWordCount.jar"); - runWithArgs( - new String[] { - "run", - "-m", - "yarn-cluster", - "-yj", - flinkUberjar.getAbsolutePath(), - "-yt", - flinkLibFolder.getAbsolutePath(), - "-ys", - "2", // test that the job is executed with a DOP of 2 - "-yjm", - "768m", - "-ytm", - "1024m", - exampleJarLocation.getAbsolutePath() - }, - /* test succeeded after this string */ - "Program execution finished", - /* prohibited strings: (to verify the parallelism) */ - // (we should see "DataSink (...) (1/2)" and "DataSink (...) (2/2)" - // instead) - new String[] {"DataSink \\(.*\\) \\(1/1\\) switched to FINISHED"}, - RunTypes.CLI_FRONTEND, - 0, - cliLoggerAuditingExtension::getMessages); - LOG.info("Finished perJobYarnCluster()"); - }); - } - - /** - * Test per-job yarn cluster and memory calculations for off-heap use (see FLINK-7400) with the - * same job as {@link #perJobYarnCluster()}. - * - *

This ensures that with (any) pre-allocated off-heap memory by us, there is some off-heap - * memory remaining for Flink's libraries. Creating task managers will thus fail if no off-heap - * memory remains. - */ - @Test - @Disabled("per-job mode is deprecated, we will be removed in 2.0 version.") - void perJobYarnClusterOffHeap() throws Exception { - runTest( - () -> { - LOG.info("Starting perJobYarnCluster()"); - File exampleJarLocation = getTestJarPath("BatchWordCount.jar"); - - // set memory constraints (otherwise this is the same test as - // perJobYarnCluster() above) - final long taskManagerMemoryMB = 1024; - - runWithArgs( - new String[] { - "run", - "-m", - "yarn-cluster", - "-yj", - flinkUberjar.getAbsolutePath(), - "-yt", - flinkLibFolder.getAbsolutePath(), - "-ys", - "2", // test that the job is executed with a DOP of 2 - "-yjm", - "768m", - "-ytm", - taskManagerMemoryMB + "m", - exampleJarLocation.getAbsolutePath() - }, - /* test succeeded after this string */ - "Program execution finished", - /* prohibited strings: (to verify the parallelism) */ - // (we should see "DataSink (...) (1/2)" and "DataSink (...) (2/2)" - // instead) - new String[] {"DataSink \\(.*\\) \\(1/1\\) switched to FINISHED"}, - RunTypes.CLI_FRONTEND, - 0, - cliLoggerAuditingExtension::getMessages); - LOG.info("Finished perJobYarnCluster()"); - }); - } - /** * Starts a session cluster on YARN, and submits a streaming job. * @@ -480,81 +387,6 @@ void testNonexistingQueueWARNmessage() throws Exception { }); } - /** - * Test per-job yarn cluster with the parallelism set at the CliFrontend instead of the YARN - * client. - */ - @Test - @Disabled("per-job mode is deprecated, we will be removed in 2.0 version.") - void perJobYarnClusterWithParallelism() throws Exception { - runTest( - () -> { - LOG.info("Starting perJobYarnClusterWithParallelism()"); - File exampleJarLocation = getTestJarPath("BatchWordCount.jar"); - runWithArgs( - new String[] { - "run", - "-p", - "2", // test that the job is executed with a DOP of 2 - "-m", - "yarn-cluster", - "-yj", - flinkUberjar.getAbsolutePath(), - "-yt", - flinkLibFolder.getAbsolutePath(), - "-ys", - "2", - "-yjm", - "768m", - "-ytm", - "1024m", - exampleJarLocation.getAbsolutePath() - }, - /* test succeeded after this string */ - "Program execution finished", - /* prohibited strings: (we want to see "DataSink (...) (2/2) switched to FINISHED") */ - new String[] {"DataSink \\(.*\\) \\(1/1\\) switched to FINISHED"}, - RunTypes.CLI_FRONTEND, - 0, - cliLoggerAuditingExtension::getMessages); - LOG.info("Finished perJobYarnClusterWithParallelism()"); - }); - } - - /** Test a fire-and-forget job submission to a YARN cluster. */ - @Test - @Disabled("per-job mode is deprecated, we will be removed in 2.0 version.") - void testDetachedPerJobYarnCluster(@TempDir File tempDir) throws Exception { - runTest( - () -> { - LOG.info("Starting testDetachedPerJobYarnCluster()"); - - File exampleJarLocation = getTestJarPath("BatchWordCount.jar"); - - testDetachedPerJobYarnClusterInternal( - tempDir, exampleJarLocation.getAbsolutePath()); - - LOG.info("Finished testDetachedPerJobYarnCluster()"); - }); - } - - /** Test a fire-and-forget job submission to a YARN cluster. */ - @Test - @Disabled("per-job mode is deprecated, we will be removed in 2.0 version.") - void testDetachedPerJobYarnClusterWithStreamingJob(@TempDir File tempDir) throws Exception { - runTest( - () -> { - LOG.info("Starting testDetachedPerJobYarnClusterWithStreamingJob()"); - - File exampleJarLocation = getTestJarPath("StreamingWordCount.jar"); - - testDetachedPerJobYarnClusterInternal( - tempDir, exampleJarLocation.getAbsolutePath()); - - LOG.info("Finished testDetachedPerJobYarnClusterWithStreamingJob()"); - }); - } - private void testDetachedPerJobYarnClusterInternal(File tempDir, String job) throws Exception { YarnClient yc = YarnClient.createYarnClient(); yc.init(YARN_CONFIGURATION); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java index 5a506732ed8aab..15b310245a782c 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java @@ -19,16 +19,16 @@ package org.apache.flink.yarn; import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.deployment.application.ApplicationConfiguration; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.PackagedProgram; -import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rest.RestClient; import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; @@ -47,7 +47,6 @@ import org.assertj.core.data.Offset; import org.junit.jupiter.api.Test; -import java.io.File; import java.net.URI; import java.time.Duration; import java.util.Arrays; @@ -79,6 +78,10 @@ void testFlinkContainerMemory() throws Exception { configuration.set( JobManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(masterMemory)); + configuration.set(DeploymentOptions.TARGET, "yarn-application"); + configuration.setString( + PipelineOptions.JARS.key(), + getTestJarPath("WindowJoin.jar").getAbsolutePath()); final TaskExecutorProcessSpec tmResourceSpec = TaskExecutorProcessUtils.processSpecFromConfig(configuration); @@ -100,14 +103,6 @@ void testFlinkContainerMemory() throws Exception { .map(file -> new Path(file.toURI())) .collect(Collectors.toList())); - final File streamingWordCountFile = getTestJarPath("WindowJoin.jar"); - - final PackagedProgram packagedProgram = - PackagedProgram.newBuilder().setJarFile(streamingWordCountFile).build(); - final JobGraph jobGraph = - PackagedProgramUtils.createJobGraph( - packagedProgram, configuration, 1, false); - try { final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() @@ -118,7 +113,10 @@ void testFlinkContainerMemory() throws Exception { final ClusterClient clusterClient = clusterDescriptor - .deployJobCluster(clusterSpecification, jobGraph, true) + .deployApplicationCluster( + clusterSpecification, + ApplicationConfiguration.fromConfiguration( + configuration)) .getClusterClient(); final ApplicationId clusterId = clusterClient.getClusterId(); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index c1c77c197d0c23..c6da95e9bb705e 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -19,8 +19,6 @@ package org.apache.flink.yarn; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.cache.DistributedCache; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.deployment.ClusterDeploymentException; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.deployment.ClusterRetrieveException; @@ -71,7 +69,6 @@ import org.apache.flink.yarn.configuration.YarnDeploymentTarget; import org.apache.flink.yarn.configuration.YarnLogConfigUtil; import org.apache.flink.yarn.entrypoint.YarnApplicationClusterEntryPoint; -import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; import org.apache.hadoop.fs.FileStatus; @@ -313,14 +310,6 @@ protected String getYarnSessionClusterEntrypoint() { return YarnSessionClusterEntrypoint.class.getName(); } - /** - * The class to start the application master with. This class runs the main method in case of - * the job cluster. - */ - protected String getYarnJobClusterEntrypoint() { - return YarnJobClusterEntrypoint.class.getName(); - } - public Configuration getFlinkConfiguration() { return flinkConfiguration; } @@ -550,25 +539,6 @@ public ClusterClientProvider deployApplicationCluster( } } - @Override - public ClusterClientProvider deployJobCluster( - ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) - throws ClusterDeploymentException { - - LOG.warn( - "Job Clusters are deprecated since Flink 1.15. Please use an Application Cluster/Application Mode instead."); - try { - return deployInternal( - clusterSpecification, - "Flink per-job cluster", - getYarnJobClusterEntrypoint(), - jobGraph, - detached); - } catch (Exception e) { - throw new ClusterDeploymentException("Could not deploy Yarn job cluster.", e); - } - } - @Override public void killCluster(ApplicationId applicationId) throws FlinkException { try { @@ -943,23 +913,6 @@ private ApplicationReport startAppMaster( userJarFiles.addAll(jarUrls.stream().map(Path::new).collect(Collectors.toSet())); } - // only for per job mode - if (jobGraph != null) { - for (Map.Entry entry : - jobGraph.getUserArtifacts().entrySet()) { - // only upload local files - if (!Utils.isRemotePath(entry.getValue().filePath)) { - Path localPath = new Path(entry.getValue().filePath); - Tuple2 remoteFileInfo = - fileUploader.uploadLocalFileToRemote(localPath, entry.getKey()); - jobGraph.setUserArtifactRemotePath( - entry.getKey(), remoteFileInfo.f0.toString()); - } - } - - jobGraph.writeUserArtifactEntriesToConfiguration(); - } - if (providedLibDirs == null || providedLibDirs.isEmpty()) { addLibFoldersToShipFiles(systemShipFiles); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/AbstractYarnCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/AbstractYarnCli.java index e7e2891c3506d2..6bedf795128e96 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/AbstractYarnCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/AbstractYarnCli.java @@ -22,7 +22,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.yarn.configuration.YarnConfigOptions; -import org.apache.flink.yarn.executors.YarnJobClusterExecutor; import org.apache.flink.yarn.executors.YarnSessionClusterExecutor; import org.apache.commons.cli.CommandLine; @@ -59,9 +58,7 @@ public boolean isActive(CommandLine commandLine) { || configuration.getOptional(YarnConfigOptions.APPLICATION_ID).isPresent(); final boolean hasYarnExecutor = YarnSessionClusterExecutor.NAME.equalsIgnoreCase( - configuration.get(DeploymentOptions.TARGET)) - || YarnJobClusterExecutor.NAME.equalsIgnoreCase( - configuration.get(DeploymentOptions.TARGET)); + configuration.get(DeploymentOptions.TARGET)); return hasYarnExecutor || yarnJobManager || hasYarnAppId; } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 4386ebc3c8958d..66fe7b1b5ea199 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -48,7 +48,6 @@ import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.flink.yarn.configuration.YarnDeploymentTarget; -import org.apache.flink.yarn.executors.YarnJobClusterExecutor; import org.apache.flink.yarn.executors.YarnSessionClusterExecutor; import org.apache.commons.cli.CommandLine; @@ -404,10 +403,8 @@ public Configuration toConfiguration(CommandLine commandLine) throws FlinkExcept effectiveConfiguration.set(HA_CLUSTER_ID, zooKeeperNamespace); effectiveConfiguration.set(YarnConfigOptions.APPLICATION_ID, applicationId.toString()); - effectiveConfiguration.set(DeploymentOptions.TARGET, YarnSessionClusterExecutor.NAME); - } else { - effectiveConfiguration.set(DeploymentOptions.TARGET, YarnJobClusterExecutor.NAME); } + effectiveConfiguration.set(DeploymentOptions.TARGET, YarnSessionClusterExecutor.NAME); if (commandLine.hasOption(jmMemory.getOpt())) { String jmMemoryVal = commandLine.getOptionValue(jmMemory.getOpt()); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnDeploymentTarget.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnDeploymentTarget.java index c7d22c77ca2822..672da106aa4193 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnDeploymentTarget.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnDeploymentTarget.java @@ -31,10 +31,7 @@ @Internal public enum YarnDeploymentTarget { SESSION("yarn-session"), - APPLICATION("yarn-application"), - - @Deprecated - PER_JOB("yarn-per-job"); + APPLICATION("yarn-application"); public static final String ERROR_MESSAGE = "No Executor found. Please make sure to export the HADOOP_CLASSPATH environment variable " @@ -81,9 +78,7 @@ private static YarnDeploymentTarget getFromName(final String deploymentTarget) { return null; } - if (PER_JOB.name.equalsIgnoreCase(deploymentTarget)) { - return PER_JOB; - } else if (SESSION.name.equalsIgnoreCase(deploymentTarget)) { + if (SESSION.name.equalsIgnoreCase(deploymentTarget)) { return SESSION; } else if (APPLICATION.name.equalsIgnoreCase(deploymentTarget)) { return APPLICATION; diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java index 820a91e93e4384..bfe36d5e7502ab 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java @@ -41,10 +41,7 @@ import static org.apache.flink.runtime.entrypoint.ClusterEntrypointUtils.tryFindUserLibDirectory; import static org.apache.flink.util.Preconditions.checkState; -/** - * This class contains utility methods for the {@link YarnSessionClusterEntrypoint} and {@link - * YarnJobClusterEntrypoint}. - */ +/** This class contains utility methods for the {@link YarnSessionClusterEntrypoint}. */ public class YarnEntrypointUtils { public static Configuration loadConfiguration( diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java deleted file mode 100644 index 04a48c7d5cd976..00000000000000 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.yarn.entrypoint; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; -import org.apache.flink.runtime.entrypoint.ClusterEntrypointUtils; -import org.apache.flink.runtime.entrypoint.DynamicParametersConfigurationParserFactory; -import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; -import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; -import org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever; -import org.apache.flink.runtime.util.EnvironmentInformation; -import org.apache.flink.runtime.util.JvmShutdownSafeguard; -import org.apache.flink.runtime.util.SignalHandler; -import org.apache.flink.util.Preconditions; -import org.apache.flink.yarn.configuration.YarnConfigOptions; - -import org.apache.hadoop.yarn.api.ApplicationConstants; - -import java.io.IOException; -import java.util.Map; - -/** - * Entry point for Yarn per-job clusters. - * - * @deprecated Per-mode has been deprecated in Flink 1.15 and will be removed in the future. Please - * use application mode instead. - */ -@Deprecated -public class YarnJobClusterEntrypoint extends JobClusterEntrypoint { - - public YarnJobClusterEntrypoint(Configuration configuration) { - super(configuration); - } - - @Override - protected String getRPCPortRange(Configuration configuration) { - return configuration.get(YarnConfigOptions.APPLICATION_MASTER_PORT); - } - - @Override - protected DefaultDispatcherResourceManagerComponentFactory - createDispatcherResourceManagerComponentFactory(Configuration configuration) - throws IOException { - return DefaultDispatcherResourceManagerComponentFactory.createJobComponentFactory( - YarnResourceManagerFactory.getInstance(), - FileJobGraphRetriever.createFrom( - configuration, - YarnEntrypointUtils.getUsrLibDir(configuration).orElse(null))); - } - - // ------------------------------------------------------------------------ - // The executable entry point for the Yarn Application Master Process - // for a single Flink job. - // ------------------------------------------------------------------------ - - public static void main(String[] args) { - - LOG.warn( - "Job Clusters are deprecated since Flink 1.15. Please use an Application Cluster/Application Mode instead."); - - // startup checks and logging - EnvironmentInformation.logEnvironmentInfo( - LOG, YarnJobClusterEntrypoint.class.getSimpleName(), args); - SignalHandler.register(LOG); - JvmShutdownSafeguard.installAsShutdownHook(LOG); - - Map env = System.getenv(); - - final String workingDirectory = env.get(ApplicationConstants.Environment.PWD.key()); - Preconditions.checkArgument( - workingDirectory != null, - "Working directory variable (%s) not set", - ApplicationConstants.Environment.PWD.key()); - - try { - YarnEntrypointUtils.logYarnEnvironmentInformation(env, LOG); - } catch (IOException e) { - LOG.warn("Could not log YARN environment information.", e); - } - - final Configuration dynamicParameters = - ClusterEntrypointUtils.parseParametersOrExit( - args, - new DynamicParametersConfigurationParserFactory(), - YarnJobClusterEntrypoint.class); - final Configuration configuration = - YarnEntrypointUtils.loadConfiguration(workingDirectory, dynamicParameters, env); - - YarnJobClusterEntrypoint yarnJobClusterEntrypoint = - new YarnJobClusterEntrypoint(configuration); - - ClusterEntrypoint.runClusterEntrypoint(yarnJobClusterEntrypoint); - } -} diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/executors/YarnJobClusterExecutor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/executors/YarnJobClusterExecutor.java deleted file mode 100644 index f797a4c9791c03..00000000000000 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/executors/YarnJobClusterExecutor.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.yarn.executors; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.client.deployment.executors.AbstractJobClusterExecutor; -import org.apache.flink.core.execution.PipelineExecutor; -import org.apache.flink.yarn.YarnClusterClientFactory; -import org.apache.flink.yarn.configuration.YarnDeploymentTarget; - -import org.apache.hadoop.yarn.api.records.ApplicationId; - -/** - * The {@link PipelineExecutor} to be used when executing a job in isolation. This executor will - * start a cluster specifically for the job at hand and tear it down when the job is finished either - * successfully or due to an error. - */ -@Internal -@Deprecated -public class YarnJobClusterExecutor - extends AbstractJobClusterExecutor { - - public static final String NAME = YarnDeploymentTarget.PER_JOB.getName(); - - public YarnJobClusterExecutor() { - super(new YarnClusterClientFactory()); - } -} diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/executors/YarnJobClusterExecutorFactory.java b/flink-yarn/src/main/java/org/apache/flink/yarn/executors/YarnJobClusterExecutorFactory.java deleted file mode 100644 index 4ef44b76826a83..00000000000000 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/executors/YarnJobClusterExecutorFactory.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.yarn.executors; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.DeploymentOptions; -import org.apache.flink.core.execution.PipelineExecutor; -import org.apache.flink.core.execution.PipelineExecutorFactory; -import org.apache.flink.yarn.configuration.YarnDeploymentTarget; - -import javax.annotation.Nonnull; - -/** An {@link PipelineExecutorFactory} for executing jobs on dedicated (per-job) clusters. */ -@Internal -@Deprecated -public class YarnJobClusterExecutorFactory implements PipelineExecutorFactory { - - @Override - public String getName() { - return YarnJobClusterExecutor.NAME; - } - - @Override - public boolean isCompatibleWith(@Nonnull final Configuration configuration) { - return YarnJobClusterExecutor.NAME.equalsIgnoreCase( - configuration.get(DeploymentOptions.TARGET)); - } - - @Override - public PipelineExecutor getExecutor(@Nonnull final Configuration configuration) { - try { - return new YarnJobClusterExecutor(); - } catch (NoClassDefFoundError e) { - throw new IllegalStateException(YarnDeploymentTarget.ERROR_MESSAGE); - } - } -} diff --git a/flink-yarn/src/main/resources/META-INF/services/org.apache.flink.core.execution.PipelineExecutorFactory b/flink-yarn/src/main/resources/META-INF/services/org.apache.flink.core.execution.PipelineExecutorFactory index 6899f90d2546af..b3e53fc8504512 100644 --- a/flink-yarn/src/main/resources/META-INF/services/org.apache.flink.core.execution.PipelineExecutorFactory +++ b/flink-yarn/src/main/resources/META-INF/services/org.apache.flink.core.execution.PipelineExecutorFactory @@ -13,5 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.yarn.executors.YarnJobClusterExecutorFactory org.apache.flink.yarn.executors.YarnSessionClusterExecutorFactory diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index cc2b719107e3bb..29b9e8f2e1ceac 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -40,7 +40,6 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.flink.yarn.configuration.YarnConfigOptions; -import org.apache.flink.yarn.executors.YarnJobClusterExecutor; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; @@ -180,21 +179,6 @@ void testNodeLabelProperty() throws Exception { assertThat(descriptor.getNodeLabel()).isEqualTo(nodeLabelCliInput); } - @Test - void testExecutorCLIisPrioritised() throws Exception { - final File directoryPath = writeYarnPropertiesFile(validPropertiesFile); - - final Configuration configuration = new Configuration(); - configuration.set( - YarnConfigOptions.PROPERTIES_FILE_LOCATION, directoryPath.getAbsolutePath()); - - validateYarnCLIisActive(configuration); - - final String[] argsUnderTest = new String[] {"-e", YarnJobClusterExecutor.NAME}; - - validateExecutorCLIisPrioritised(configuration, argsUnderTest); - } - private void validateExecutorCLIisPrioritised( Configuration configuration, String[] argsUnderTest) throws IOException, CliArgsException { diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterClientFactoryTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterClientFactoryTest.java index 8c51c3184b2a8e..db6c3efd24ffdb 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterClientFactoryTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterClientFactoryTest.java @@ -23,7 +23,6 @@ import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; -import org.apache.flink.yarn.executors.YarnJobClusterExecutor; import org.apache.flink.yarn.executors.YarnSessionClusterExecutor; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -34,11 +33,6 @@ /** Test for the {@link YarnClusterClientFactory} discovery. */ class YarnClusterClientFactoryTest { - @Test - void testYarnClusterClientFactoryDiscoveryWithPerJobExecutor() { - testYarnClusterClientFactoryDiscoveryHelper(YarnJobClusterExecutor.NAME); - } - @Test void testYarnClusterClientFactoryDiscoveryWithSessionExecutor() { testYarnClusterClientFactoryDiscoveryHelper(YarnSessionClusterExecutor.NAME); diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypointTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypointTest.java deleted file mode 100644 index 9d3eaa52305715..00000000000000 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypointTest.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.yarn.entrypoint; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.yarn.configuration.YarnConfigOptions; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link YarnJobClusterEntrypoint}. */ -class YarnJobClusterEntrypointTest { - - @Test - void testCreateDispatcherResourceManagerComponentFactoryFailIfUsrLibDirDoesNotExist() { - final Configuration configuration = new Configuration(); - configuration.set( - YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR, - YarnConfigOptions.UserJarInclusion.DISABLED); - final YarnJobClusterEntrypoint yarnJobClusterEntrypoint = - new YarnJobClusterEntrypoint(configuration); - assertThatThrownBy( - () -> - yarnJobClusterEntrypoint - .createDispatcherResourceManagerComponentFactory( - configuration)) - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("the usrlib directory does not exist."); - } -}