Skip to content

Commit

Permalink
[FLINK-27111][table][docs] Update docs regarding TableEnvironment con…
Browse files Browse the repository at this point in the history
…figuration (apache#19387)

* [FLINK-27111][table][docs] Update docs regarding TableEnvironment configuration

Following the work on [FLINK-16835] update the docs accordingly to
mention the use of `EnvironmentSettings` and that `TableEnvironement`
related configuration can also be set in `flink-conf.yaml`.

This closes apache#19387
  • Loading branch information
matriv authored Apr 8, 2022
1 parent 388dac8 commit 7b7d96b
Showing 1 changed file with 55 additions and 20 deletions.
75 changes: 55 additions & 20 deletions docs/content/docs/dev/table/config.md
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,12 @@ that the required state size is capped (see [streaming concepts]({{< ref "docs/d

### Overview

In every table environment, the `TableConfig` offers options for configuring the current session.
When instantiating a `TableEnvironment`, `EnviromentSettings` can be used to pass the desired
configuration for the current session, by passing a `Configuration` object to the
`EnviromentSettings`.

Additionally, in every table environment, the `TableConfig` offers options for configuring the
current session.

For common or important configuration options, the `TableConfig` provides getters and setters methods
with detailed inline documentation.
Expand All @@ -51,40 +56,63 @@ table environment.
{{< tab "Java" >}}
```java
// instantiate table environment
TableEnvironment tEnv = ...;

// access flink configuration
TableConfig configuration = tEnv.getConfig();
Configuration configuration = new Configuration();
// set low-level key-value options
configuration.set("table.exec.mini-batch.enabled", "true");
configuration.set("table.exec.mini-batch.allow-latency", "5 s");
configuration.set("table.exec.mini-batch.size", "5000");
configuration.setString("table.exec.mini-batch.enabled", "true");
configuration.setString("table.exec.mini-batch.allow-latency", "5 s");
configuration.setString("table.exec.mini-batch.size", "5000");
EnvironmentSettings settings = EnvironmentSettings.newInstance()
.inStreamingMode().withConfiguration(configuration).build();
TableEnvironment tEnv = TableEnvironment.create(settings);

// access flink configuration after table environment instantiation
TableConfig tableConfig = tEnv.getConfig();
// set low-level key-value options
tableConfig.set("table.exec.mini-batch.enabled", "true");
tableConfig.set("table.exec.mini-batch.allow-latency", "5 s");
tableConfig.set("table.exec.mini-batch.size", "5000");
```
{{< /tab >}}
{{< tab "Scala" >}}
```scala
// instantiate table environment
val tEnv: TableEnvironment = ...

// access flink configuration
val configuration = tEnv.getConfig()
val configuration = new Configuration;
// set low-level key-value options
configuration.set("table.exec.mini-batch.enabled", "true")
configuration.set("table.exec.mini-batch.allow-latency", "5 s")
configuration.set("table.exec.mini-batch.size", "5000")
configuration.setString("table.exec.mini-batch.enabled", "true")
configuration.setString("table.exec.mini-batch.allow-latency", "5 s")
configuration.setString("table.exec.mini-batch.size", "5000")
val settings = EnvironmentSettings.newInstance
.inStreamingMode.withConfiguration(configuration).build
val tEnv: TableEnvironment = TableEnvironment.create(settings)

// access flink configuration after table environment instantiation
val tableConfig = tEnv.getConfig()
// set low-level key-value options
tableConfig.set("table.exec.mini-batch.enabled", "true")
tableConfig.set("table.exec.mini-batch.allow-latency", "5 s")
tableConfig.set("table.exec.mini-batch.size", "5000")
```
{{< /tab >}}
{{< tab "Python" >}}
```python
# instantiate table environment
t_env = ...

# access flink configuration
configuration = t_env.get_config()
# set low-level key-value options
configuration = Configuration()
configuration.set("table.exec.mini-batch.enabled", "true")
configuration.set("table.exec.mini-batch.allow-latency", "5 s")
configuration.set("table.exec.mini-batch.size", "5000")
settings = EnvironmentSettings.new_instance() \
... .in_streaming_mode() \
... .with_configuration(configuration) \
... .build()

t_env = TableEnvironment.create(settings)

# access flink configuration after table environment instantiation
table_config = t_env.get_config()
# set low-level key-value options
table_config.set("table.exec.mini-batch.enabled", "true")
table_config.set("table.exec.mini-batch.allow-latency", "5 s")
table_config.set("table.exec.mini-batch.size", "5000")
```
{{< /tab >}}
{{< tab "SQL CLI" >}}
Expand All @@ -96,6 +124,13 @@ Flink SQL> SET 'table.exec.mini-batch.size' = '5000';
{{< /tab >}}
{{< /tabs >}}

{{< hint info >}}
**Note:** All of the following configuration options can also be set globally in
`conf/flink-conf.yaml` (see [configuration]({{< ref "docs/deployment/config" >}}) and can be later
on overridden in the application, through `EnvironmentSettings`, before instantiating
the `TableEnvironment`, or through the `TableConfig` of the `TableEnvironment`.
{{< /hint >}}

### Execution Options

The following options can be used to tune the performance of the query execution.
Expand Down

0 comments on commit 7b7d96b

Please sign in to comment.