Skip to content

Commit

Permalink
[FLINK-23118] Drop Mesos support
Browse files Browse the repository at this point in the history
  • Loading branch information
zentol committed Jun 25, 2021
1 parent 66120f9 commit e8b7f3d
Show file tree
Hide file tree
Showing 154 changed files with 133 additions and 13,040 deletions.
2 changes: 1 addition & 1 deletion .github/PULL_REQUEST_TEMPLATE.md
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ This change added tests and can be verified as follows:
- The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / no)
- The serializers: (yes / no / don't know)
- The runtime per-record code paths (performance sensitive): (yes / no / don't know)
- Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / no / don't know)
- Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / no / don't know)
- The S3 file system connector: (yes / no / don't know)

## Documentation
Expand Down
6 changes: 3 additions & 3 deletions docs/content.zh/docs/concepts/flink-architecture.md
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ under the License.

# Flink 架构

Flink 是一个分布式系统,需要有效分配和管理计算资源才能执行流应用程序。它集成了所有常见的集群资源管理器,例如[Hadoop YARN](https://hadoop.apache.org/docs/stable/hadoop-yarn/hadoop-yarn-site/YARN.html)[Apache Mesos](https://mesos.apache.org/)[Kubernetes](https://kubernetes.io/),但也可以设置作为独立集群甚至库运行。
Flink 是一个分布式系统,需要有效分配和管理计算资源才能执行流应用程序。它集成了所有常见的集群资源管理器,例如[Hadoop YARN](https://hadoop.apache.org/docs/stable/hadoop-yarn/hadoop-yarn-site/YARN.html),但也可以设置作为独立集群甚至库运行。

本节概述了 Flink 架构,并且描述了其主要组件如何交互以执行应用程序和从故障中恢复。

Expand All @@ -37,15 +37,15 @@ Flink 运行时由两种类型的进程组成:一个 _JobManager_ 和一个或

*Client* 不是运行时和程序执行的一部分,而是用于准备数据流并将其发送给 JobManager。之后,客户端可以断开连接(_分离模式_),或保持连接来接收进程报告(_附加模式_)。客户端可以作为触发执行 Java/Scala 程序的一部分运行,也可以在命令行进程`./bin/flink run ...`中运行。

可以通过多种方式启动 JobManager 和 TaskManager:直接在机器上作为[standalone 集群]({{< ref "docs/deployment/resource-providers/standalone/overview" >}})启动、在容器中启动、或者通过[YARN]({{< ref "docs/deployment/resource-providers/yarn" >}})[Mesos]({{< ref "docs/deployment/resource-providers/mesos" >}})等资源框架管理并启动。TaskManager 连接到 JobManagers,宣布自己可用,并被分配工作。
可以通过多种方式启动 JobManager 和 TaskManager:直接在机器上作为[standalone 集群]({{< ref "docs/deployment/resource-providers/standalone/overview" >}})启动、在容器中启动、或者通过[YARN]({{< ref "docs/deployment/resource-providers/yarn" >}})等资源框架管理并启动。TaskManager 连接到 JobManagers,宣布自己可用,并被分配工作。

### JobManager

_JobManager_ 具有许多与协调 Flink 应用程序的分布式执行有关的职责:它决定何时调度下一个 task(或一组 task)、对完成的 task 或执行失败做出反应、协调 checkpoint、并且协调从失败中恢复等等。这个进程由三个不同的组件组成:

* **ResourceManager**

_ResourceManager_ 负责 Flink 集群中的资源提供、回收、分配 - 它管理 **task slots**,这是 Flink 集群中资源调度的单位(请参考[TaskManagers](#taskmanagers))。Flink 为不同的环境和资源提供者(例如 YARN、Mesos、Kubernetes 和 standalone 部署)实现了对应的 ResourceManager。在 standalone 设置中,ResourceManager 只能分配可用 TaskManager 的 slots,而不能自行启动新的 TaskManager。
_ResourceManager_ 负责 Flink 集群中的资源提供、回收、分配 - 它管理 **task slots**,这是 Flink 集群中资源调度的单位(请参考[TaskManagers](#taskmanagers))。Flink 为不同的环境和资源提供者(例如 YARN、Kubernetes 和 standalone 部署)实现了对应的 ResourceManager。在 standalone 设置中,ResourceManager 只能分配可用 TaskManager 的 slots,而不能自行启动新的 TaskManager。

* **Dispatcher**

Expand Down
2 changes: 1 addition & 1 deletion docs/content.zh/docs/connectors/datastream/kafka.md
Original file line number Diff line number Diff line change
Expand Up @@ -457,7 +457,7 @@ Flink 的 Kafka 连接器通过 Flink 的 [metric 系统]({{< ref "docs/ops/metr
Flink 通过 Kafka 连接器提供了一流的支持,可以对 Kerberos 配置的 Kafka 安装进行身份验证。只需在 `flink-conf.yaml` 中配置 Flink。像这样为 Kafka 启用 Kerberos 身份验证:

1. 通过设置以下内容配置 Kerberos 票据
- `security.kerberos.login.use-ticket-cache`:默认情况下,这个值是 `true`,Flink 将尝试在 `kinit` 管理的票据缓存中使用 Kerberos 票据。注意!在 YARN 上部署的 Flink jobs 中使用 Kafka 连接器时,使用票据缓存的 Kerberos 授权将不起作用。使用 Mesos 进行部署时也是如此,因为 Mesos 部署不支持使用票据缓存进行授权。
- `security.kerberos.login.use-ticket-cache`:默认情况下,这个值是 `true`,Flink 将尝试在 `kinit` 管理的票据缓存中使用 Kerberos 票据。注意!在 YARN 上部署的 Flink jobs 中使用 Kafka 连接器时,使用票据缓存的 Kerberos 授权将不起作用。
- `security.kerberos.login.keytab``security.kerberos.login.principal`:要使用 Kerberos keytabs,需为这两个属性设置值。

2.`KafkaClient` 追加到 `security.kerberos.login.contexts`:这告诉 Flink 将配置的 Kerberos 票据提供给 Kafka 登录上下文以用于 Kafka 身份验证。
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ under the License.

当 Flink 部署在资源管理系统(Kubernetes、Yarn)上时,扩展资源框架将确保分配的 Pod、Container 包含所需的扩展资源。目前,许多资源管理系统都支持扩展资源。
例如,Kubernetes 从 v1.10 开始通过 [Device Plugin](https://kubernetes.io/docs/concepts/extend-kubernetes/compute-storage-net/device-plugins/) 机制支持 GPU、FPGA 等资源调度,Yarn 从 2.10 和 3.1 开始支持 GPU 和 FPGA 的调度。
目前,扩展资源框架并不支持 Mesos 模式。在 Standalone 模式下,由用户负责确保扩展资源的可用性。
在 Standalone 模式下,由用户负责确保扩展资源的可用性。

扩展资源框架向算子提供扩展资源相关*信息*,这些信息由你配置的扩展资源 *Driver* 生成,包含了使用扩展资源所需要的基本属性。

Expand Down
2 changes: 0 additions & 2 deletions docs/content.zh/docs/deployment/cli.md
Original file line number Diff line number Diff line change
Expand Up @@ -335,8 +335,6 @@ parameter combinations:
* Kubernetes
* `./bin/flink run --target kubernetes-session`: Submission to an already running Flink on Kubernetes cluster
* `./bin/flink run-application --target kubernetes-application`: Submission spinning up a Flink on Kubernetes cluster in Application Mode
* Mesos
* `./bin/flink run --target remote`: Submission to an already running Flink on Mesos cluster
* Standalone:
* `./bin/flink run --target local`: Local submission using a MiniCluster in Session Mode
* `./bin/flink run --target remote`: Submission to an already running Flink cluster
Expand Down
25 changes: 6 additions & 19 deletions docs/content.zh/docs/deployment/config.md
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ The options in this section are the ones most commonly needed for a basic distri

These options are only necessary for *standalone* application- or session deployments ([simple standalone]({{< ref "docs/deployment/resource-providers/standalone/overview" >}}) or [Kubernetes]({{< ref "docs/deployment/resource-providers/standalone/kubernetes" >}})).

If you use Flink with [Yarn]({{< ref "docs/deployment/resource-providers/yarn" >}}), [Mesos]({{< ref "docs/deployment/resource-providers/mesos" >}}), or the [*active* Kubernetes integration]({{< ref "docs/deployment/resource-providers/native_kubernetes" >}}), the hostnames and ports are automatically discovered.
If you use Flink with [Yarn]({{< ref "docs/deployment/resource-providers/yarn" >}}) or the [*active* Kubernetes integration]({{< ref "docs/deployment/resource-providers/native_kubernetes" >}}), the hostnames and ports are automatically discovered.

- `rest.address`, `rest.port`: These are used by the client to connect to Flink. Set this to the hostname where the JobManager runs, or to the hostname of the (Kubernetes) service in front of the JobManager's REST interface.

Expand Down Expand Up @@ -95,7 +95,7 @@ You can configure checkpointing directly in code within your Flink job or applic

This data is NOT relied upon for persistence/recovery, but if this data gets deleted, it typically causes a heavyweight recovery operation. It is hence recommended to set this to a directory that is not automatically periodically purged.

Yarn, Mesos, and Kubernetes setups automatically configure this value to the local working directories by default.
Yarn and Kubernetes setups automatically configure this value to the local working directories by default.

----
----
Expand All @@ -112,7 +112,7 @@ The JobManager hostname and port are only relevant for standalone setups without
In that setup, the config values are used by the TaskManagers to find (and connect to) the JobManager.
In all highly-available setups, the TaskManagers discover the JobManager via the High-Availability-Service (for example ZooKeeper).

Setups using resource orchestration frameworks (K8s, Yarn, Mesos) typically use the framework's service discovery facilities.
Setups using resource orchestration frameworks (K8s, Yarn) typically use the framework's service discovery facilities.

You do not need to configure any TaskManager hosts and ports, unless the setup requires the use of specific port ranges or specific network interfaces to bind to.

Expand Down Expand Up @@ -206,7 +206,7 @@ Please refer to the [Flink and Kerberos Docs]({{< ref "docs/deployment/security/

# Resource Orchestration Frameworks

This section contains options related to integrating Flink with resource orchestration frameworks, like Kubernetes, Yarn, Mesos, etc.
This section contains options related to integrating Flink with resource orchestration frameworks, like Kubernetes, Yarn, etc.

Note that is not always necessary to integrate Flink with the resource orchestration framework.
For example, you can easily deploy Flink applications on Kubernetes without Flink knowing that it runs on Kubernetes (and without specifying any of the Kubernetes config options here.) See [this setup guide]({{< ref "docs/deployment/resource-providers/standalone/kubernetes" >}}) for an example.
Expand All @@ -221,19 +221,6 @@ The options in this section are necessary for setups where Flink itself actively

{{< generated/kubernetes_config_configuration >}}

### Mesos

{{< hint warning >}}
Apache Mesos support was deprecated in Flink 1.13 and is subject to removal in the future (see
[FLINK-22352](https://issues.apache.org/jira/browse/FLINK-22352) for further details).
{{< /hint >}}

{{< generated/mesos_configuration >}}

**Mesos TaskManager**

{{< generated/mesos_task_manager_configuration >}}

----
----

Expand Down Expand Up @@ -413,7 +400,7 @@ The Blob Server is a component in the JobManager. It is used for distribution of

**ResourceManager**

These configuration keys control basic Resource Manager behavior, independent of the used resource orchestration management framework (YARN, Mesos, etc.)
These configuration keys control basic Resource Manager behavior, independent of the used resource orchestration management framework (YARN, etc.)

{{< generated/resource_manager_configuration >}}

Expand Down Expand Up @@ -443,7 +430,7 @@ Flink does not use Akka for data transport.

# Forwarding Environment Variables

You can configure environment variables to be set on the JobManager and TaskManager processes started on Yarn/Mesos.
You can configure environment variables to be set on the JobManager and TaskManager processes started on Yarn.

- `containerized.master.env.`: Prefix for passing custom environment variables to Flink's JobManager process.
For example for passing LD_LIBRARY_PATH as an env variable to the JobManager, set containerized.master.env.LD_LIBRARY_PATH: "/usr/lib/native"
Expand Down
2 changes: 1 addition & 1 deletion docs/content.zh/docs/deployment/elastic_scaling.md
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ Additionally, one can configure [`jobmanager.adaptive-scheduler.min-parallelism-

Since Reactive Mode is a new, experimental feature, not all features supported by the default scheduler are also available with Reactive Mode (and its adaptive scheduler). The Flink community is working on addressing these limitations.

- **Deployment is only supported as a standalone application deployment**. Active resource providers (such as native Kubernetes, YARN or Mesos) are explicitly not supported. Standalone session clusters are not supported either. The application deployment is limited to single job applications.
- **Deployment is only supported as a standalone application deployment**. Active resource providers (such as native Kubernetes, YARN) are explicitly not supported. Standalone session clusters are not supported either. The application deployment is limited to single job applications.

The only supported deployment options are [Standalone in Application Mode]({{< ref "docs/deployment/resource-providers/standalone/overview" >}}#application-mode) ([described](#getting-started) on this page), [Docker in Application Mode]({{< ref "docs/deployment/resource-providers/standalone/docker" >}}#application-mode-on-docker) and [Standalone Kubernetes Application Cluster]({{< ref "docs/deployment/resource-providers/standalone/kubernetes" >}}#deploy-application-cluster).

Expand Down
13 changes: 4 additions & 9 deletions docs/content.zh/docs/deployment/memory/mem_migration.md
Original file line number Diff line number Diff line change
Expand Up @@ -150,11 +150,9 @@ Flink 自带的[默认 flink-conf.yaml](#default-configuration-in-flink-confyaml
尽管这两个参数以“堆(Heap)”命名,实际上它们指定的内存既包含了 JVM 堆内存,也包含了其他堆外内存部分。
这两个配置参数目前已被弃用。

Flink 在 Mesos 上还有另一个具有同样语义的配置参数 `mesos.resourcemanager.tasks.mem`,目前也已经被弃用。

如果配置了上述弃用的参数,同时又没有配置与之对应的新配置参数,那它们将按如下规则对应到新的配置参数。
* 独立部署模式(Standalone Deployment)下:Flink 总内存([`taskmanager.memory.flink.size`]({{< ref "docs/deployment/config" >}}#taskmanager-memory-flink-size))
* 容器化部署模式(Containerized Deployement)下(Yarn、Mesos):进程总内存([`taskmanager.memory.process.size`]({{< ref "docs/deployment/config" >}}#taskmanager-memory-process-size))
* 容器化部署模式(Containerized Deployement)下(Yarn):进程总内存([`taskmanager.memory.process.size`]({{< ref "docs/deployment/config" >}}#taskmanager-memory-process-size))

建议您尽早使用新的配置参数取代启用的配置参数,它们在今后的版本中可能会被彻底移除。

Expand Down Expand Up @@ -195,8 +193,7 @@ Flink 现在总是会预留一部分 JVM 堆内存供框架使用([`taskmanage

#### 占比

此前,如果不指定明确的大小,也可以将托管内存配置为占用总内存减去网络内存和容器切除内存(仅在 [Yarn]({{< ref "docs/deployment/resource-providers/yarn" >}}) 和
[Mesos]({{< ref "docs/deployment/resource-providers/mesos" >}}) 上)之后剩余部分的固定比例(`taskmanager.memory.fraction`)。
此前,如果不指定明确的大小,也可以将托管内存配置为占用总内存减去网络内存和容器切除内存(仅在 [Yarn]({{< ref "docs/deployment/resource-providers/yarn" >}}))之后剩余部分的固定比例(`taskmanager.memory.fraction`)。
该配置参数已经被彻底移除,配置它不会产生任何效果。
请使用新的配置参数 [`taskmanager.memory.managed.fraction`]({{< ref "docs/deployment/config" >}}#taskmanager-memory-managed-fraction)。
在未通过 [`taskmanager.memory.managed.size`]({{< ref "docs/deployment/config" >}}#taskmanager-memory-managed-size) 指定明确大小的情况下,新的配置参数将指定[托管内存]({{< ref "docs/deployment/memory/mem_setup_tm" >}}#managed-memory)在 [Flink 总内存]({{< ref "docs/deployment/memory/mem_setup" >}}#configure-total-memory)中的所占比例。
Expand All @@ -206,7 +203,7 @@ Flink 现在总是会预留一部分 JVM 堆内存供框架使用([`taskmanage
#### RocksDB State Backend

流处理作业如果选择使用 [RocksDBStateBackend]({{< ref "docs/ops/state/state_backends" >}}#rocksdbstatebackend),它使用的本地内存现在也被归为[托管内存]({{< ref "docs/deployment/memory/mem_setup_tm" >}}#managed-memory)。
默认情况下,RocksDB 将限制其内存用量不超过[托管内存]({{< ref "docs/deployment/memory/mem_setup_tm" >}}#managed-memory)大小,以避免在 [Yarn]({{< ref "docs/deployment/resource-providers/yarn" >}}) [Mesos]({{< ref "docs/deployment/resource-providers/mesos" >}}) 上容器被杀。你也可以通过设置 [state.backend.rocksdb.memory.managed]({{< ref "docs/deployment/config" >}}#state-backend-rocksdb-memory-managed) 来关闭 RocksDB 的内存控制。
默认情况下,RocksDB 将限制其内存用量不超过[托管内存]({{< ref "docs/deployment/memory/mem_setup_tm" >}}#managed-memory)大小,以避免在 [Yarn]({{< ref "docs/deployment/resource-providers/yarn" >}}) 上容器被杀。你也可以通过设置 [state.backend.rocksdb.memory.managed]({{< ref "docs/deployment/config" >}}#state-backend-rocksdb-memory-managed) 来关闭 RocksDB 的内存控制。
请参考[如何升级容器切除内存](#container-cut-off-memory)

<a name="other-changes" />
Expand All @@ -231,13 +228,11 @@ Flink 现在总是会预留一部分 JVM 堆内存供框架使用([`taskmanage
*JVM 堆空间*的实际大小,是参数指定的大小减去容器切除(Cut-Off)内存后剩余的部分。
容器切除内存在 *1.11* 及以上版本中已被彻底移除。

上述两个参数此前对 [Mesos]({{< ref "docs/deployment/resource-providers/mesos" >}}) 部署模式并不生效。
Flink 在 Mesos 上启动 JobManager 进程时并未设置任何 JVM 内存参数。
*1.11* 版本开始,Flink 将采用与[独立部署模式]({{< ref "docs/deployment/resource-providers/standalone/overview" >}})相同的方式设置这些参数。

这两个配置参数目前已被弃用。
如果配置了上述弃用的参数,同时又没有配置与之对应的新配置参数,那它们将按如下规则对应到新的配置参数。
* 独立部署模式(Standalone Deployment)、Mesos 部署模式下:JVM 堆内存([`jobmanager.memory.heap.size`]({{< ref "docs/deployment/config" >}}#jobmanager-memory-heap-size))
* 独立部署模式(Standalone Deployment):JVM 堆内存([`jobmanager.memory.heap.size`]({{< ref "docs/deployment/config" >}}#jobmanager-memory-heap-size))
* 容器化部署模式(Containerized Deployement)下(Kubernetes、Yarn):进程总内存([`jobmanager.memory.process.size`]({{< ref "docs/deployment/config" >}}#jobmanager-memory-process-size))

建议您尽早使用新的配置参数取代启用的配置参数,它们在今后的版本中可能会被彻底移除。
Expand Down
2 changes: 1 addition & 1 deletion docs/content.zh/docs/deployment/memory/mem_setup.md
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ Flink 会根据默认值或其他配置参数自动调整剩余内存部分的
更多详情请参考[如何为独立部署模式配置内存]({{< ref "docs/deployment/memory/mem_tuning" >}}#configure-memory-for-standalone-deployment)。

通过配置*进程总内存*可以指定由 Flink *JVM 进程*使用的总内存大小。
对于容器化部署模式(Containerized Deployment),这相当于申请的容器(Container)大小,详情请参考[如何配置容器内存]({{< ref "docs/deployment/memory/mem_tuning" >}}#configure-memory-for-containers)([Kubernetes]({{< ref "docs/deployment/resource-providers/standalone/kubernetes" >}})[Yarn]({{< ref "docs/deployment/resource-providers/yarn" >}}) [Mesos]({{< ref "docs/deployment/resource-providers/mesos" >}}))。
对于容器化部署模式(Containerized Deployment),这相当于申请的容器(Container)大小,详情请参考[如何配置容器内存]({{< ref "docs/deployment/memory/mem_tuning" >}}#configure-memory-for-containers)([Kubernetes]({{< ref "docs/deployment/resource-providers/standalone/kubernetes" >}}) [Yarn]({{< ref "docs/deployment/resource-providers/yarn" >}}))。

此外,还可以通过设置 *Flink 总内存*的特定内部组成部分的方式来进行内存配置。
不同进程需要设置的内存组成部分是不一样的。
Expand Down
Loading

0 comments on commit e8b7f3d

Please sign in to comment.