Skip to content

Commit

Permalink
[FLINK-37187][doc] Add doc for submitting Materialized Table refresh …
Browse files Browse the repository at this point in the history
…job to Yarn/K8s (#26137)

----------

Co-authored-by: Ron <ldliulsy@163.com>

This closes #26137
  • Loading branch information
hackergin authored Feb 11, 2025
1 parent 5422703 commit 596648c
Show file tree
Hide file tree
Showing 6 changed files with 401 additions and 30 deletions.
198 changes: 198 additions & 0 deletions docs/content.zh/docs/dev/table/materialized-table/deployment.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,198 @@
---
title: 部署
weight: 3
type: docs
aliases:
- /dev/table/materialized-table/deployment.html
---
<!--
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.
-->

# Introduction

物化表的创建及运维涉及多个组件的协同工作。本文将从架构解析、环境准备、部署流程到操作实践,系统地说明物化表的完整部署方案。

# 架构介绍

- **Client**: 可以是任何能够与 [Flink SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}) 交互的客户端,如 [SQL 客户端]({{< ref "docs/dev/table/sqlClient" >}})、[Flink JDBC 驱动]({{< ref "docs/dev/table/jdbcDriver" >}}) 等。
- **Flink SQL Gateway**: 支持创建、修改和删除物化表。并包含了一个内置的工作流调度器,用于定期刷新全量模式的物化表。
- **Flink Cluster**: 用于运行物化表刷新作业的 Flink 集群。
- **Catalog**: 负责管理物化表元数据的创建、查询、修改和删除。
- **Catalog Store**: 提供 Catalog 属性持久化功能,以便在操作物化表时自动初始化 Catalog 并获取相关的元数据。

{{< img src="/fig/materialized-table-architecture.svg" alt="Illustration of Flink Materialized Table Architecture" width="85%" >}}


# 部署准备

## Flink 集群环境准备

物化表刷新作业目前支持在以下集群环境中运行:
* [Standalone clusters]({{<ref "docs/deployment/resource-providers/standalone/overview">}})
* [YARN clusters]({{<ref "docs/deployment/resource-providers/yarn" >}})
* [Kubernetes clusters]({{<ref "docs/deployment/resource-providers/native_kubernetes" >}})

## 部署 SQL Gateway

物化表必须通过 SQL Gateway 创建,SQL Gateway 需要针对元数据持久化和作业调度进行特定的配置。

### 配置 Catalog Store

`config.yaml` 中增加 `catalog store` 相关配置:
```yaml
table:
catalog-store:
kind: file
file:
path: {path_to_catalog_store} # 替换成实际的路径
```
更多详情配置可参考 [Catalog Store]({{<ref "docs/dev/table/catalogs">}}#catalog-store)。
### 配置工作流调度器插件
`config.yaml` 增加工作流调度器配置,用于定时调度刷新作业。 当前我们仅支持 `embedded` 调度器:

```yaml
workflow-scheduler:
type: embedded
```

### 启动 SQL Gateway

使用以下命令启动 SQL Gateway:
```
./sql-gateway.sh start
```
<span class="label label-danger">注意</span>
Catalog 必须支持创建物化表,目前只有 [Paimon Catalog](https://paimon.apache.org/docs/master/concepts/table-types/#materialized-table) 支持。
# 操作指南
## 连接到 SQL Gateway
使用 SQL Client 的示例:
```shell
./sql-client.sh gateway --endpoint {gateway_endpoint}:{gateway_port}
```

## 创建物化表

### 在 Standalone 集群运行刷新作业

```sql
Flink SQL> SET 'execution.mode' = 'remote';
[INFO] Execute statement succeeded.

FLINK SQL> CREATE MATERIALIZED TABLE my_materialized_table
> ...
[INFO] Execute statement succeeded.
```

### 在 session 模式下运行刷新作业

在 session 模式下执行时,需要提前创建 session 集群,具体可以参考文档 [yarn-session]({{< ref "docs/deployment/resource-providers/yarn" >}}#starting-a-flink-session-on-yarn) 和 [kubernetes-session]({{<ref "docs/deployment/resource-providers/native_kubernetes" >}}#starting-a-flink-session-on-kubernetes)

**Kubernetes session 模式:**

```sql
Flink SQL> SET 'execution.mode' = 'kubernetes-session';
[INFO] Execute statement succeeded.

Flink SQL> SET 'kubernetes.cluster-id' = 'flink-cluster-mt-session-1';
[INFO] Execute statement succeeded.

FLINK SQL> CREATE MATERIALIZED TABLE my_materialized_table
> ...
[INFO] Execute statement succeeded.
```

设置 `execution.mode``kubernetes-session` 并设置参数 `kubernetes.cluster-id` 指向一个已经存在的 Kubernetes session 集群.

**YARN session 模式:**

```sql
Flink SQL> SET 'execution.mode' = 'yarn-session';
[INFO] Execute statement succeeded.

Flink SQL> SET 'yarn.application.id' = 'application-xxxx';
[INFO] Execute statement succeeded.

FLINK SQL> CREATE MATERIALIZED TABLE my_materialized_table
> ...
[INFO] Execute statement succeeded.
```
设置 `execution.mode``yarn-session` 并设置参数 `yarn.application.id` 指向一个已经存在的 YARN session 集群。

### 在 application 模式下运行刷新作业

**Kubernetes application 模式:**

```sql
Flink SQL> SET 'execution.mode' = 'kubernetes-application';
[INFO] Execute statement succeeded.

Flink SQL> SET 'kubernetes.cluster-id' = 'flink-cluster-mt-application-1';
[INFO] Execute statement succeeded.

FLINK SQL> CREATE MATERIALIZED TABLE my_materialized_table
> ...
[INFO] Execute statement succeeded.
```
设置 `execution.mode``kubernetes-application``kubernetes.cluster-id` 是一个可选配置,如果未配置,在提交作业时会自动生成。

**YARN application 模式:**

```sql
Flink SQL> SET 'execution.mode' = 'yarn-application';
[INFO] Execute statement succeeded.

FLINK SQL> CREATE MATERIALIZED TABLE my_materialized_table
> ...
[INFO] Execute statement succeeded.
```
设置 `execution.mode``yarn-application``yarn.application.id` 无需配置。

## 运维操作

集群信息(如 `execution.mode``kubernetes.cluster-id`)已持久化在 Catalog 中,暂停或恢复物化表刷新作业时无需重复设置。

### 暂停刷新作业
```sql
-- 暂停物化表刷新作业
Flink SQL> ALTER MATERIALIZED TABLE my_materialized_table SUSPEND
[INFO] Execute statement succeeded.
```

### 恢复刷新作业
```sql
-- 恢复物化表刷新作业
Flink SQL> ALTER MATERIALIZED TABLE my_materialized_table RESUME
[INFO] Execute statement succeeded.
```

### 修改查询定义
```sql
-- 修改物化表查询定义
Flink SQL> ALTER MATERIALIZED TABLE my_materialized_table
> AS
> ...
[INFO] Execute statement succeeded.
```
4 changes: 0 additions & 4 deletions docs/content.zh/docs/dev/table/materialized-table/overview.md
Original file line number Diff line number Diff line change
Expand Up @@ -28,10 +28,6 @@ under the License.

物化表是 Flink SQL 引入的一种新的表类型,旨在简化批处理和流处理数据管道,提供一致的开发体验。在创建物化表时,通过指定数据新鲜度和查询,Flink 引擎会自动推导出物化表的 Schema ,并创建相应的数据刷新管道,以达到指定的新鲜度。

{{< hint warning >}}
**注意**:该功能目前是一个 MVP(最小可行产品)功能,仅在 [SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}})中可用,并且只支持部署作业到 Flink [Standalone]({{< ref "docs/deployment/resource-providers/standalone/overview" >}})集群。
{{< /hint >}}

# 核心概念

物化表包含以下核心概念:数据新鲜度、刷新模式、查询定义和 `Schema`
Expand Down
12 changes: 1 addition & 11 deletions docs/content.zh/docs/dev/table/materialized-table/quickstart.md
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
---
title: 快速入门
weight: 3
weight: 4
type: docs
aliases:
- /dev/table/materialized-table/quickstart.html
Expand Down Expand Up @@ -29,16 +29,6 @@ under the License.

本入门指南将帮助你快速了解并开始使用物化表。内容包括环境设置,以及创建、修改和删除持续模式和全量模式的物化表。

# 架构介绍

- **Client**: 可以是任何能够与 [Flink SQL Gateway]({{< ref "docs/dev/table/sql-gateway/overview" >}}) 交互的客户端,如 [SQL 客户端]({{< ref "docs/dev/table/sqlClient" >}})、[Flink JDBC 驱动]({{< ref "docs/dev/table/jdbcDriver" >}}) 等。
- **Flink SQL Gateway**: 支持创建、修改和删除物化表。并包含了一个内置的工作流调度器,用于定期刷新全量模式的物化表。
- **Flink Cluster**: 用于运行物化表刷新作业的 Flink 集群。
- **Catalog**: 负责管理物化表元数据的创建、查询、修改和删除。
- **Catalog Store**: 提供 Catalog 属性持久化功能,以便在操作物化表时自动初始化 Catalog 并获取相关的元数据。

{{< img src="/fig/materialized-table-architecture.svg" alt="Illustration of Flink Materialized Table Architecture" width="85%" >}}

# 环境搭建

## 目录准备
Expand Down
Loading

0 comments on commit 596648c

Please sign in to comment.