The task manager’s port used for data exchange operations.
+
The task manager’s external port used for data exchange operations.
taskmanager.data.ssl.enabled
@@ -54,7 +54,7 @@
taskmanager.host
(none)
String
-
The address of the network interface that the TaskManager binds to. This option can be used to define explicitly a binding address. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file.
+
The external address of the network interface where the TaskManager is exposed. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file.
taskmanager.jvm-exit-on-oom
@@ -81,24 +81,6 @@
Integer
The number of parallel operator or user function instances that a single TaskManager can run. If this value is larger than 1, a single TaskManager takes multiple instances of a function or operator. That way, the TaskManager can utilize multiple CPU cores, but at the same time, the available memory is divided between the different operator or function instances. This value is typically proportional to the number of physical CPU cores that the TaskManager's machine has (e.g., equal to the number of cores, or half the number of cores).
-
-
taskmanager.registration.initial-backoff
-
500 ms
-
Duration
-
The initial registration backoff between two consecutive registration attempts. The backoff is doubled for each new registration attempt until it reaches the maximum registration backoff.
-
-
-
taskmanager.registration.max-backoff
-
30 s
-
Duration
-
The maximum registration backoff between two consecutive registration attempts. The max registration backoff requires a time unit specifier (ms/s/min/h/d).
-
-
-
taskmanager.registration.refused-backoff
-
10 s
-
Duration
-
The backoff after a registration has been refused by the job manager before retrying to connect.
-
taskmanager.registration.timeout
5 min
@@ -109,7 +91,7 @@
taskmanager.rpc.port
"0"
String
-
The task manager’s IPC port. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine.
+
The external RPC port where the TaskManager is exposed. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine.
The task manager’s port used for data exchange operations.
+
The task manager’s external port used for data exchange operations.
taskmanager.host
(none)
String
-
The address of the network interface that the TaskManager binds to. This option can be used to define explicitly a binding address. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file.
+
The external address of the network interface where the TaskManager is exposed. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file.
taskmanager.rpc.port
"0"
String
-
The task manager’s IPC port. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine.
+
The external RPC port where the TaskManager is exposed. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine.
Total Flink Memory size for the JobManager. This includes all the memory that a JobManager consumes, except for JVM Metaspace and JVM Overhead. It consists of JVM Heap Memory and Off-heap Memory. See also 'jobmanager.memory.process.size' for total process memory size configuration.
+
+
+
jobmanager.memory.heap.size
+
(none)
+
MemorySize
+
JVM Heap Memory size for JobManager. The minimum recommended JVM Heap size is 128.000mb (134217728 bytes).
+
+
+
jobmanager.memory.jvm-metaspace.size
+
256 mb
+
MemorySize
+
JVM Metaspace Size for the JobManager.
+
+
+
jobmanager.memory.jvm-overhead.fraction
+
0.1
+
Float
+
Fraction of Total Process Memory to be reserved for JVM Overhead. This is off-heap memory reserved for JVM overhead, such as thread stack space, compile cache, etc. This includes native memory but not direct memory, and will not be counted when Flink calculates JVM max direct memory size parameter. The size of JVM Overhead is derived to make up the configured fraction of the Total Process Memory. If the derived size is less or greater than the configured min or max size, the min or max size will be used. The exact size of JVM Overhead can be explicitly specified by setting the min and max size to the same value.
+
+
+
jobmanager.memory.jvm-overhead.max
+
1 gb
+
MemorySize
+
Max JVM Overhead size for the JobManager. This is off-heap memory reserved for JVM overhead, such as thread stack space, compile cache, etc. This includes native memory but not direct memory, and will not be counted when Flink calculates JVM max direct memory size parameter. The size of JVM Overhead is derived to make up the configured fraction of the Total Process Memory. If the derived size is less or greater than the configured min or max size, the min or max size will be used. The exact size of JVM Overhead can be explicitly specified by setting the min and max size to the same value.
+
+
+
jobmanager.memory.jvm-overhead.min
+
192 mb
+
MemorySize
+
Min JVM Overhead size for the JobManager. This is off-heap memory reserved for JVM overhead, such as thread stack space, compile cache, etc. This includes native memory but not direct memory, and will not be counted when Flink calculates JVM max direct memory size parameter. The size of JVM Overhead is derived to make up the configured fraction of the Total Process Memory. If the derived size is less or greater than the configured min or max size, the min or max size will be used. The exact size of JVM Overhead can be explicitly specified by setting the min and max size to the same value.
+
+
+
jobmanager.memory.off-heap.size
+
128 mb
+
MemorySize
+
Off-heap Memory size for JobManager. The JVM direct memory limit of the Job Manager process (-XX:MaxDirectMemorySize) will be set to this value. This option covers all off-heap memory usage including direct and native memory allocation.
+
+
+
jobmanager.memory.process.size
+
(none)
+
MemorySize
+
Total Process Memory size for the JobManager. This includes all the memory that a JobManager JVM process consumes, consisting of Total Flink Memory, JVM Metaspace, and JVM Overhead. In containerized setups, this should be set to the container memory. See also 'jobmanager.memory.flink.size' for Total Flink Memory size configuration.
Defines the class resolution strategy when loading classes from user code, meaning whether to first check the user code jar ("child-first") or the application classpath ("parent-first"). The default settings indicate to load classes first from the user code jar, which means that user code jars can include and load different dependencies than Flink uses (transitively).
+
+
fs.allowed-fallback-filesystems
+
(none)
+
String
+
A (semicolon-separated) list of file schemes, for which Hadoop can be used instead of an appropriate Flink plugin. (example: s3;wasb)
The tolerable checkpoint failure number. If set to 0, that meanswe do not tolerance any checkpoint failure.
+
+
execution.checkpointing.unaligned
+
false
+
Boolean
+
Enables unaligned checkpoints, which greatly reduce checkpointing times under backpressure.
Unaligned checkpoints contain data stored in buffers as part of the checkpoint state, which allows checkpoint barriers to overtake these buffers. Thus, the checkpoint duration becomes independent of the current throughput as checkpoint barriers are effectively not embedded into the stream of data anymore.
Unaligned checkpoints can only be enabled if `execution.checkpointing.mode` is `EXACTLY_ONCE`
+
diff --git a/docs/_includes/generated/execution_config_configuration.html b/docs/_includes/generated/execution_config_configuration.html
index 80043227ac10e999d3f9168c0872f62aeab85096..25b0ac69d80e037f53f54a369996c843871fc91c 100644
--- a/docs/_includes/generated/execution_config_configuration.html
+++ b/docs/_includes/generated/execution_config_configuration.html
@@ -54,11 +54,9 @@ By default no operator is disabled.
table.exec.shuffle-mode
Batch
-
"batch"
+
"ALL_EDGES_BLOCKING"
String
-
Sets exec shuffle mode. Only batch or pipeline can be set.
-batch: the job will run stage by stage.
-pipeline: the job will run in streaming mode, but it may cause resource deadlock that receiver waits for resource to start when the sender holds resource to wait to send data to the receiver.
+
Sets exec shuffle mode. Accepted values are:
`ALL_EDGES_BLOCKING`: All edges will use blocking shuffle.
`FORWARD_EDGES_PIPELINED`: Forward edges will use pipelined shuffle, others blocking.
`POINTWISE_EDGES_PIPELINED`: Pointwise edges will use pipelined shuffle, others blocking. Pointwise edges include forward and rescale edges.
`ALL_EDGES_PIPELINED`: All edges will use pipelined shuffle.
`batch`: the same as `ALL_EDGES_BLOCKING`. Deprecated.
`pipelined`: the same as `ALL_EDGES_PIPELINED`. Deprecated.
Note: Blocking shuffle means data will be fully produced before sent to consumer tasks. Pipelined shuffle means data will be sent to consumer tasks once produced.
Dictionary for JobManager to store the archives of completed jobs.
+
+
jobmanager.bind-host
+
(none)
+
String
+
The local address of the network interface that the job manager binds to. If not configured, '0.0.0.0' will be used.
+
jobmanager.execution.attempts-history-size
16
@@ -27,10 +33,52 @@
This option specifies how the job computation recovers from task failures. Accepted values are:
'full': Restarts all tasks to recover the job.
'region': Restarts all tasks that could be affected by the task failure. More details can be found here.
-
jobmanager.heap.size
-
"1024m"
-
String
-
JVM heap size for the JobManager.
+
jobmanager.memory.flink.size
+
(none)
+
MemorySize
+
Total Flink Memory size for the JobManager. This includes all the memory that a JobManager consumes, except for JVM Metaspace and JVM Overhead. It consists of JVM Heap Memory and Off-heap Memory. See also 'jobmanager.memory.process.size' for total process memory size configuration.
+
+
+
jobmanager.memory.heap.size
+
(none)
+
MemorySize
+
JVM Heap Memory size for JobManager. The minimum recommended JVM Heap size is 128.000mb (134217728 bytes).
+
+
+
jobmanager.memory.jvm-metaspace.size
+
256 mb
+
MemorySize
+
JVM Metaspace Size for the JobManager.
+
+
+
jobmanager.memory.jvm-overhead.fraction
+
0.1
+
Float
+
Fraction of Total Process Memory to be reserved for JVM Overhead. This is off-heap memory reserved for JVM overhead, such as thread stack space, compile cache, etc. This includes native memory but not direct memory, and will not be counted when Flink calculates JVM max direct memory size parameter. The size of JVM Overhead is derived to make up the configured fraction of the Total Process Memory. If the derived size is less or greater than the configured min or max size, the min or max size will be used. The exact size of JVM Overhead can be explicitly specified by setting the min and max size to the same value.
+
+
+
jobmanager.memory.jvm-overhead.max
+
1 gb
+
MemorySize
+
Max JVM Overhead size for the JobManager. This is off-heap memory reserved for JVM overhead, such as thread stack space, compile cache, etc. This includes native memory but not direct memory, and will not be counted when Flink calculates JVM max direct memory size parameter. The size of JVM Overhead is derived to make up the configured fraction of the Total Process Memory. If the derived size is less or greater than the configured min or max size, the min or max size will be used. The exact size of JVM Overhead can be explicitly specified by setting the min and max size to the same value.
+
+
+
jobmanager.memory.jvm-overhead.min
+
192 mb
+
MemorySize
+
Min JVM Overhead size for the JobManager. This is off-heap memory reserved for JVM overhead, such as thread stack space, compile cache, etc. This includes native memory but not direct memory, and will not be counted when Flink calculates JVM max direct memory size parameter. The size of JVM Overhead is derived to make up the configured fraction of the Total Process Memory. If the derived size is less or greater than the configured min or max size, the min or max size will be used. The exact size of JVM Overhead can be explicitly specified by setting the min and max size to the same value.
+
+
+
jobmanager.memory.off-heap.size
+
128 mb
+
MemorySize
+
Off-heap Memory size for JobManager. The JVM direct memory limit of the Job Manager process (-XX:MaxDirectMemorySize) will be set to this value. This option covers all off-heap memory usage including direct and native memory allocation.
+
+
+
jobmanager.memory.process.size
+
(none)
+
MemorySize
+
Total Process Memory size for the JobManager. This includes all the memory that a JobManager JVM process consumes, consisting of Total Flink Memory, JVM Metaspace, and JVM Overhead. In containerized setups, this should be set to the container memory. See also 'jobmanager.memory.flink.size' for Total Flink Memory size configuration.
jobmanager.rpc.address
@@ -38,6 +86,12 @@
String
The config parameter defining the network address to connect to for communication with the job manager. This value is only interpreted in setups where a single JobManager with static name or address exists (simple standalone setups, or container setups with dynamic service name resolution). It is not used in many high-availability setups, when a leader-election service (like ZooKeeper) is used to elect and discover the JobManager leader from potentially multiple standby JobManagers.
+
+
jobmanager.rpc.bind-port
+
(none)
+
Integer
+
The local RPC port that the JobManager binds to. If not configured, the external port (configured by 'jobmanager.rpc.port') will be used.
The cluster id used for identifying the unique flink cluster. If it's not set, the client will generate a random UUID name.
+
The cluster-id, which should be no more than 45 characters, is used for identifying a unique Flink cluster. If not set, the client will automatically generate it with a random ID.
kubernetes.config.file
@@ -34,9 +34,9 @@
kubernetes.container.image.pull-policy
-
"IfNotPresent"
-
String
-
Kubernetes image pull policy. Valid values are Always, Never, and IfNotPresent. The default policy is IfNotPresent to avoid putting pressure to image repository.
+
IfNotPresent
+
Enum
Possible values: [IfNotPresent, Always, Never]
+
The Kubernetes container image pull policy (IfNotPresent or Always or Never). The default policy is IfNotPresent to avoid putting pressure to image repository.
kubernetes.container.image.pull-secrets
@@ -68,18 +68,48 @@
String
The directory that logs of jobmanager and taskmanager be saved in the pod.
+
+
kubernetes.hadoop.conf.config-map.name
+
(none)
+
String
+
Specify the name of an existing ConfigMap that contains custom Hadoop configuration to be mounted on the JobManager(s) and TaskManagers.
+
+
+
kubernetes.jobmanager.annotations
+
(none)
+
Map
+
The user-specified annotations that are set to the JobManager pod. The value could be in the form of a1:v1,a2:v2
+
kubernetes.jobmanager.cpu
1.0
Double
The number of cpu used by job manager
+
+
kubernetes.jobmanager.labels
+
(none)
+
Map
+
The labels to be set for JobManager pod. Specified as key:value pairs separated by commas. For example, version:alphav1,deploy:test.
+
+
+
kubernetes.jobmanager.node-selector
+
(none)
+
Map
+
The node selector to be set for JobManager pod. Specified as key:value pairs separated by commas. For example, environment:production,disk:ssd.
+
kubernetes.jobmanager.service-account
"default"
String
Service account that is used by jobmanager within kubernetes cluster. The job manager uses this service account when requesting taskmanager pods from the API server.
+
+
kubernetes.jobmanager.tolerations
+
(none)
+
List<Map>
+
The user-specified tolerations to be set to the JobManager pod. The value should be in the form of key:key1,operator:Equal,value:value1,effect:NoSchedule;key:key2,operator:Exists,effect:NoExecute,tolerationSeconds:6000
+
kubernetes.namespace
"default"
@@ -88,15 +118,15 @@
kubernetes.rest-service.exposed.type
-
"LoadBalancer"
-
String
-
It could be ClusterIP/NodePort/LoadBalancer(default). When set to ClusterIP, the rest servicewill not be created.
+
LoadBalancer
+
Enum
Possible values: [ClusterIP, NodePort, LoadBalancer]
+
The type of the rest service (ClusterIP or NodePort or LoadBalancer). When set to ClusterIP, the rest service will not be created.
-
kubernetes.service.create-timeout
-
"1 min"
-
String
-
Timeout used for creating the service. The timeout value requires a time-unit specifier (ms/s/min/h/d).
+
kubernetes.taskmanager.annotations
+
(none)
+
Map
+
The user-specified annotations that are set to the TaskManager pod. The value could be in the form of a1:v1,a2:v2
kubernetes.taskmanager.cpu
@@ -104,5 +134,23 @@
Double
The number of cpu used by task manager. By default, the cpu is set to the number of slots per TaskManager
+
+
kubernetes.taskmanager.labels
+
(none)
+
Map
+
The labels to be set for TaskManager pods. Specified as key:value pairs separated by commas. For example, version:alphav1,deploy:test.
+
+
+
kubernetes.taskmanager.node-selector
+
(none)
+
Map
+
The node selector to be set for TaskManager pods. Specified as key:value pairs separated by commas. For example, environment:production,disk:ssd.
+
+
+
kubernetes.taskmanager.tolerations
+
(none)
+
List<Map>
+
The user-specified tolerations to be set to the TaskManager pod. The value should be in the form of key:key1,operator:Equal,value:value1,effect:NoSchedule;key:key2,operator:Exists,effect:NoExecute,tolerationSeconds:6000
Add python archive files for job. The archive files will be extracted to the working directory of python UDF worker. Currently only zip-format is supported. For each archive file, a target directory is specified. If the target directory name is specified, the archive file will be extracted to a name can directory with the specified name. Otherwise, the archive file will be extracted to a directory with the same name of the archive file. The files uploaded via this option are accessible via relative path. '#' could be used as the separator of the archive file path and the target directory name. Comma (',') could be used as the separator to specify multiple archive files. This option can be used to upload the virtual environment, the data files used in Python UDF. The data files could be accessed in Python UDF, e.g.: f = open('data/data.txt', 'r'). The option is equivalent to the command line option "-pyarch".
+
+
+
python.client.executable
+
"python"
+
String
+
The python interpreter used to launch the python process when compiling the jobs containing Python UDFs. Equivalent to the environment variable PYFLINK_EXECUTABLE. The priority is as following: 1. the configuration 'python.client.executable' defined in the source code; 2. the environment variable PYFLINK_EXECUTABLE; 3. the configuration 'python.client.executable' defined in flink-conf.yaml
+
+
+
python.executable
+
"python"
+
String
+
Specify the path of the python interpreter used to execute the python UDF worker. The python UDF worker depends on Python 3.5+, Apache Beam (version == 2.19.0), Pip (version >= 7.1.0) and SetupTools (version >= 37.0.0). Please ensure that the specified environment meets the above requirements. The option is equivalent to the command line option "-pyexec".
+
+
+
python.files
+
(none)
+
String
+
Attach custom python files for job. These files will be added to the PYTHONPATH of both the local client and the remote python UDF worker. The standard python resource file suffixes such as .py/.egg/.zip or directory are all supported. Comma (',') could be used as the separator to specify multiple files. The option is equivalent to the command line option "-pyfs".
+
python.fn-execution.arrow.batch.size
-
1000
+
10000
Integer
The maximum number of elements to include in an arrow batch for Python user-defined function execution. The arrow batch size should not exceed the bundle size. Otherwise, the bundle size will be used as the arrow batch size.
@@ -22,7 +46,7 @@
python.fn-execution.bundle.size
-
1000
+
100000
Integer
The maximum number of elements to include in a bundle for Python user-defined function execution. The elements are processed asynchronously. One bundle of elements are processed before processing the next bundle of elements. A larger value can improve the throughput, but at the cost of more memory usage and higher latency.
@@ -38,5 +62,17 @@
String
The amount of memory to be allocated by the Python framework. The sum of the value of this configuration and "python.fn-execution.buffer.memory.size" represents the total memory of a Python worker. The memory will be accounted as managed memory if the actual memory allocated to an operator is no less than the total memory of a Python worker. Otherwise, this configuration takes no effect.
+
+
python.metric.enabled
+
true
+
Boolean
+
When it is false, metric for Python will be disabled. You can disable the metric to achieve better performance at some circumstance.
+
+
+
python.requirements
+
(none)
+
String
+
Specify a requirements.txt file which defines the third-party dependencies. These dependencies will be installed and added to the PYTHONPATH of the python UDF worker. A directory which contains the installation packages of these dependencies could be specified optionally. Use '#' as the separator if the optional parameter exists. The option is equivalent to the command line option "-pyreq".
Returns the status for the delete operation of a cluster data set.
+
+
+
Path parameters
+
+
+
+
+
triggerid - 32-character hexadecimal string that identifies an asynchronous operation trigger ID. The ID was returned then the operation was triggered.
This determines the factory for timer service state implementation. Options are either HEAP (heap-based, default) or ROCKSDB for an implementation based on RocksDB .
This determines the factory for timer service state implementation. Options are either HEAP (heap-based, default) or ROCKSDB for an implementation based on RocksDB .
diff --git a/docs/_includes/generated/table_config_configuration.html b/docs/_includes/generated/table_config_configuration.html
new file mode 100644
index 0000000000000000000000000000000000000000..c1bdb677964b65724b69437bcb734883b634736b
--- /dev/null
+++ b/docs/_includes/generated/table_config_configuration.html
@@ -0,0 +1,24 @@
+
+
+
+
Key
+
Default
+
Type
+
Description
+
+
+
+
+
table.dynamic-table-options.enabled
BatchStreaming
+
false
+
Boolean
+
Enable or disable the OPTIONS hint used to specify table optionsdynamically, if disabled, an exception would be thrown if any OPTIONS hint is specified
+
+
+
table.sql-dialect
BatchStreaming
+
"default"
+
String
+
The SQL dialect defines how to parse a SQL query. A different SQL dialect may support different SQL grammar. Currently supported dialects are: default and hive
Time we wait for the timers in milliseconds to finish all pending timer threads when the stream task is cancelled.
+
+
taskmanager.bind-host
+
(none)
+
String
+
The local address of the network interface that the task manager binds to. If not configured, '0.0.0.0' will be used.
+
taskmanager.debug.memory.log
false
@@ -42,7 +48,7 @@
taskmanager.host
(none)
String
-
The address of the network interface that the TaskManager binds to. This option can be used to define explicitly a binding address. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file.
+
The external address of the network interface where the TaskManager is exposed. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file.
taskmanager.jvm-exit-on-oom
@@ -63,35 +69,23 @@
Integer
The number of parallel operator or user function instances that a single TaskManager can run. If this value is larger than 1, a single TaskManager takes multiple instances of a function or operator. That way, the TaskManager can utilize multiple CPU cores, but at the same time, the available memory is divided between the different operator or function instances. This value is typically proportional to the number of physical CPU cores that the TaskManager's machine has (e.g., equal to the number of cores, or half the number of cores).
-
-
taskmanager.registration.initial-backoff
-
500 ms
-
Duration
-
The initial registration backoff between two consecutive registration attempts. The backoff is doubled for each new registration attempt until it reaches the maximum registration backoff.
-
-
-
taskmanager.registration.max-backoff
-
30 s
-
Duration
-
The maximum registration backoff between two consecutive registration attempts. The max registration backoff requires a time unit specifier (ms/s/min/h/d).
-
-
-
taskmanager.registration.refused-backoff
-
10 s
-
Duration
-
The backoff after a registration has been refused by the job manager before retrying to connect.
-
taskmanager.registration.timeout
5 min
Duration
Defines the timeout for the TaskManager registration. If the duration is exceeded without a successful registration, then the TaskManager terminates.
+
+
taskmanager.rpc.bind-port
+
(none)
+
Integer
+
The local RPC port that the TaskManager binds to. If not configured, the external port (configured by 'taskmanager.rpc.port') will be used.
+
taskmanager.rpc.port
"0"
String
-
The task manager’s IPC port. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine.
+
The external RPC port where the TaskManager is exposed. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine.
When a Flink job is submitted to YARN, the JobManager’s host and the number of available processing slots is written into a properties file, so that the Flink client is able to pick those details up. This configuration parameter allows changing the default location of that file (for example for environments sharing a Flink installation between users).
+
+
yarn.security.kerberos.localized-keytab-path
+
"krb5.keytab"
+
String
+
Local (on NodeManager) path where kerberos keytab file will be localized to. If yarn.security.kerberos.ship-local-keytab set to true, Flink willl ship the keytab file as a YARN local resource. In this case, the path is relative to the local resource directory. If set to false, Flink will try to directly locate the keytab from the path itself.
+
+
+
yarn.security.kerberos.ship-local-keytab
+
true
+
Boolean
+
When this is true Flink will ship the keytab file configured via security.kerberos.login.keytab as a localized YARN resource.
+
yarn.ship-directories
(none)
diff --git a/docs/build_docs.bat b/docs/build_docs.bat
deleted file mode 100644
index 024b513d9031ca1a667a2be53d5a2dff9c5f80ea..0000000000000000000000000000000000000000
--- a/docs/build_docs.bat
+++ /dev/null
@@ -1,57 +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.
-::###############################################################################
-
-@echo off
-:start
-call jekyll -version >nul 2>&1
-if "%errorlevel%"=="0" goto check_redcarpet
-echo ERROR: Could not find jekyll.
-echo Please install with 'gem install jekyll' (see http://jekyllrb.com).
-exit /b 1
-
-:check_redcarpet
-call redcarpet -version >nul 2>&1
-if "%errorlevel%"=="0" goto check_pygments
-echo WARN: Could not find redcarpet.
-echo Please install with 'gem install redcarpet' (see https://github.com/vmg/redcarpet).
-echo Redcarpet is needed for Markdown parsing and table of contents generation.
-goto check_pygments
-
-:check_pygments
-call python -c "import pygments" >nul 2>&1
-if "%errorlevel%"=="0" goto execute
-echo WARN: Could not find pygments.
-echo Please install with 'sudo easy_install Pygments' (requires Python; see http://pygments.org).
-echo Pygments is needed for syntax highlighting of the code examples.
-goto execute
-
-:execute
-SET "DOCS_SRC=%cd%"
-SET "DOCS_DST=%DOCS_SRC%\target"
-
-::default jekyll command is to just build site
-::if flag p is set, start the webserver too.
-IF "%1"=="" GOTO :build
-IF "%1"=="-p" GOTO :serve
-GOTO :build
-
-:build
-jekyll build --source %DOCS_SRC% --destination %DOCS_DST%
-
-:serve
-jekyll serve --baseurl "" --watch --source %DOCS_SRC% --destination %DOCS_DST%
diff --git a/docs/concepts/flink-architecture.md b/docs/concepts/flink-architecture.md
index 3bc14dd098d04fa62ee2056d2122ed3dc3fe3ecd..8414943fd167d5b67e51c566bca4231a4ce139b6 100644
--- a/docs/concepts/flink-architecture.md
+++ b/docs/concepts/flink-architecture.md
@@ -44,7 +44,7 @@ The Flink runtime consists of two types of processes:
tasks, coordinates checkpoints, coordinates recovery on failures, etc.
There is always at least one *Flink Master*. A high-availability setup
- might have multiple *Flink Masters*, one of which one is always the
+ might have multiple *Flink Masters*, one of which is always the
*leader*, and the others are *standby*.
- The *TaskManagers* (also called *workers*) execute the *tasks* (or more
@@ -54,10 +54,10 @@ The Flink runtime consists of two types of processes:
There must always be at least one TaskManager.
The Flink Master and TaskManagers can be started in various ways: directly on
-the machines as a [standalone cluster]({{ site.baseurl }}{% link
+the machines as a [standalone cluster]({% link
ops/deployment/cluster_setup.md %}), in containers, or managed by resource
-frameworks like [YARN]({{ site.baseurl }}{% link ops/deployment/yarn_setup.md
-%}) or [Mesos]({{ site.baseurl }}{% link ops/deployment/mesos.md %}).
+frameworks like [YARN]({% link ops/deployment/yarn_setup.md
+%}) or [Mesos]({% link ops/deployment/mesos.md %}).
TaskManagers connect to Flink Masters, announcing themselves as available, and
are assigned work.
@@ -77,9 +77,9 @@ For distributed execution, Flink *chains* operator subtasks together into
*tasks*. Each task is executed by one thread. Chaining operators together into
tasks is a useful optimization: it reduces the overhead of thread-to-thread
handover and buffering, and increases overall throughput while decreasing
-latency. The chaining behavior can be configured; see the [chaining docs]({{
-site.baseurl }}{% link dev/stream/operators/index.md
-%}#task-chaining-and-resource-groups) for details.
+latency. The chaining behavior can be configured; see the [chaining docs]({%
+link dev/stream/operators/index.md %}#task-chaining-and-resource-groups) for
+details.
The sample dataflow in the figure below is executed with five subtasks, and
hence with five parallel threads.
@@ -102,7 +102,7 @@ certain amount of reserved managed memory. Note that no CPU isolation happens
here; currently slots only separate the managed memory of tasks.
By adjusting the number of task slots, users can define how subtasks are
-isolated from each other. Having one slot per TaskManager means each task
+isolated from each other. Having one slot per TaskManager means that each task
group runs in a separate JVM (which can be started in a separate container, for
example). Having multiple slots means more subtasks share the same JVM. Tasks
in the same JVM share TCP connections (via multiplexing) and heartbeat
diff --git a/docs/concepts/glossary.md b/docs/concepts/glossary.md
index 0645786f8ba0ce2aecbba39a158356ed95ba0797..697b1b044f6cf21560737c7fe182e2e1e9bd615f 100644
--- a/docs/concepts/glossary.md
+++ b/docs/concepts/glossary.md
@@ -25,11 +25,16 @@ under the License.
#### Flink Application Cluster
-A Flink Application Cluster is a dedicated [Flink Cluster](#flink-cluster) that only
+A Flink Application Cluster is a dedicated [Flink Cluster](#flink-cluster) that
+only executes [Flink Jobs](#flink-job) from one [Flink
+Application](#flink-application). The lifetime of the [Flink
+Cluster](#flink-cluster) is bound to the lifetime of the Flink Application.
+
+#### Flink Job Cluster
+
+A Flink Job Cluster is a dedicated [Flink Cluster](#flink-cluster) that only
executes a single [Flink Job](#flink-job). The lifetime of the
-[Flink Cluster](#flink-cluster) is bound to the lifetime of the Flink Job. Formerly
-Flink Application Clusters were also known as Flink Clusters in *job mode*. Compare to
-[Flink Session Cluster](#flink-session-cluster).
+[Flink Cluster](#flink-cluster) is bound to the lifetime of the Flink Job.
#### Flink Cluster
@@ -60,11 +65,22 @@ Java, this corresponds to the definition of *Instance* or *Object* in Java. In t
Flink, the term *parallel instance* is also frequently used to emphasize that multiple instances of
the same [Operator](#operator) or [Function](#function) type are running in parallel.
+#### Flink Application
+
+A Flink application is a Java Application that submits one or multiple [Flink
+Jobs](#flink-job) from the `main()` method (or by some other means). Submitting
+jobs is usually done by calling `execute()` on an execution environment.
+
+The jobs of an application can either be submitted to a long running [Flink
+Session Cluster](#flink-session-cluster), to a dedicated [Flink Application
+Cluster](#flink-application-cluster), or to a [Flink Job
+Cluster](#flink-job-cluster).
+
#### Flink Job
-A Flink Job is the runtime representation of a Flink program. A Flink Job can either be submitted
-to a long running [Flink Session Cluster](#flink-session-cluster) or it can be started as a
-self-contained [Flink Application Cluster](#flink-application-cluster).
+A Flink Job is the runtime representation of a [logical graph](#logical-graph)
+(also often called dataflow graph) that is created and submitted by calling
+`execute()` in a [Flink Application](#flink-application).
#### JobGraph
@@ -78,9 +94,12 @@ whole [Flink Master](#flink-master) was called JobManager.
#### Logical Graph
-A logical graph is a directed graph describing the high-level logic of a stream processing program.
-The nodes are [Operators](#operator) and the edges indicate input/output-relationships of the
-operators and correspond to data streams or data sets.
+A logical graph is a directed graph where the nodes are [Operators](#operator)
+and the edges define input/output-relationships of the operators and correspond
+to data streams or data sets. A logical graph is created by submitting jobs
+from a [Flink Application](#flink-application).
+
+Logical graphs are also often referred to as *dataflow graphs*.
#### Managed State
diff --git a/docs/concepts/index.md b/docs/concepts/index.md
index b236a5f4c8f9ae016c1e4c16628bababf4fc01d6..ab7b44ed46aca446732c69b814472376ebfd1618 100644
--- a/docs/concepts/index.md
+++ b/docs/concepts/index.md
@@ -1,8 +1,8 @@
---
-title: Concepts
+title: Concepts in Depth
nav-id: concepts
-nav-pos: 2
-nav-title: ' Concepts'
+nav-pos: 3
+nav-title: ' Concepts in Depth'
nav-parent_id: root
nav-show_overview: true
permalink: /concepts/index.html
@@ -27,22 +27,34 @@ specific language governing permissions and limitations
under the License.
-->
+The [Hands-on Tutorials]({% link tutorials/index.md %}) explain the basic concepts
+of stateful and timely stream processing that underlie Flink's APIs, and provide examples of how
+these mechanisms are used in applications. Stateful stream processing is introduced in the context
+of [Data Pipelines & ETL]({% link tutorials/etl.md %}#stateful-transformations)
+and is further developed in the section on [Fault Tolerance]({% link
+tutorials/fault_tolerance.md %}). Timely stream processing is introduced in the section on
+[Streaming Analytics]({% link tutorials/streaming_analytics.md %}).
+
+This _Concepts in Depth_ section provides a deeper understanding of how Flink's architecture and runtime
+implement these concepts.
+
+## Flink's APIs
+
Flink offers different levels of abstraction for developing streaming/batch applications.
- - The lowest level abstraction simply offers **stateful streaming**. It is
- embedded into the [DataStream API]({{ site.baseurl}}{% link
- dev/datastream_api.md %}) via the [Process Function]({{ site.baseurl }}{%
- link dev/stream/operators/process_function.md %}). It allows users freely
- process events from one or more streams, and use consistent fault tolerant
- *state*. In addition, users can register event time and processing time
- callbacks, allowing programs to realize sophisticated computations.
+ - The lowest level abstraction simply offers **stateful and timely stream processing**. It is
+ embedded into the [DataStream API]({% link dev/datastream_api.md %}) via the [Process
+ Function]({% link dev/stream/operators/process_function.md %}). It allows
+ users to freely process events from one or more streams, and provides consistent, fault tolerant
+ *state*. In addition, users can register event time and processing time callbacks, allowing
+ programs to realize sophisticated computations.
- - In practice, most applications would not need the above described low level
- abstraction, but would instead program against the **Core APIs** like the
- [DataStream API]({{ site.baseurl }}{% link dev/datastream_api.md %})
- (bounded/unbounded streams) and the [DataSet API]({{ site.baseurl }}{% link
+ - In practice, many applications do not need the low-level
+ abstractions described above, and can instead program against the **Core APIs**: the
+ [DataStream API]({% link dev/datastream_api.md %})
+ (bounded/unbounded streams) and the [DataSet API]({% link
dev/batch/index.md %}) (bounded data sets). These fluent APIs offer the
common building blocks for data processing, like various forms of
user-specified transformations, joins, aggregations, windows, state, etc.
@@ -50,25 +62,25 @@ Flink offers different levels of abstraction for developing streaming/batch appl
respective programming languages.
The low level *Process Function* integrates with the *DataStream API*,
- making it possible to go the lower level abstraction for certain operations
- only. The *DataSet API* offers additional primitives on bounded data sets,
+ making it possible to use the lower-level abstraction on an as-needed basis.
+ The *DataSet API* offers additional primitives on bounded data sets,
like loops/iterations.
- The **Table API** is a declarative DSL centered around *tables*, which may
be dynamically changing tables (when representing streams). The [Table
- API]({{ site.baseurl }}{% link dev/table/index.md %}) follows the
+ API]({% link dev/table/index.md %}) follows the
(extended) relational model: Tables have a schema attached (similar to
tables in relational databases) and the API offers comparable operations,
such as select, project, join, group-by, aggregate, etc. Table API
programs declaratively define *what logical operation should be done*
rather than specifying exactly *how the code for the operation looks*.
Though the Table API is extensible by various types of user-defined
- functions, it is less expressive than the *Core APIs*, but more concise to
+ functions, it is less expressive than the *Core APIs*, and more concise to
use (less code to write). In addition, Table API programs also go through
an optimizer that applies optimization rules before execution.
One can seamlessly convert between tables and *DataStream*/*DataSet*,
- allowing programs to mix *Table API* and with the *DataStream* and
+ allowing programs to mix the *Table API* with the *DataStream* and
*DataSet* APIs.
- The highest level abstraction offered by Flink is **SQL**. This abstraction
@@ -77,7 +89,3 @@ Flink offers different levels of abstraction for developing streaming/batch appl
}}{% link dev/table/index.md %}#sql) abstraction closely interacts with the
Table API, and SQL queries can be executed over tables defined in the
*Table API*.
-
-This _concepts_ section explains the basic concepts behind the different APIs,
-that is the concepts behind Flink as a stateful and timely stream processing
-system.
diff --git a/docs/concepts/index.zh.md b/docs/concepts/index.zh.md
index 2498a434abb5b07de86f134f6dd736bf85118bab..2bacf68dbb592efec008c294a1f80a1da8366665 100644
--- a/docs/concepts/index.zh.md
+++ b/docs/concepts/index.zh.md
@@ -1,8 +1,8 @@
---
-title: 概念
+title: 概念透析
nav-id: concepts
-nav-pos: 2
-nav-title: ' 概念'
+nav-pos: 3
+nav-title: ' 概念透析'
nav-parent_id: root
nav-show_overview: true
permalink: /concepts/index.html
@@ -27,57 +27,66 @@ specific language governing permissions and limitations
under the License.
-->
+The [Hands-on Tutorials]({% link tutorials/index.zh.md %}) explain the basic concepts
+of stateful and timely stream processing that underlie Flink's APIs, and provide examples of how
+these mechanisms are used in applications. Stateful stream processing is introduced in the context
+of [Data Pipelines & ETL]({% link tutorials/etl.zh.md %}#stateful-transformations)
+and is further developed in the section on [Fault Tolerance]({% link
+tutorials/fault_tolerance.zh.md %}). Timely stream processing is introduced in the section on
+[Streaming Analytics]({% link tutorials/streaming_analytics.zh.md %}).
+
+This _Concepts in Depth_ section provides a deeper understanding of how Flink's architecture and runtime
+implement these concepts.
+
+## Flink's APIs
+
Flink offers different levels of abstraction for developing streaming/batch applications.
- - The lowest level abstraction simply offers **stateful streaming**. It is
+ - The lowest level abstraction simply offers **stateful and timely stream processing**. It is
embedded into the [DataStream API]({{ site.baseurl}}{% link
- dev/datastream_api.md %}) via the [Process Function]({{ site.baseurl }}{%
- link dev/stream/operators/process_function.md %}). It allows users freely
- process events from one or more streams, and use consistent fault tolerant
+ dev/datastream_api.zh.md %}) via the [Process Function]({{ site.baseurl }}{%
+ link dev/stream/operators/process_function.zh.md %}). It allows users to freely
+ process events from one or more streams, and provides consistent, fault tolerant
*state*. In addition, users can register event time and processing time
callbacks, allowing programs to realize sophisticated computations.
- - In practice, most applications would not need the above described low level
- abstraction, but would instead program against the **Core APIs** like the
- [DataStream API]({{ site.baseurl }}{% link dev/datastream_api.md %})
- (bounded/unbounded streams) and the [DataSet API]({{ site.baseurl }}{% link
- dev/batch/index.md %}) (bounded data sets). These fluent APIs offer the
+ - In practice, many applications do not need the low-level
+ abstractions described above, and can instead program against the **Core APIs**: the
+ [DataStream API]({% link dev/datastream_api.zh.md %})
+ (bounded/unbounded streams) and the [DataSet API]({% link
+ dev/batch/index.zh.md %}) (bounded data sets). These fluent APIs offer the
common building blocks for data processing, like various forms of
user-specified transformations, joins, aggregations, windows, state, etc.
Data types processed in these APIs are represented as classes in the
respective programming languages.
The low level *Process Function* integrates with the *DataStream API*,
- making it possible to go the lower level abstraction for certain operations
- only. The *DataSet API* offers additional primitives on bounded data sets,
+ making it possible to use the lower-level abstraction on an as-needed basis.
+ The *DataSet API* offers additional primitives on bounded data sets,
like loops/iterations.
- The **Table API** is a declarative DSL centered around *tables*, which may
be dynamically changing tables (when representing streams). The [Table
- API]({{ site.baseurl }}{% link dev/table/index.md %}) follows the
+ API]({% link dev/table/index.zh.md %}) follows the
(extended) relational model: Tables have a schema attached (similar to
tables in relational databases) and the API offers comparable operations,
such as select, project, join, group-by, aggregate, etc. Table API
programs declaratively define *what logical operation should be done*
rather than specifying exactly *how the code for the operation looks*.
Though the Table API is extensible by various types of user-defined
- functions, it is less expressive than the *Core APIs*, but more concise to
+ functions, it is less expressive than the *Core APIs*, and more concise to
use (less code to write). In addition, Table API programs also go through
an optimizer that applies optimization rules before execution.
One can seamlessly convert between tables and *DataStream*/*DataSet*,
- allowing programs to mix *Table API* and with the *DataStream* and
+ allowing programs to mix the *Table API* with the *DataStream* and
*DataSet* APIs.
- The highest level abstraction offered by Flink is **SQL**. This abstraction
is similar to the *Table API* both in semantics and expressiveness, but
represents programs as SQL query expressions. The [SQL]({{ site.baseurl
- }}{% link dev/table/index.md %}#sql) abstraction closely interacts with the
+ }}{% link dev/table/index.zh.md %}#sql) abstraction closely interacts with the
Table API, and SQL queries can be executed over tables defined in the
*Table API*.
-
-This _concepts_ section explains the basic concepts behind the different APIs,
-that is the concepts behind Flink as a stateful and timely stream processing
-system.
diff --git a/docs/concepts/stateful-stream-processing.md b/docs/concepts/stateful-stream-processing.md
index b7a06ba921bda444574e97357e82e7675cc749ac..b8cdd634c0492b58952218c19aedae7132b80928 100644
--- a/docs/concepts/stateful-stream-processing.md
+++ b/docs/concepts/stateful-stream-processing.md
@@ -47,11 +47,11 @@ and [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.md %}).
Knowledge about the state also allows for rescaling Flink applications, meaning
that Flink takes care of redistributing state across parallel instances.
-[Queryable state]({{ site.baseurl }}{% link dev/stream/state/queryable_state.md
+[Queryable state]({% link dev/stream/state/queryable_state.md
%}) allows you to access state from outside of Flink during runtime.
When working with state, it might also be useful to read about [Flink's state
-backends]({{ site.baseurl }}{% link ops/state/state_backends.md %}). Flink
+backends]({% link ops/state/state_backends.md %}). Flink
provides different state backends that specify how and where state is stored.
* This will be replaced by the TOC
@@ -115,15 +115,15 @@ streams are reset to the point of the state snapshot. Any records that are
processed as part of the restarted parallel dataflow are guaranteed to not have
affected the previously checkpointed state.
-{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({{
-site.baseurl }}{% link dev/stream/state/checkpointing.md %}) for details on how
-to enable and configure checkpointing.
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({%
+link dev/stream/state/checkpointing.md %}) for details on how to enable and
+configure checkpointing.
{% info Note %} For this mechanism to realize its full guarantees, the data
stream source (such as message queue or broker) needs to be able to rewind the
stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
this ability and Flink's connector to Kafka exploits this. See [Fault
-Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}{% link
+Tolerance Guarantees of Data Sources and Sinks]({% link
dev/connectors/guarantees.md %}) for more information about the guarantees
provided by Flink's connectors.
@@ -247,7 +247,7 @@ If state was snapshotted incrementally, the operators start with the state of
the latest full snapshot and then apply a series of incremental snapshot
updates to that state.
-See [Restart Strategies]({{ site.baseurl }}{% link dev/task_failure_recovery.md
+See [Restart Strategies]({% link dev/task_failure_recovery.md
%}#restart-strategies) for more information.
### State Backends
@@ -255,7 +255,7 @@ See [Restart Strategies]({{ site.baseurl }}{% link dev/task_failure_recovery.md
`TODO: expand this section`
The exact data structures in which the key/values indexes are stored depends on
-the chosen [state backend]({{ site.baseurl }}{% link
+the chosen [state backend]({% link
ops/state/state_backends.md %}). One state backend stores data in an in-memory
hash map, another state backend uses [RocksDB](http://rocksdb.org) as the
key/value store. In addition to defining the data structure that holds the
@@ -276,7 +276,7 @@ All programs that use checkpointing can resume execution from a **savepoint**.
Savepoints allow both updating your programs and your Flink cluster without
losing any state.
-[Savepoints]({{ site.baseurl }}{% link ops/state/savepoints.md %}) are
+[Savepoints]({% link ops/state/savepoints.md %}) are
**manually triggered checkpoints**, which take a snapshot of the program and
write it out to a state backend. They rely on the regular checkpointing
mechanism for this.
diff --git a/docs/concepts/stateful-stream-processing.zh.md b/docs/concepts/stateful-stream-processing.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..e7d19ad23470804891992a43ea2532301a2a6eab
--- /dev/null
+++ b/docs/concepts/stateful-stream-processing.zh.md
@@ -0,0 +1,340 @@
+---
+title: Stateful Stream Processing
+nav-id: stateful-stream-processing
+nav-pos: 2
+nav-title: Stateful Stream Processing
+nav-parent_id: concepts
+---
+
+
+While many operations in a dataflow simply look at one individual *event at a
+time* (for example an event parser), some operations remember information
+across multiple events (for example window operators). These operations are
+called **stateful**.
+
+Some examples of stateful operations:
+
+ - When an application searches for certain event patterns, the state will
+ store the sequence of events encountered so far.
+ - When aggregating events per minute/hour/day, the state holds the pending
+ aggregates.
+ - When training a machine learning model over a stream of data points, the
+ state holds the current version of the model parameters.
+ - When historic data needs to be managed, the state allows efficient access
+ to events that occurred in the past.
+
+Flink needs to be aware of the state in order to make it fault tolerant using
+[checkpoints]({{ site.baseurl}}{% link dev/stream/state/checkpointing.zh.md %})
+and [savepoints]({{ site.baseurl }}{%link ops/state/savepoints.zh.md %}).
+
+Knowledge about the state also allows for rescaling Flink applications, meaning
+that Flink takes care of redistributing state across parallel instances.
+
+[Queryable state]({% link dev/stream/state/queryable_state.zh.md
+%}) allows you to access state from outside of Flink during runtime.
+
+When working with state, it might also be useful to read about [Flink's state
+backends]({% link ops/state/state_backends.zh.md %}). Flink
+provides different state backends that specify how and where state is stored.
+
+* This will be replaced by the TOC
+{:toc}
+
+## What is State?
+
+`TODO: expand this section`
+
+{% top %}
+
+## State in Stream & Batch Processing
+
+`TODO: What is this section about? Do we even need it?`
+
+{% top %}
+
+## Keyed State
+
+Keyed state is maintained in what can be thought of as an embedded key/value
+store. The state is partitioned and distributed strictly together with the
+streams that are read by the stateful operators. Hence, access to the key/value
+state is only possible on *keyed streams*, i.e. after a keyed/partitioned data
+exchange, and is restricted to the values associated with the current event's
+key. Aligning the keys of streams and state makes sure that all state updates
+are local operations, guaranteeing consistency without transaction overhead.
+This alignment also allows Flink to redistribute the state and adjust the
+stream partitioning transparently.
+
+
+
+Keyed State is further organized into so-called *Key Groups*. Key Groups are
+the atomic unit by which Flink can redistribute Keyed State; there are exactly
+as many Key Groups as the defined maximum parallelism. During execution each
+parallel instance of a keyed operator works with the keys for one or more Key
+Groups.
+
+## State Persistence
+
+Flink implements fault tolerance using a combination of **stream replay** and
+**checkpointing**. A checkpoint marks a specific point in each of the
+input streams along with the corresponding state for each of the operators. A
+streaming dataflow can be resumed from a checkpoint while maintaining
+consistency *(exactly-once processing semantics)* by restoring the state of the
+operators and replaying the records from the point of the checkpoint.
+
+The checkpoint interval is a means of trading off the overhead of fault
+tolerance during execution with the recovery time (the number of records that
+need to be replayed).
+
+The fault tolerance mechanism continuously draws snapshots of the distributed
+streaming data flow. For streaming applications with small state, these
+snapshots are very light-weight and can be drawn frequently without much impact
+on performance. The state of the streaming applications is stored at a
+configurable place, usually in a distributed file system.
+
+In case of a program failure (due to machine-, network-, or software failure),
+Flink stops the distributed streaming dataflow. The system then restarts the
+operators and resets them to the latest successful checkpoint. The input
+streams are reset to the point of the state snapshot. Any records that are
+processed as part of the restarted parallel dataflow are guaranteed to not have
+affected the previously checkpointed state.
+
+{% info Note %} By default, checkpointing is disabled. See [Checkpointing]({%
+link dev/stream/state/checkpointing.zh.md %}) for details on how to enable and
+configure checkpointing.
+
+{% info Note %} For this mechanism to realize its full guarantees, the data
+stream source (such as message queue or broker) needs to be able to rewind the
+stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has
+this ability and Flink's connector to Kafka exploits this. See [Fault
+Tolerance Guarantees of Data Sources and Sinks]({% link
+dev/connectors/guarantees.zh.md %}) for more information about the guarantees
+provided by Flink's connectors.
+
+{% info Note %} Because Flink's checkpoints are realized through distributed
+snapshots, we use the words *snapshot* and *checkpoint* interchangeably. Often
+we also use the term *snapshot* to mean either *checkpoint* or *savepoint*.
+
+### Checkpointing
+
+The central part of Flink's fault tolerance mechanism is drawing consistent
+snapshots of the distributed data stream and operator state. These snapshots
+act as consistent checkpoints to which the system can fall back in case of a
+failure. Flink's mechanism for drawing these snapshots is described in
+"[Lightweight Asynchronous Snapshots for Distributed
+Dataflows](http://arxiv.org/abs/1506.08603)". It is inspired by the standard
+[Chandy-Lamport
+algorithm](http://research.microsoft.com/en-us/um/people/lamport/pubs/chandy.pdf)
+for distributed snapshots and is specifically tailored to Flink's execution
+model.
+
+Keep in mind that everything to do with checkpointing can be done
+asynchronously. The checkpoint barriers don't travel in lock step and
+operations can asynchronously snapshot their state.
+
+
+#### Barriers
+
+A core element in Flink's distributed snapshotting are the *stream barriers*.
+These barriers are injected into the data stream and flow with the records as
+part of the data stream. Barriers never overtake records, they flow strictly in
+line. A barrier separates the records in the data stream into the set of
+records that goes into the current snapshot, and the records that go into the
+next snapshot. Each barrier carries the ID of the snapshot whose records it
+pushed in front of it. Barriers do not interrupt the flow of the stream and are
+hence very lightweight. Multiple barriers from different snapshots can be in
+the stream at the same time, which means that various snapshots may happen
+concurrently.
+
+
+
+
+
+Stream barriers are injected into the parallel data flow at the stream sources.
+The point where the barriers for snapshot *n* are injected (let's call it
+Sn) is the position in the source stream up to which the
+snapshot covers the data. For example, in Apache Kafka, this position would be
+the last record's offset in the partition. This position Sn
+is reported to the *checkpoint coordinator* (Flink's JobManager).
+
+The barriers then flow downstream. When an intermediate operator has received a
+barrier for snapshot *n* from all of its input streams, it emits a barrier for
+snapshot *n* into all of its outgoing streams. Once a sink operator (the end of
+a streaming DAG) has received the barrier *n* from all of its input streams, it
+acknowledges that snapshot *n* to the checkpoint coordinator. After all sinks
+have acknowledged a snapshot, it is considered completed.
+
+Once snapshot *n* has been completed, the job will never again ask the source
+for records from before Sn, since at that point these records
+(and their descendant records) will have passed through the entire data flow
+topology.
+
+
+
+
+
+Operators that receive more than one input stream need to *align* the input
+streams on the snapshot barriers. The figure above illustrates this:
+
+ - As soon as the operator receives snapshot barrier *n* from an incoming
+ stream, it cannot process any further records from that stream until it has
+ received the barrier *n* from the other inputs as well. Otherwise, it would
+ mix records that belong to snapshot *n* and with records that belong to
+ snapshot *n+1*.
+ - Streams that report barrier *n* are temporarily set aside. Records that are
+ received from these streams are not processed, but put into an input
+ buffer.
+ - Once the last stream has received barrier *n*, the operator emits all
+ pending outgoing records, and then emits snapshot *n* barriers itself.
+ - After that, it resumes processing records from all input streams,
+ processing records from the input buffers before processing the records
+ from the streams.
+
+#### Snapshotting Operator State
+
+When operators contain any form of *state*, this state must be part of the
+snapshots as well.
+
+Operators snapshot their state at the point in time when they have received all
+snapshot barriers from their input streams, and before emitting the barriers to
+their output streams. At that point, all updates to the state from records
+before the barriers will have been made, and no updates that depend on records
+from after the barriers have been applied. Because the state of a snapshot may
+be large, it is stored in a configurable *[state backend]({{ site.baseurl }}{%
+link ops/state/state_backends.zh.md %})*. By default, this is the JobManager's
+memory, but for production use a distributed reliable storage should be
+configured (such as HDFS). After the state has been stored, the operator
+acknowledges the checkpoint, emits the snapshot barrier into the output
+streams, and proceeds.
+
+The resulting snapshot now contains:
+
+ - For each parallel stream data source, the offset/position in the stream
+ when the snapshot was started
+ - For each operator, a pointer to the state that was stored as part of the
+ snapshot
+
+
+
+
+
+#### Recovery
+
+Recovery under this mechanism is straightforward: Upon a failure, Flink selects
+the latest completed checkpoint *k*. The system then re-deploys the entire
+distributed dataflow, and gives each operator the state that was snapshotted as
+part of checkpoint *k*. The sources are set to start reading the stream from
+position Sk. For example in Apache Kafka, that means telling
+the consumer to start fetching from offset Sk.
+
+If state was snapshotted incrementally, the operators start with the state of
+the latest full snapshot and then apply a series of incremental snapshot
+updates to that state.
+
+See [Restart Strategies]({% link dev/task_failure_recovery.zh.md
+%}#restart-strategies) for more information.
+
+### State Backends
+
+`TODO: expand this section`
+
+The exact data structures in which the key/values indexes are stored depends on
+the chosen [state backend]({% link
+ops/state/state_backends.zh.md %}). One state backend stores data in an in-memory
+hash map, another state backend uses [RocksDB](http://rocksdb.org) as the
+key/value store. In addition to defining the data structure that holds the
+state, the state backends also implement the logic to take a point-in-time
+snapshot of the key/value state and store that snapshot as part of a
+checkpoint. State backends can be configured without changing your application
+logic.
+
+
+
+{% top %}
+
+### Savepoints
+
+`TODO: expand this section`
+
+All programs that use checkpointing can resume execution from a **savepoint**.
+Savepoints allow both updating your programs and your Flink cluster without
+losing any state.
+
+[Savepoints]({% link ops/state/savepoints.zh.md %}) are
+**manually triggered checkpoints**, which take a snapshot of the program and
+write it out to a state backend. They rely on the regular checkpointing
+mechanism for this.
+
+Savepoints are similar to checkpoints except that they are
+**triggered by the user** and **don't automatically expire** when newer
+checkpoints are completed.
+
+{% top %}
+
+### Exactly Once vs. At Least Once
+
+The alignment step may add latency to the streaming program. Usually, this
+extra latency is on the order of a few milliseconds, but we have seen cases
+where the latency of some outliers increased noticeably. For applications that
+require consistently super low latencies (few milliseconds) for all records,
+Flink has a switch to skip the stream alignment during a checkpoint. Checkpoint
+snapshots are still drawn as soon as an operator has seen the checkpoint
+barrier from each input.
+
+When the alignment is skipped, an operator keeps processing all inputs, even
+after some checkpoint barriers for checkpoint *n* arrived. That way, the
+operator also processes elements that belong to checkpoint *n+1* before the
+state snapshot for checkpoint *n* was taken. On a restore, these records will
+occur as duplicates, because they are both included in the state snapshot of
+checkpoint *n*, and will be replayed as part of the data after checkpoint *n*.
+
+{% info Note %} Alignment happens only for operators with multiple predecessors
+(joins) as well as operators with multiple senders (after a stream
+repartitioning/shuffle). Because of that, dataflows with only embarrassingly
+parallel streaming operations (`map()`, `flatMap()`, `filter()`, ...) actually
+give *exactly once* guarantees even in *at least once* mode.
+
+{% top %}
+
+## End-to-end Exactly-Once Programs
+
+`TODO: add`
+
+## State and Fault Tolerance in Batch Programs
+
+Flink executes [batch programs](../dev/batch/index.html) as a special case of
+streaming programs, where the streams are bounded (finite number of elements).
+A *DataSet* is treated internally as a stream of data. The concepts above thus
+apply to batch programs in the same way as well as they apply to streaming
+programs, with minor exceptions:
+
+ - [Fault tolerance for batch programs](../dev/batch/fault_tolerance.html)
+ does not use checkpointing. Recovery happens by fully replaying the
+ streams. That is possible, because inputs are bounded. This pushes the
+ cost more towards the recovery, but makes the regular processing cheaper,
+ because it avoids checkpoints.
+
+ - Stateful operations in the DataSet API use simplified in-memory/out-of-core
+ data structures, rather than key/value indexes.
+
+ - The DataSet API introduces special synchronized (superstep-based)
+ iterations, which are only possible on bounded streams. For details, check
+ out the [iteration docs]({{ site.baseurl }}/dev/batch/iterations.html).
+
+{% top %}
diff --git a/docs/concepts/stream-processing.md b/docs/concepts/stream-processing.md
deleted file mode 100644
index aa094a7b2a8a4aabe9c1238cf5ba79d3477a89c7..0000000000000000000000000000000000000000
--- a/docs/concepts/stream-processing.md
+++ /dev/null
@@ -1,96 +0,0 @@
----
-title: Stream Processing
-nav-id: stream-processing
-nav-pos: 1
-nav-title: Stream Processing
-nav-parent_id: concepts
----
-
-
-`TODO: Add introduction`
-* This will be replaced by the TOC
-{:toc}
-
-## A Unified System for Batch & Stream Processing
-
-`TODO`
-
-{% top %}
-
-## Programs and Dataflows
-
-The basic building blocks of Flink programs are **streams** and
-**transformations**. Conceptually a *stream* is a (potentially never-ending)
-flow of data records, and a *transformation* is an operation that takes one or
-more streams as input, and produces one or more output streams as a result.
-
-When executed, Flink programs are mapped to **streaming dataflows**, consisting
-of **streams** and transformation **operators**. Each dataflow starts with one
-or more **sources** and ends in one or more **sinks**. The dataflows resemble
-arbitrary **directed acyclic graphs** *(DAGs)*. Although special forms of
-cycles are permitted via *iteration* constructs, for the most part we will
-gloss over this for simplicity.
-
-
-
-Often there is a one-to-one correspondence between the transformations in the
-programs and the operators in the dataflow. Sometimes, however, one
-transformation may consist of multiple operators.
-
-{% top %}
-
-## Parallel Dataflows
-
-Programs in Flink are inherently parallel and distributed. During execution, a
-*stream* has one or more **stream partitions**, and each *operator* has one or
-more **operator subtasks**. The operator subtasks are independent of one
-another, and execute in different threads and possibly on different machines or
-containers.
-
-The number of operator subtasks is the **parallelism** of that particular
-operator. The parallelism of a stream is always that of its producing operator.
-Different operators of the same program may have different levels of
-parallelism.
-
-
-
-Streams can transport data between two operators in a *one-to-one* (or
-*forwarding*) pattern, or in a *redistributing* pattern:
-
- - **One-to-one** streams (for example between the *Source* and the *map()*
- operators in the figure above) preserve the partitioning and ordering of
- the elements. That means that subtask[1] of the *map()* operator will see
- the same elements in the same order as they were produced by subtask[1] of
- the *Source* operator.
-
- - **Redistributing** streams (as between *map()* and *keyBy/window* above, as
- well as between *keyBy/window* and *Sink*) change the partitioning of
- streams. Each *operator subtask* sends data to different target subtasks,
- depending on the selected transformation. Examples are *keyBy()* (which
- re-partitions by hashing the key), *broadcast()*, or *rebalance()* (which
- re-partitions randomly). In a *redistributing* exchange the ordering among
- the elements is only preserved within each pair of sending and receiving
- subtasks (for example, subtask[1] of *map()* and subtask[2] of
- *keyBy/window*). So in this example, the ordering within each key is
- preserved, but the parallelism does introduce non-determinism regarding the
- order in which the aggregated results for different keys arrive at the
- sink.
-
-{% top %}
diff --git a/docs/concepts/timely-stream-processing.md b/docs/concepts/timely-stream-processing.md
index a67d6de0d9c5d02857a4875a0692bd5431cfa218..cb2c1579e8b89ab901783cf4bcce967d50061e44 100644
--- a/docs/concepts/timely-stream-processing.md
+++ b/docs/concepts/timely-stream-processing.md
@@ -183,7 +183,7 @@ evaluation of event time windows.
For this reason, streaming programs may explicitly expect some *late* elements.
Late elements are elements that arrive after the system's event time clock (as
signaled by the watermarks) has already passed the time of the late element's
-timestamp. See [Allowed Lateness]({{ site.baseurl }}{% link
+timestamp. See [Allowed Lateness]({% link
dev/stream/operators/windows.md %}#allowed-lateness) for more information on
how to work with late elements in event time windows.
@@ -204,8 +204,7 @@ overlap), and *session windows* (punctuated by a gap of inactivity).
Please check out this [blog
post](https://flink.apache.org/news/2015/12/04/Introducing-windows.html) for
-additional examples of windows or take a look a [window documentation]({{
-site.baseurl }}{% link dev/stream/operators/windows.md %}) of the DataStream
-API.
+additional examples of windows or take a look a [window documentation]({% link
+dev/stream/operators/windows.md %}) of the DataStream API.
{% top %}
diff --git a/docs/concepts/timely-stream-processing.zh.md b/docs/concepts/timely-stream-processing.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..9f80f72ad3fb6af66f3d198c85dd1b93c0064f7c
--- /dev/null
+++ b/docs/concepts/timely-stream-processing.zh.md
@@ -0,0 +1,210 @@
+---
+title: Timely Stream Processing
+nav-id: timely-stream-processing
+nav-pos: 3
+nav-title: Timely Stream Processing
+nav-parent_id: concepts
+---
+
+
+`TODO: add introduction`
+
+* This will be replaced by the TOC
+{:toc}
+
+## Latency & Completeness
+
+`TODO: add these two sections`
+
+### Latency vs. Completeness in Batch & Stream Processing
+
+{% top %}
+
+## Notions of Time: Event Time and Processing Time
+
+When referring to time in a streaming program (for example to define windows),
+one can refer to different notions of *time*:
+
+- **Processing time:** Processing time refers to the system time of the machine
+ that is executing the respective operation.
+
+ When a streaming program runs on processing time, all time-based operations
+ (like time windows) will use the system clock of the machines that run the
+ respective operator. An hourly processing time window will include all
+ records that arrived at a specific operator between the times when the system
+ clock indicated the full hour. For example, if an application begins running
+ at 9:15am, the first hourly processing time window will include events
+ processed between 9:15am and 10:00am, the next window will include events
+ processed between 10:00am and 11:00am, and so on.
+
+ Processing time is the simplest notion of time and requires no coordination
+ between streams and machines. It provides the best performance and the
+ lowest latency. However, in distributed and asynchronous environments
+ processing time does not provide determinism, because it is susceptible to
+ the speed at which records arrive in the system (for example from the message
+ queue), to the speed at which the records flow between operators inside the
+ system, and to outages (scheduled, or otherwise).
+
+- **Event time:** Event time is the time that each individual event occurred on
+ its producing device. This time is typically embedded within the records
+ before they enter Flink, and that *event timestamp* can be extracted from
+ each record. In event time, the progress of time depends on the data, not on
+ any wall clocks. Event time programs must specify how to generate *Event Time
+ Watermarks*, which is the mechanism that signals progress in event time. This
+ watermarking mechanism is described in a later section,
+ [below](#event-time-and-watermarks).
+
+ In a perfect world, event time processing would yield completely consistent
+ and deterministic results, regardless of when events arrive, or their
+ ordering. However, unless the events are known to arrive in-order (by
+ timestamp), event time processing incurs some latency while waiting for
+ out-of-order events. As it is only possible to wait for a finite period of
+ time, this places a limit on how deterministic event time applications can
+ be.
+
+ Assuming all of the data has arrived, event time operations will behave as
+ expected, and produce correct and consistent results even when working with
+ out-of-order or late events, or when reprocessing historic data. For example,
+ an hourly event time window will contain all records that carry an event
+ timestamp that falls into that hour, regardless of the order in which they
+ arrive, or when they are processed. (See the section on [late
+ events](#late-elements) for more information.)
+
+ Note that sometimes when event time programs are processing live data in
+ real-time, they will use some *processing time* operations in order to
+ guarantee that they are progressing in a timely fashion.
+
+
+
+{% top %}
+
+## Event Time and Watermarks
+
+*Note: Flink implements many techniques from the Dataflow Model. For a good
+introduction to event time and watermarks, have a look at the articles below.*
+
+ - [Streaming
+ 101](https://www.oreilly.com/ideas/the-world-beyond-batch-streaming-101) by
+ Tyler Akidau
+ - The [Dataflow Model
+ paper](https://research.google.com/pubs/archive/43864.pdf)
+
+
+A stream processor that supports *event time* needs a way to measure the
+progress of event time. For example, a window operator that builds hourly
+windows needs to be notified when event time has passed beyond the end of an
+hour, so that the operator can close the window in progress.
+
+*Event time* can progress independently of *processing time* (measured by wall
+clocks). For example, in one program the current *event time* of an operator
+may trail slightly behind the *processing time* (accounting for a delay in
+receiving the events), while both proceed at the same speed. On the other
+hand, another streaming program might progress through weeks of event time with
+only a few seconds of processing, by fast-forwarding through some historic data
+already buffered in a Kafka topic (or another message queue).
+
+------
+
+The mechanism in Flink to measure progress in event time is **watermarks**.
+Watermarks flow as part of the data stream and carry a timestamp *t*. A
+*Watermark(t)* declares that event time has reached time *t* in that stream,
+meaning that there should be no more elements from the stream with a timestamp
+*t' <= t* (i.e. events with timestamps older or equal to the watermark).
+
+The figure below shows a stream of events with (logical) timestamps, and
+watermarks flowing inline. In this example the events are in order (with
+respect to their timestamps), meaning that the watermarks are simply periodic
+markers in the stream.
+
+
+
+Watermarks are crucial for *out-of-order* streams, as illustrated below, where
+the events are not ordered by their timestamps. In general a watermark is a
+declaration that by that point in the stream, all events up to a certain
+timestamp should have arrived. Once a watermark reaches an operator, the
+operator can advance its internal *event time clock* to the value of the
+watermark.
+
+
+
+Note that event time is inherited by a freshly created stream element (or
+elements) from either the event that produced them or from watermark that
+triggered creation of those elements.
+
+### Watermarks in Parallel Streams
+
+Watermarks are generated at, or directly after, source functions. Each parallel
+subtask of a source function usually generates its watermarks independently.
+These watermarks define the event time at that particular parallel source.
+
+As the watermarks flow through the streaming program, they advance the event
+time at the operators where they arrive. Whenever an operator advances its
+event time, it generates a new watermark downstream for its successor
+operators.
+
+Some operators consume multiple input streams; a union, for example, or
+operators following a *keyBy(...)* or *partition(...)* function. Such an
+operator's current event time is the minimum of its input streams' event times.
+As its input streams update their event times, so does the operator.
+
+The figure below shows an example of events and watermarks flowing through
+parallel streams, and operators tracking event time.
+
+
+
+## Lateness
+
+It is possible that certain elements will violate the watermark condition,
+meaning that even after the *Watermark(t)* has occurred, more elements with
+timestamp *t' <= t* will occur. In fact, in many real world setups, certain
+elements can be arbitrarily delayed, making it impossible to specify a time by
+which all elements of a certain event timestamp will have occurred.
+Furthermore, even if the lateness can be bounded, delaying the watermarks by
+too much is often not desirable, because it causes too much delay in the
+evaluation of event time windows.
+
+For this reason, streaming programs may explicitly expect some *late* elements.
+Late elements are elements that arrive after the system's event time clock (as
+signaled by the watermarks) has already passed the time of the late element's
+timestamp. See [Allowed Lateness]({% link
+dev/stream/operators/windows.zh.md %}#allowed-lateness) for more information on
+how to work with late elements in event time windows.
+
+## Windowing
+
+Aggregating events (e.g., counts, sums) works differently on streams than in
+batch processing. For example, it is impossible to count all elements in a
+stream, because streams are in general infinite (unbounded). Instead,
+aggregates on streams (counts, sums, etc), are scoped by **windows**, such as
+*"count over the last 5 minutes"*, or *"sum of the last 100 elements"*.
+
+Windows can be *time driven* (example: every 30 seconds) or *data driven*
+(example: every 100 elements). One typically distinguishes different types of
+windows, such as *tumbling windows* (no overlap), *sliding windows* (with
+overlap), and *session windows* (punctuated by a gap of inactivity).
+
+
+
+Please check out this [blog
+post](https://flink.apache.org/news/2015/12/04/Introducing-windows.html) for
+additional examples of windows or take a look a [window documentation]({% link
+dev/stream/operators/windows.zh.md %}) of the DataStream API.
+
+{% top %}
diff --git a/docs/connectors/index.md b/docs/connectors/index.md
new file mode 100644
index 0000000000000000000000000000000000000000..5a744166b3ccce0a74c2b63cfc37d8446d8bc4aa
--- /dev/null
+++ b/docs/connectors/index.md
@@ -0,0 +1,28 @@
+---
+title: "Connectors"
+nav-id: connectors-root
+nav-title: ' Connectors'
+nav-parent_id: root
+nav-pos: 7
+---
+
+
+* toc
+{:toc}
diff --git a/docs/connectors/index.zh.md b/docs/connectors/index.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..5a744166b3ccce0a74c2b63cfc37d8446d8bc4aa
--- /dev/null
+++ b/docs/connectors/index.zh.md
@@ -0,0 +1,28 @@
+---
+title: "Connectors"
+nav-id: connectors-root
+nav-title: ' Connectors'
+nav-parent_id: root
+nav-pos: 7
+---
+
+
+* toc
+{:toc}
diff --git a/docs/dev/api_concepts.md b/docs/dev/api_concepts.md
deleted file mode 100644
index 234b074562cabfed06778e727c8c044ca65dc107..0000000000000000000000000000000000000000
--- a/docs/dev/api_concepts.md
+++ /dev/null
@@ -1,913 +0,0 @@
----
-title: "Basic API Concepts"
-nav-parent_id: dev
-nav-pos: 1
-nav-show_overview: true
-nav-id: api-concepts
----
-
-
-Flink programs are regular programs that implement transformations on distributed collections
-(e.g., filtering, mapping, updating state, joining, grouping, defining windows, aggregating).
-Collections are initially created from sources (e.g., by reading from files, kafka topics, or from local, in-memory
-collections). Results are returned via sinks, which may for example write the data to
-(distributed) files, or to standard output (for example, the command line terminal).
-Flink programs run in a variety of contexts, standalone, or embedded in other programs.
-The execution can happen in a local JVM, or on clusters of many machines.
-
-Depending on the type of data sources, i.e. bounded or unbounded sources, you would either
-write a batch program or a streaming program where the DataSet API is used for batch
-and the DataStream API is used for streaming. This guide will introduce the basic concepts
-that are common to both APIs but please see our
-[Streaming Guide]({{ site.baseurl }}/dev/datastream_api.html) and
-[Batch Guide]({{ site.baseurl }}/dev/batch/index.html) for concrete information about
-writing programs with each API.
-
-**NOTE:** When showing actual examples of how the APIs can be used we will use
-`StreamingExecutionEnvironment` and the `DataStream` API. The concepts are exactly the same
-in the `DataSet` API, just replace by `ExecutionEnvironment` and `DataSet`.
-
-* This will be replaced by the TOC
-{:toc}
-
-DataSet and DataStream
-----------------------
-
-Flink has the special classes `DataSet` and `DataStream` to represent data in a program. You
-can think of them as immutable collections of data that can contain duplicates. In the case
-of `DataSet` the data is finite while for a `DataStream` the number of elements can be unbounded.
-
-These collections differ from regular Java collections in some key ways. First, they
-are immutable, meaning that once they are created you cannot add or remove elements. You can also
-not simply inspect the elements inside.
-
-A collection is initially created by adding a source in a Flink program and new collections are
-derived from these by transforming them using API methods such as `map`, `filter` and so on.
-
-Anatomy of a Flink Program
---------------------------
-
-Flink programs look like regular programs that transform collections of data.
-Each program consists of the same basic parts:
-
-1. Obtain an `execution environment`,
-2. Load/create the initial data,
-3. Specify transformations on this data,
-4. Specify where to put the results of your computations,
-5. Trigger the program execution
-
-
-
-
-
-
-We will now give an overview of each of those steps, please refer to the respective sections for
-more details. Note that all core classes of the Java DataSet API are found in the package
-{% gh_link /flink-java/src/main/java/org/apache/flink/api/java "org.apache.flink.api.java" %}
-while the classes of the Java DataStream API can be found in
-{% gh_link /flink-streaming-java/src/main/java/org/apache/flink/streaming/api "org.apache.flink.streaming.api" %}.
-
-The `StreamExecutionEnvironment` is the basis for all Flink programs. You can
-obtain one using these static methods on `StreamExecutionEnvironment`:
-
-{% highlight java %}
-getExecutionEnvironment()
-
-createLocalEnvironment()
-
-createRemoteEnvironment(String host, int port, String... jarFiles)
-{% endhighlight %}
-
-Typically, you only need to use `getExecutionEnvironment()`, since this
-will do the right thing depending on the context: if you are executing
-your program inside an IDE or as a regular Java program it will create
-a local environment that will execute your program on your local machine. If
-you created a JAR file from your program, and invoke it through the
-[command line]({{ site.baseurl }}/ops/cli.html), the Flink cluster manager
-will execute your main method and `getExecutionEnvironment()` will return
-an execution environment for executing your program on a cluster.
-
-For specifying data sources the execution environment has several methods
-to read from files using various methods: you can just read them line by line,
-as CSV files, or using completely custom data input formats. To just read
-a text file as a sequence of lines, you can use:
-
-{% highlight java %}
-final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-DataStream text = env.readTextFile("file:///path/to/file");
-{% endhighlight %}
-
-This will give you a DataStream on which you can then apply transformations to create new
-derived DataStreams.
-
-You apply transformations by calling methods on DataStream with a transformation
-functions. For example, a map transformation looks like this:
-
-{% highlight java %}
-DataStream input = ...;
-
-DataStream parsed = input.map(new MapFunction() {
- @Override
- public Integer map(String value) {
- return Integer.parseInt(value);
- }
-});
-{% endhighlight %}
-
-This will create a new DataStream by converting every String in the original
-collection to an Integer.
-
-Once you have a DataStream containing your final results, you can write it to an outside system
-by creating a sink. These are just some example methods for creating a sink:
-
-{% highlight java %}
-writeAsText(String path)
-
-print()
-{% endhighlight %}
-
-
-
-
-We will now give an overview of each of those steps, please refer to the respective sections for
-more details. Note that all core classes of the Scala DataSet API are found in the package
-{% gh_link /flink-scala/src/main/scala/org/apache/flink/api/scala "org.apache.flink.api.scala" %}
-while the classes of the Scala DataStream API can be found in
-{% gh_link /flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala "org.apache.flink.streaming.api.scala" %}.
-
-The `StreamExecutionEnvironment` is the basis for all Flink programs. You can
-obtain one using these static methods on `StreamExecutionEnvironment`:
-
-{% highlight scala %}
-getExecutionEnvironment()
-
-createLocalEnvironment()
-
-createRemoteEnvironment(host: String, port: Int, jarFiles: String*)
-{% endhighlight %}
-
-Typically, you only need to use `getExecutionEnvironment()`, since this
-will do the right thing depending on the context: if you are executing
-your program inside an IDE or as a regular Java program it will create
-a local environment that will execute your program on your local machine. If
-you created a JAR file from your program, and invoke it through the
-[command line]({{ site.baseurl }}/ops/cli.html), the Flink cluster manager
-will execute your main method and `getExecutionEnvironment()` will return
-an execution environment for executing your program on a cluster.
-
-For specifying data sources the execution environment has several methods
-to read from files using various methods: you can just read them line by line,
-as CSV files, or using completely custom data input formats. To just read
-a text file as a sequence of lines, you can use:
-
-{% highlight scala %}
-val env = StreamExecutionEnvironment.getExecutionEnvironment()
-
-val text: DataStream[String] = env.readTextFile("file:///path/to/file")
-{% endhighlight %}
-
-This will give you a DataStream on which you can then apply transformations to create new
-derived DataStreams.
-
-You apply transformations by calling methods on DataSet with a transformation
-functions. For example, a map transformation looks like this:
-
-{% highlight scala %}
-val input: DataSet[String] = ...
-
-val mapped = input.map { x => x.toInt }
-{% endhighlight %}
-
-This will create a new DataStream by converting every String in the original
-collection to an Integer.
-
-Once you have a DataStream containing your final results, you can write it to an outside system
-by creating a sink. These are just some example methods for creating a sink:
-
-{% highlight scala %}
-writeAsText(path: String)
-
-print()
-{% endhighlight %}
-
-
-
-
-Once you specified the complete program you need to **trigger the program execution** by calling
-`execute()` on the `StreamExecutionEnvironment`.
-Depending on the type of the `ExecutionEnvironment` the execution will be triggered on your local
-machine or submit your program for execution on a cluster.
-
-The `execute()` method will wait for the job to finish and then return a `JobExecutionResult`, this contains execution
-times and accumulator results.
-
-If you don't want to wait for the job to finish, you can trigger asynchronous job execution by calling
-`executeAysnc()` on the `StreamExecutionEnvironment`. It will return a `JobClient` with which you can
-communicate with the job you just submitted. For instance, here is how to implement the semantics
-of `execute()` by using `executeAsync()`.
-
-{% highlight java %}
-final JobClient jobClient = env.executeAsync();
-
-final JobExecutionResult jobExecutionResult = jobClient.getJobExecutionResult(userClassloader).get();
-{% endhighlight %}
-
-Please see the [Streaming Guide]({{ site.baseurl }}/dev/datastream_api.html)
-for information about streaming data sources and sink and for more in-depths information
-about the supported transformations on DataStream.
-
-Check out the [Batch Guide]({{ site.baseurl }}/dev/batch/index.html)
-for information about batch data sources and sink and for more in-depths information
-about the supported transformations on DataSet.
-
-
-{% top %}
-
-Lazy Evaluation
----------------
-
-All Flink programs are executed lazily: When the program's main method is executed, the data loading
-and transformations do not happen directly. Rather, each operation is created and added to the
-program's plan. The operations are actually executed when the execution is explicitly triggered by
-an `execute()` call on the execution environment. Whether the program is executed locally
-or on a cluster depends on the type of execution environment
-
-The lazy evaluation lets you construct sophisticated programs that Flink executes as one
-holistically planned unit.
-
-{% top %}
-
-Specifying Keys
----------------
-
-Some transformations (join, coGroup, keyBy, groupBy) require that a key be defined on
-a collection of elements. Other transformations (Reduce, GroupReduce,
-Aggregate, Windows) allow data being grouped on a key before they are
-applied.
-
-A DataSet is grouped as
-{% highlight java %}
-DataSet<...> input = // [...]
-DataSet<...> reduced = input
- .groupBy(/*define key here*/)
- .reduceGroup(/*do something*/);
-{% endhighlight %}
-
-while a key can be specified on a DataStream using
-{% highlight java %}
-DataStream<...> input = // [...]
-DataStream<...> windowed = input
- .keyBy(/*define key here*/)
- .window(/*window specification*/);
-{% endhighlight %}
-
-The data model of Flink is not based on key-value pairs. Therefore,
-you do not need to physically pack the data set types into keys and
-values. Keys are "virtual": they are defined as functions over the
-actual data to guide the grouping operator.
-
-**NOTE:** In the following discussion we will use the `DataStream` API and `keyBy`.
-For the DataSet API you just have to replace by `DataSet` and `groupBy`.
-
-### Define keys for Tuples
-{:.no_toc}
-
-The simplest case is grouping Tuples on one or more
-fields of the Tuple:
-
-
-
-Here, we group the tuples on a composite key consisting of the first and the
-second field.
-
-A note on nested Tuples: If you have a DataStream with a nested tuple, such as:
-
-{% highlight java %}
-DataStream,String,Long>> ds;
-{% endhighlight %}
-
-Specifying `keyBy(0)` will cause the system to use the full `Tuple2` as a key (with the Integer and Float being the key). If you want to "navigate" into the nested `Tuple2`, you have to use field expression keys which are explained below.
-
-### Define keys using Field Expressions
-{:.no_toc}
-
-You can use String-based field expressions to reference nested fields and define keys for grouping, sorting, joining, or coGrouping.
-
-Field expressions make it very easy to select fields in (nested) composite types such as [Tuple](#tuples-and-case-classes) and [POJO](#pojos) types.
-
-
-
-
-In the example below, we have a `WC` POJO with two fields "word" and "count". To group by the field `word`, we just pass its name to the `keyBy()` function.
-{% highlight java %}
-// some ordinary POJO (Plain old Java Object)
-public class WC {
- public String word;
- public int count;
-}
-DataStream words = // [...]
-DataStream wordCounts = words.keyBy("word").window(/*window specification*/);
-{% endhighlight %}
-
-**Field Expression Syntax**:
-
-- Select POJO fields by their field name. For example `"user"` refers to the "user" field of a POJO type.
-
-- Select Tuple fields by their field name or 0-offset field index. For example `"f0"` and `"5"` refer to the first and sixth field of a Java Tuple type, respectively.
-
-- You can select nested fields in POJOs and Tuples. For example `"user.zip"` refers to the "zip" field of a POJO which is stored in the "user" field of a POJO type. Arbitrary nesting and mixing of POJOs and Tuples is supported such as `"f1.user.zip"` or `"user.f3.1.zip"`.
-
-- You can select the full type using the `"*"` wildcard expressions. This does also work for types which are not Tuple or POJO types.
-
-**Field Expression Example**:
-
-{% highlight java %}
-public static class WC {
- public ComplexNestedClass complex; //nested POJO
- private int count;
- // getter / setter for private field (count)
- public int getCount() {
- return count;
- }
- public void setCount(int c) {
- this.count = c;
- }
-}
-public static class ComplexNestedClass {
- public Integer someNumber;
- public float someFloat;
- public Tuple3 word;
- public IntWritable hadoopCitizen;
-}
-{% endhighlight %}
-
-These are valid field expressions for the example code above:
-
-- `"count"`: The count field in the `WC` class.
-
-- `"complex"`: Recursively selects all fields of the field complex of POJO type `ComplexNestedClass`.
-
-- `"complex.word.f2"`: Selects the last field of the nested `Tuple3`.
-
-- `"complex.hadoopCitizen"`: Selects the Hadoop `IntWritable` type.
-
-
-
-
-In the example below, we have a `WC` POJO with two fields "word" and "count". To group by the field `word`, we just pass its name to the `keyBy()` function.
-{% highlight scala %}
-// some ordinary POJO (Plain old Java Object)
-class WC(var word: String, var count: Int) {
- def this() { this("", 0L) }
-}
-val words: DataStream[WC] = // [...]
-val wordCounts = words.keyBy("word").window(/*window specification*/)
-
-// or, as a case class, which is less typing
-case class WC(word: String, count: Int)
-val words: DataStream[WC] = // [...]
-val wordCounts = words.keyBy("word").window(/*window specification*/)
-{% endhighlight %}
-
-**Field Expression Syntax**:
-
-- Select POJO fields by their field name. For example `"user"` refers to the "user" field of a POJO type.
-
-- Select Tuple fields by their 1-offset field name or 0-offset field index. For example `"_1"` and `"5"` refer to the first and sixth field of a Scala Tuple type, respectively.
-
-- You can select nested fields in POJOs and Tuples. For example `"user.zip"` refers to the "zip" field of a POJO which is stored in the "user" field of a POJO type. Arbitrary nesting and mixing of POJOs and Tuples is supported such as `"_2.user.zip"` or `"user._4.1.zip"`.
-
-- You can select the full type using the `"_"` wildcard expressions. This does also work for types which are not Tuple or POJO types.
-
-**Field Expression Example**:
-
-{% highlight scala %}
-class WC(var complex: ComplexNestedClass, var count: Int) {
- def this() { this(null, 0) }
-}
-
-class ComplexNestedClass(
- var someNumber: Int,
- someFloat: Float,
- word: (Long, Long, String),
- hadoopCitizen: IntWritable) {
- def this() { this(0, 0, (0, 0, ""), new IntWritable(0)) }
-}
-{% endhighlight %}
-
-These are valid field expressions for the example code above:
-
-- `"count"`: The count field in the `WC` class.
-
-- `"complex"`: Recursively selects all fields of the field complex of POJO type `ComplexNestedClass`.
-
-- `"complex.word._3"`: Selects the last field of the nested `Tuple3`.
-
-- `"complex.hadoopCitizen"`: Selects the Hadoop `IntWritable` type.
-
-
-
-
-### Define keys using Key Selector Functions
-{:.no_toc}
-
-An additional way to define keys are "key selector" functions. A key selector function
-takes a single element as input and returns the key for the element. The key can be of any type and be derived from deterministic computations.
-
-The following example shows a key selector function that simply returns the field of an object:
-
-
-
-{% highlight java %}
-// some ordinary POJO
-public class WC {public String word; public int count;}
-DataStream words = // [...]
-KeyedStream keyed = words
- .keyBy(new KeySelector() {
- public String getKey(WC wc) { return wc.word; }
- });
-{% endhighlight %}
-
-
-
-{% highlight scala %}
-// some ordinary case class
-case class WC(word: String, count: Int)
-val words: DataStream[WC] = // [...]
-val keyed = words.keyBy( _.word )
-{% endhighlight %}
-
-
-
-{% top %}
-
-Specifying Transformation Functions
---------------------------
-
-Most transformations require user-defined functions. This section lists different ways
-of how they can be specified
-
-
-
-
-#### Implementing an interface
-
-The most basic way is to implement one of the provided interfaces:
-
-{% highlight java %}
-class MyMapFunction implements MapFunction {
- public Integer map(String value) { return Integer.parseInt(value); }
-};
-data.map(new MyMapFunction());
-{% endhighlight %}
-
-#### Anonymous classes
-
-You can pass a function as an anonymous class:
-{% highlight java %}
-data.map(new MapFunction () {
- public Integer map(String value) { return Integer.parseInt(value); }
-});
-{% endhighlight %}
-
-#### Java 8 Lambdas
-
-Flink also supports Java 8 Lambdas in the Java API.
-
-{% highlight java %}
-data.filter(s -> s.startsWith("http://"));
-{% endhighlight %}
-
-{% highlight java %}
-data.reduce((i1,i2) -> i1 + i2);
-{% endhighlight %}
-
-#### Rich functions
-
-All transformations that require a user-defined function can
-instead take as argument a *rich* function. For example, instead of
-
-{% highlight java %}
-class MyMapFunction implements MapFunction {
- public Integer map(String value) { return Integer.parseInt(value); }
-};
-{% endhighlight %}
-
-you can write
-
-{% highlight java %}
-class MyMapFunction extends RichMapFunction {
- public Integer map(String value) { return Integer.parseInt(value); }
-};
-{% endhighlight %}
-
-and pass the function as usual to a `map` transformation:
-
-{% highlight java %}
-data.map(new MyMapFunction());
-{% endhighlight %}
-
-Rich functions can also be defined as an anonymous class:
-{% highlight java %}
-data.map (new RichMapFunction() {
- public Integer map(String value) { return Integer.parseInt(value); }
-});
-{% endhighlight %}
-
-
-
-
-
-#### Lambda Functions
-
-As already seen in previous examples all operations accept lambda functions for describing
-the operation:
-{% highlight scala %}
-val data: DataSet[String] = // [...]
-data.filter { _.startsWith("http://") }
-{% endhighlight %}
-
-{% highlight scala %}
-val data: DataSet[Int] = // [...]
-data.reduce { (i1,i2) => i1 + i2 }
-// or
-data.reduce { _ + _ }
-{% endhighlight %}
-
-#### Rich functions
-
-All transformations that take as argument a lambda function can
-instead take as argument a *rich* function. For example, instead of
-
-{% highlight scala %}
-data.map { x => x.toInt }
-{% endhighlight %}
-
-you can write
-
-{% highlight scala %}
-class MyMapFunction extends RichMapFunction[String, Int] {
- def map(in: String):Int = { in.toInt }
-};
-{% endhighlight %}
-
-and pass the function to a `map` transformation:
-
-{% highlight scala %}
-data.map(new MyMapFunction())
-{% endhighlight %}
-
-Rich functions can also be defined as an anonymous class:
-{% highlight scala %}
-data.map (new RichMapFunction[String, Int] {
- def map(in: String):Int = { in.toInt }
-})
-{% endhighlight %}
-
-
-
-
-Rich functions provide, in addition to the user-defined function (map,
-reduce, etc), four methods: `open`, `close`, `getRuntimeContext`, and
-`setRuntimeContext`. These are useful for parameterizing the function
-(see [Passing Parameters to Functions]({{ site.baseurl }}/dev/batch/index.html#passing-parameters-to-functions)),
-creating and finalizing local state, accessing broadcast variables (see
-[Broadcast Variables]({{ site.baseurl }}/dev/batch/index.html#broadcast-variables)), and for accessing runtime
-information such as accumulators and counters (see
-[Accumulators and Counters](#accumulators--counters)), and information
-on iterations (see [Iterations]({{ site.baseurl }}/dev/batch/iterations.html)).
-
-{% top %}
-
-Supported Data Types
---------------------
-
-Flink places some restrictions on the type of elements that can be in a DataSet or DataStream.
-The reason for this is that the system analyzes the types to determine
-efficient execution strategies.
-
-There are seven different categories of data types:
-
-1. **Java Tuples** and **Scala Case Classes**
-2. **Java POJOs**
-3. **Primitive Types**
-4. **Regular Classes**
-5. **Values**
-6. **Hadoop Writables**
-7. **Special Types**
-
-#### Tuples and Case Classes
-
-
-
-
-Tuples are composite types that contain a fixed number of fields with various types.
-The Java API provides classes from `Tuple1` up to `Tuple25`. Every field of a tuple
-can be an arbitrary Flink type including further tuples, resulting in nested tuples. Fields of a
-tuple can be accessed directly using the field's name as `tuple.f4`, or using the generic getter method
-`tuple.getField(int position)`. The field indices start at 0. Note that this stands in contrast
-to the Scala tuples, but it is more consistent with Java's general indexing.
-
-{% highlight java %}
-DataStream> wordCounts = env.fromElements(
- new Tuple2("hello", 1),
- new Tuple2("world", 2));
-
-wordCounts.map(new MapFunction, Integer>() {
- @Override
- public Integer map(Tuple2 value) throws Exception {
- return value.f1;
- }
-});
-
-wordCounts.keyBy(0); // also valid .keyBy("f0")
-
-
-{% endhighlight %}
-
-
-
-
-Scala case classes (and Scala tuples which are a special case of case classes), are composite types that contain a fixed number of fields with various types. Tuple fields are addressed by their 1-offset names such as `_1` for the first field. Case class fields are accessed by their name.
-
-{% highlight scala %}
-case class WordCount(word: String, count: Int)
-val input = env.fromElements(
- WordCount("hello", 1),
- WordCount("world", 2)) // Case Class Data Set
-
-input.keyBy("word")// key by field expression "word"
-
-val input2 = env.fromElements(("hello", 1), ("world", 2)) // Tuple2 Data Set
-
-input2.keyBy(0, 1) // key by field positions 0 and 1
-{% endhighlight %}
-
-
-
-
-#### POJOs
-
-Java and Scala classes are treated by Flink as a special POJO data type if they fulfill the following requirements:
-
-- The class must be public.
-
-- It must have a public constructor without arguments (default constructor).
-
-- All fields are either public or must be accessible through getter and setter functions. For a field called `foo` the getter and setter methods must be named `getFoo()` and `setFoo()`.
-
-- The type of a field must be supported by a registered serializer.
-
-POJOs are generally represented with a `PojoTypeInfo` and serialized with the `PojoSerializer` (using [Kryo](https://github.com/EsotericSoftware/kryo) as configurable fallback).
-The exception is when the POJOs are actually Avro types (Avro Specific Records) or produced as "Avro Reflect Types".
-In that case the POJO's are represented by an `AvroTypeInfo` and serialized with the `AvroSerializer`.
-You can also register your own custom serializer if required; see [Serialization](https://ci.apache.org/projects/flink/flink-docs-stable/dev/types_serialization.html#serialization-of-pojo-types) for further information.
-
-Flink analyzes the structure of POJO types, i.e., it learns about the fields of a POJO. As a result POJO types are easier to use than general types. Moreover, Flink can process POJOs more efficiently than general types.
-
-The following example shows a simple POJO with two public fields.
-
-
-
-{% highlight java %}
-public class WordWithCount {
-
- public String word;
- public int count;
-
- public WordWithCount() {}
-
- public WordWithCount(String word, int count) {
- this.word = word;
- this.count = count;
- }
-}
-
-DataStream wordCounts = env.fromElements(
- new WordWithCount("hello", 1),
- new WordWithCount("world", 2));
-
-wordCounts.keyBy("word"); // key by field expression "word"
-
-{% endhighlight %}
-
-
-{% highlight scala %}
-class WordWithCount(var word: String, var count: Int) {
- def this() {
- this(null, -1)
- }
-}
-
-val input = env.fromElements(
- new WordWithCount("hello", 1),
- new WordWithCount("world", 2)) // Case Class Data Set
-
-input.keyBy("word")// key by field expression "word"
-
-{% endhighlight %}
-
-
-
-#### Primitive Types
-
-Flink supports all Java and Scala primitive types such as `Integer`, `String`, and `Double`.
-
-#### General Class Types
-
-Flink supports most Java and Scala classes (API and custom).
-Restrictions apply to classes containing fields that cannot be serialized, like file pointers, I/O streams, or other native
-resources. Classes that follow the Java Beans conventions work well in general.
-
-All classes that are not identified as POJO types (see POJO requirements above) are handled by Flink as general class types.
-Flink treats these data types as black boxes and is not able to access their content (e.g., for efficient sorting). General types are de/serialized using the serialization framework [Kryo](https://github.com/EsotericSoftware/kryo).
-
-#### Values
-
-*Value* types describe their serialization and deserialization manually. Instead of going through a
-general purpose serialization framework, they provide custom code for those operations by means of
-implementing the `org.apache.flinktypes.Value` interface with the methods `read` and `write`. Using
-a Value type is reasonable when general purpose serialization would be highly inefficient. An
-example would be a data type that implements a sparse vector of elements as an array. Knowing that
-the array is mostly zero, one can use a special encoding for the non-zero elements, while the
-general purpose serialization would simply write all array elements.
-
-The `org.apache.flinktypes.CopyableValue` interface supports manual internal cloning logic in a
-similar way.
-
-Flink comes with pre-defined Value types that correspond to basic data types. (`ByteValue`,
-`ShortValue`, `IntValue`, `LongValue`, `FloatValue`, `DoubleValue`, `StringValue`, `CharValue`,
-`BooleanValue`). These Value types act as mutable variants of the basic data types: Their value can
-be altered, allowing programmers to reuse objects and take pressure off the garbage collector.
-
-
-#### Hadoop Writables
-
-You can use types that implement the `org.apache.hadoop.Writable` interface. The serialization logic
-defined in the `write()`and `readFields()` methods will be used for serialization.
-
-#### Special Types
-
-You can use special types, including Scala's `Either`, `Option`, and `Try`.
-The Java API has its own custom implementation of `Either`.
-Similarly to Scala's `Either`, it represents a value of two possible types, *Left* or *Right*.
-`Either` can be useful for error handling or operators that need to output two different types of records.
-
-#### Type Erasure & Type Inference
-
-*Note: This Section is only relevant for Java.*
-
-The Java compiler throws away much of the generic type information after compilation. This is
-known as *type erasure* in Java. It means that at runtime, an instance of an object does not know
-its generic type any more. For example, instances of `DataStream` and `DataStream` look the
-same to the JVM.
-
-Flink requires type information at the time when it prepares the program for execution (when the
-main method of the program is called). The Flink Java API tries to reconstruct the type information
-that was thrown away in various ways and store it explicitly in the data sets and operators. You can
-retrieve the type via `DataStream.getType()`. The method returns an instance of `TypeInformation`,
-which is Flink's internal way of representing types.
-
-The type inference has its limits and needs the "cooperation" of the programmer in some cases.
-Examples for that are methods that create data sets from collections, such as
-`ExecutionEnvironment.fromCollection(),` where you can pass an argument that describes the type. But
-also generic functions like `MapFunction` may need extra type information.
-
-The
-{% gh_link /flink-core/src/main/java/org/apache/flink/api/java/typeutils/ResultTypeQueryable.java "ResultTypeQueryable" %}
-interface can be implemented by input formats and functions to tell the API
-explicitly about their return type. The *input types* that the functions are invoked with can
-usually be inferred by the result types of the previous operations.
-
-{% top %}
-
-Accumulators & Counters
----------------------------
-
-Accumulators are simple constructs with an **add operation** and a **final accumulated result**,
-which is available after the job ended.
-
-The most straightforward accumulator is a **counter**: You can increment it using the
-```Accumulator.add(V value)``` method. At the end of the job Flink will sum up (merge) all partial
-results and send the result to the client. Accumulators are useful during debugging or if you
-quickly want to find out more about your data.
-
-Flink currently has the following **built-in accumulators**. Each of them implements the
-{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java "Accumulator" %}
-interface.
-
-- {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/IntCounter.java "__IntCounter__" %},
- {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongCounter.java "__LongCounter__" %}
- and {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/DoubleCounter.java "__DoubleCounter__" %}:
- See below for an example using a counter.
-- {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java "__Histogram__" %}:
- A histogram implementation for a discrete number of bins. Internally it is just a map from Integer
- to Integer. You can use this to compute distributions of values, e.g. the distribution of
- words-per-line for a word count program.
-
-__How to use accumulators:__
-
-First you have to create an accumulator object (here a counter) in the user-defined transformation
-function where you want to use it.
-
-{% highlight java %}
-private IntCounter numLines = new IntCounter();
-{% endhighlight %}
-
-Second you have to register the accumulator object, typically in the ```open()``` method of the
-*rich* function. Here you also define the name.
-
-{% highlight java %}
-getRuntimeContext().addAccumulator("num-lines", this.numLines);
-{% endhighlight %}
-
-You can now use the accumulator anywhere in the operator function, including in the ```open()``` and
-```close()``` methods.
-
-{% highlight java %}
-this.numLines.add(1);
-{% endhighlight %}
-
-The overall result will be stored in the ```JobExecutionResult``` object which is
-returned from the `execute()` method of the execution environment
-(currently this only works if the execution waits for the
-completion of the job).
-
-{% highlight java %}
-myJobExecutionResult.getAccumulatorResult("num-lines")
-{% endhighlight %}
-
-All accumulators share a single namespace per job. Thus you can use the same accumulator in
-different operator functions of your job. Flink will internally merge all accumulators with the same
-name.
-
-A note on accumulators and iterations: Currently the result of accumulators is only available after
-the overall job has ended. We plan to also make the result of the previous iteration available in the
-next iteration. You can use
-{% gh_link /flink-java/src/main/java/org/apache/flink/api/java/operators/IterativeDataSet.java#L98 "Aggregators" %}
-to compute per-iteration statistics and base the termination of iterations on such statistics.
-
-__Custom accumulators:__
-
-To implement your own accumulator you simply have to write your implementation of the Accumulator
-interface. Feel free to create a pull request if you think your custom accumulator should be shipped
-with Flink.
-
-You have the choice to implement either
-{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java "Accumulator" %}
-or {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/SimpleAccumulator.java "SimpleAccumulator" %}.
-
-```Accumulator``` is most flexible: It defines a type ```V``` for the value to add, and a
-result type ```R``` for the final result. E.g. for a histogram, ```V``` is a number and ```R``` is
- a histogram. ```SimpleAccumulator``` is for the cases where both types are the same, e.g. for counters.
-
-{% top %}
diff --git a/docs/dev/application_parameters.md b/docs/dev/application_parameters.md
new file mode 100644
index 0000000000000000000000000000000000000000..be6e6ec05b6e93ee2895f4d41842b0993e7e19c2
--- /dev/null
+++ b/docs/dev/application_parameters.md
@@ -0,0 +1,140 @@
+---
+title: "Handling Application Parameters"
+nav-id: application_parameters
+nav-show_overview: true
+nav-parent_id: streaming
+nav-pos: 50
+---
+
+
+* This will be replaced by the TOC
+{:toc}
+
+Handling Application Parameters
+-------------------------------
+Almost all Flink applications, both batch and streaming, rely on external configuration parameters.
+They are used to specify input and output sources (like paths or addresses), system parameters (parallelism, runtime configuration), and application specific parameters (typically used within user functions).
+
+Flink provides a simple utility called `ParameterTool` to provide some basic tooling for solving these problems.
+Please note that you don't have to use the `ParameterTool` described here. Other frameworks such as [Commons CLI](https://commons.apache.org/proper/commons-cli/) and
+[argparse4j](http://argparse4j.sourceforge.net/) also work well with Flink.
+
+
+### Getting your configuration values into the `ParameterTool`
+
+The `ParameterTool` provides a set of predefined static methods for reading the configuration. The tool is internally expecting a `Map`, so it's very easy to integrate it with your own configuration style.
+
+
+#### From `.properties` files
+
+The following method will read a [Properties](https://docs.oracle.com/javase/tutorial/essential/environment/properties.html) file and provide the key/value pairs:
+{% highlight java %}
+String propertiesFilePath = "/home/sam/flink/myjob.properties";
+ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFilePath);
+
+File propertiesFile = new File(propertiesFilePath);
+ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFile);
+
+InputStream propertiesFileInputStream = new FileInputStream(file);
+ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFileInputStream);
+{% endhighlight %}
+
+
+#### From the command line arguments
+
+This allows getting arguments like `--input hdfs:///mydata --elements 42` from the command line.
+{% highlight java %}
+public static void main(String[] args) {
+ ParameterTool parameter = ParameterTool.fromArgs(args);
+ // .. regular code ..
+{% endhighlight %}
+
+
+#### From system properties
+
+When starting a JVM, you can pass system properties to it: `-Dinput=hdfs:///mydata`. You can also initialize the `ParameterTool` from these system properties:
+
+{% highlight java %}
+ParameterTool parameter = ParameterTool.fromSystemProperties();
+{% endhighlight %}
+
+
+### Using the parameters in your Flink program
+
+Now that we've got the parameters from somewhere (see above) we can use them in various ways.
+
+**Directly from the `ParameterTool`**
+
+The `ParameterTool` itself has methods for accessing the values.
+{% highlight java %}
+ParameterTool parameters = // ...
+parameter.getRequired("input");
+parameter.get("output", "myDefaultValue");
+parameter.getLong("expectedCount", -1L);
+parameter.getNumberOfParameters()
+// .. there are more methods available.
+{% endhighlight %}
+
+You can use the return values of these methods directly in the `main()` method of the client submitting the application.
+For example, you could set the parallelism of a operator like this:
+
+{% highlight java %}
+ParameterTool parameters = ParameterTool.fromArgs(args);
+int parallelism = parameters.get("mapParallelism", 2);
+DataSet> counts = text.flatMap(new Tokenizer()).setParallelism(parallelism);
+{% endhighlight %}
+
+Since the `ParameterTool` is serializable, you can pass it to the functions itself:
+
+{% highlight java %}
+ParameterTool parameters = ParameterTool.fromArgs(args);
+DataSet> counts = text.flatMap(new Tokenizer(parameters));
+{% endhighlight %}
+
+and then use it inside the function for getting values from the command line.
+
+#### Register the parameters globally
+
+Parameters registered as global job parameters in the `ExecutionConfig` can be accessed as configuration values from the JobManager web interface and in all functions defined by the user.
+
+Register the parameters globally:
+
+{% highlight java %}
+ParameterTool parameters = ParameterTool.fromArgs(args);
+
+// set up the execution environment
+final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+env.getConfig().setGlobalJobParameters(parameters);
+{% endhighlight %}
+
+Access them in any rich user function:
+
+{% highlight java %}
+public static final class Tokenizer extends RichFlatMapFunction> {
+
+ @Override
+ public void flatMap(String value, Collector> out) {
+ ParameterTool parameters = (ParameterTool)
+ getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+ parameters.getRequired("input");
+ // .. do more ..
+{% endhighlight %}
+
+{% top %}
diff --git a/docs/dev/application_parameters.zh.md b/docs/dev/application_parameters.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..be6e6ec05b6e93ee2895f4d41842b0993e7e19c2
--- /dev/null
+++ b/docs/dev/application_parameters.zh.md
@@ -0,0 +1,140 @@
+---
+title: "Handling Application Parameters"
+nav-id: application_parameters
+nav-show_overview: true
+nav-parent_id: streaming
+nav-pos: 50
+---
+
+
+* This will be replaced by the TOC
+{:toc}
+
+Handling Application Parameters
+-------------------------------
+Almost all Flink applications, both batch and streaming, rely on external configuration parameters.
+They are used to specify input and output sources (like paths or addresses), system parameters (parallelism, runtime configuration), and application specific parameters (typically used within user functions).
+
+Flink provides a simple utility called `ParameterTool` to provide some basic tooling for solving these problems.
+Please note that you don't have to use the `ParameterTool` described here. Other frameworks such as [Commons CLI](https://commons.apache.org/proper/commons-cli/) and
+[argparse4j](http://argparse4j.sourceforge.net/) also work well with Flink.
+
+
+### Getting your configuration values into the `ParameterTool`
+
+The `ParameterTool` provides a set of predefined static methods for reading the configuration. The tool is internally expecting a `Map`, so it's very easy to integrate it with your own configuration style.
+
+
+#### From `.properties` files
+
+The following method will read a [Properties](https://docs.oracle.com/javase/tutorial/essential/environment/properties.html) file and provide the key/value pairs:
+{% highlight java %}
+String propertiesFilePath = "/home/sam/flink/myjob.properties";
+ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFilePath);
+
+File propertiesFile = new File(propertiesFilePath);
+ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFile);
+
+InputStream propertiesFileInputStream = new FileInputStream(file);
+ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFileInputStream);
+{% endhighlight %}
+
+
+#### From the command line arguments
+
+This allows getting arguments like `--input hdfs:///mydata --elements 42` from the command line.
+{% highlight java %}
+public static void main(String[] args) {
+ ParameterTool parameter = ParameterTool.fromArgs(args);
+ // .. regular code ..
+{% endhighlight %}
+
+
+#### From system properties
+
+When starting a JVM, you can pass system properties to it: `-Dinput=hdfs:///mydata`. You can also initialize the `ParameterTool` from these system properties:
+
+{% highlight java %}
+ParameterTool parameter = ParameterTool.fromSystemProperties();
+{% endhighlight %}
+
+
+### Using the parameters in your Flink program
+
+Now that we've got the parameters from somewhere (see above) we can use them in various ways.
+
+**Directly from the `ParameterTool`**
+
+The `ParameterTool` itself has methods for accessing the values.
+{% highlight java %}
+ParameterTool parameters = // ...
+parameter.getRequired("input");
+parameter.get("output", "myDefaultValue");
+parameter.getLong("expectedCount", -1L);
+parameter.getNumberOfParameters()
+// .. there are more methods available.
+{% endhighlight %}
+
+You can use the return values of these methods directly in the `main()` method of the client submitting the application.
+For example, you could set the parallelism of a operator like this:
+
+{% highlight java %}
+ParameterTool parameters = ParameterTool.fromArgs(args);
+int parallelism = parameters.get("mapParallelism", 2);
+DataSet> counts = text.flatMap(new Tokenizer()).setParallelism(parallelism);
+{% endhighlight %}
+
+Since the `ParameterTool` is serializable, you can pass it to the functions itself:
+
+{% highlight java %}
+ParameterTool parameters = ParameterTool.fromArgs(args);
+DataSet> counts = text.flatMap(new Tokenizer(parameters));
+{% endhighlight %}
+
+and then use it inside the function for getting values from the command line.
+
+#### Register the parameters globally
+
+Parameters registered as global job parameters in the `ExecutionConfig` can be accessed as configuration values from the JobManager web interface and in all functions defined by the user.
+
+Register the parameters globally:
+
+{% highlight java %}
+ParameterTool parameters = ParameterTool.fromArgs(args);
+
+// set up the execution environment
+final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+env.getConfig().setGlobalJobParameters(parameters);
+{% endhighlight %}
+
+Access them in any rich user function:
+
+{% highlight java %}
+public static final class Tokenizer extends RichFlatMapFunction> {
+
+ @Override
+ public void flatMap(String value, Collector> out) {
+ ParameterTool parameters = (ParameterTool)
+ getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+ parameters.getRequired("input");
+ // .. do more ..
+{% endhighlight %}
+
+{% top %}
diff --git a/docs/dev/batch/connectors.md b/docs/dev/batch/connectors.md
index 6a6fd79f04a3d82c6941cdf06acb64429f275c13..a66ba9d4951c740cf923e8fb6effbd1b43be49ed 100644
--- a/docs/dev/batch/connectors.md
+++ b/docs/dev/batch/connectors.md
@@ -1,7 +1,7 @@
---
-title: "Connectors"
-nav-parent_id: batch
-nav-pos: 4
+title: "DataSet Connectors"
+nav-parent_id: connectors-root
+nav-pos: 10
---
-Flink's fault tolerance mechanism recovers programs in the presence of failures and
-continues to execute them. Such failures include machine hardware failures, network failures,
-transient program failures, etc.
+当程序出现错误的时候,Flink 的容错机制能恢复并继续运行程序。这种错误包括机器硬件故障、网络故障、瞬态程序故障等等。
-Flink can guarantee exactly-once state updates to user-defined state only when the source participates in the
-snapshotting mechanism. The following table lists the state update guarantees of Flink coupled with the bundled connectors.
+只有当 source 参与了快照机制的时候,Flink 才能保证对自定义状态的精确一次更新。下表列举了 Flink 与其自带连接器的状态更新的保证。
-Please read the documentation of each connector to understand the details of the fault tolerance guarantees.
+请阅读各个连接器的文档来了解容错保证的细节。
@@ -43,50 +40,48 @@ Please read the documentation of each connector to understand the details of the
Apache Kafka
-
exactly once
-
Use the appropriate Kafka connector for your version
+
精确一次
+
根据你的版本用恰当的 Kafka 连接器
AWS Kinesis Streams
-
exactly once
+
精确一次
RabbitMQ
-
at most once (v 0.10) / exactly once (v 1.0)
+
至多一次 (v 0.10) / 精确一次 (v 1.0)
Twitter Streaming API
-
at most once
+
至多一次
Google PubSub
-
at least once
+
至少一次
Collections
-
exactly once
+
精确一次
Files
-
exactly once
+
精确一次
Sockets
-
at most once
+
至多一次
-To guarantee end-to-end exactly-once record delivery (in addition to exactly-once state semantics), the data sink needs
-to take part in the checkpointing mechanism. The following table lists the delivery guarantees (assuming exactly-once
-state updates) of Flink coupled with bundled sinks:
+为了保证端到端精确一次的数据交付(在精确一次的状态语义上更进一步),sink需要参与 checkpointing 机制。下表列举了 Flink 与其自带 sink 的交付保证(假设精确一次状态更新)。
@@ -99,47 +94,47 @@ state updates) of Flink coupled with bundled sinks:
HDFS BucketingSink
-
exactly once
-
Implementation depends on Hadoop version
+
精确一次
+
实现方法取决于 Hadoop 的版本
Elasticsearch
-
at least once
+
至少一次
Kafka producer
-
at least once / exactly once
-
exactly once with transactional producers (v 0.11+)
+
至少一次 / 精确一次
+
当使用事务生产者时,保证精确一次 (v 0.11+)
Cassandra sink
-
at least once / exactly once
-
exactly once only for idempotent updates
+
至少一次 / 精确一次
+
只有当更新是幂等时,保证精确一次
AWS Kinesis Streams
-
at least once
+
至少一次
File sinks
-
exactly once
+
精确一次
Socket sinks
-
at least once
+
至少一次
Standard output
-
at least once
+
至少一次
Redis sink
-
at least once
+
至少一次
diff --git a/docs/dev/connectors/index.md b/docs/dev/connectors/index.md
index 61320148ba606f6a99ab742f97ba77d7e3ff490c..6ef88e8cf0b24c8a6e8c46bf1bd4a8aa258a9482 100644
--- a/docs/dev/connectors/index.md
+++ b/docs/dev/connectors/index.md
@@ -1,9 +1,9 @@
---
-title: "Streaming Connectors"
+title: "DataStream Connectors"
nav-id: connectors
-nav-title: Connectors
-nav-parent_id: streaming
-nav-pos: 30
+nav-title: DataStream Connectors
+nav-parent_id: connectors-root
+nav-pos: 1
nav-show_overview: true
---
+
+* This will be replaced by the TOC
+{:toc}
+
+
+---
+title: "JDBC Connector"
+nav-title: JDBC
+nav-parent_id: connectors
+nav-pos: 9
+---
+
+
+* This will be replaced by the TOC
+{:toc}
+
+
+This connector provides a sink that writes data to a JDBC database.
+
+To use it, add the following dependency to your project (along with your JDBC-driver):
+
+{% highlight xml %}
+
+ org.apache.flink
+ flink-jdbc{{ site.scala_version_suffix }}
+ {{site.version }}
+
+{% endhighlight %}
+
+Note that the streaming connectors are currently __NOT__ part of the binary distribution. See how to link with them for cluster execution [here]({{ site.baseurl}}/dev/projectsetup/dependencies.html).
+
+Created JDBC sink provides at-least-once guarantee.
+Effectively exactly-once can be achived using upsert statements or idempotent updates.
+
+Example usage:
+{% highlight java %}
+StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+env
+ .fromElements(...)
+ .addSink(JdbcFacade.sink(
+ "insert into books (id, title, author, price, qty) values (?,?,?,?,?)",
+ (ps, t) -> {
+ ps.setInt(1, t.id);
+ ps.setString(2, t.title);
+ ps.setString(3, t.author);
+ ps.setDouble(4, t.price);
+ ps.setInt(5, t.qty);
+ },
+ new JdbcConnectionOptions.JdbcConnectionOptionsBuilder()
+ .withUrl(getDbMetadata().getUrl())
+ .withDriverName(getDbMetadata().getDriverClass())
+ .build()));
+env.execute();
+{% endhighlight %}
+
+Please refer to the [API documentation]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/api/java/io/jdbc/JdbcSink.html) for more details.
diff --git a/docs/dev/connectors/jdbc.zh.md b/docs/dev/connectors/jdbc.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..6dc261a4e51e4e663a4f927a5e276cab038f959c
--- /dev/null
+++ b/docs/dev/connectors/jdbc.zh.md
@@ -0,0 +1,97 @@
+---
+title: "JDBC Connector"
+nav-title: JDBC
+nav-parent_id: connectors
+nav-pos: 9
+---
+
+
+* This will be replaced by the TOC
+{:toc}
+
+
+---
+title: "JDBC Connector"
+nav-title: JDBC
+nav-parent_id: connectors
+nav-pos: 9
+---
+
+
+* This will be replaced by the TOC
+{:toc}
+
+
+This connector provides a sink that writes data to a JDBC database.
+
+To use it, add the following dependency to your project (along with your JDBC-driver):
+
+{% highlight xml %}
+
+ org.apache.flink
+ flink-jdbc{{ site.scala_version_suffix }}
+ {{site.version }}
+
+{% endhighlight %}
+
+Note that the streaming connectors are currently __NOT__ part of the binary distribution. See how to link with them for cluster execution [here]({{ site.baseurl}}/dev/projectsetup/dependencies.html).
+
+Created JDBC sink provides at-least-once guarantee.
+Effectively exactly-once can be achived using upsert statements or idempotent updates.
+
+Example usage:
+{% highlight java %}
+StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+env
+ .fromElements(...)
+ .addSink(JdbcFacade.sink(
+ "insert into books (id, title, author, price, qty) values (?,?,?,?,?)",
+ (ps, t) -> {
+ ps.setInt(1, t.id);
+ ps.setString(2, t.title);
+ ps.setString(3, t.author);
+ ps.setDouble(4, t.price);
+ ps.setInt(5, t.qty);
+ },
+ new JdbcConnectionOptions.JdbcConnectionOptionsBuilder()
+ .withUrl(getDbMetadata().getUrl())
+ .withDriverName(getDbMetadata().getDriverClass())
+ .build()));
+env.execute();
+{% endhighlight %}
+
+Please refer to the [API documentation]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/api/java/io/jdbc/JdbcSink.html) for more details.
diff --git a/docs/dev/connectors/pubsub.md b/docs/dev/connectors/pubsub.md
index 59b3e0096be8cfa4a2fa8378efd14bf4f0263b31..6055d9f67ae90e4338e3d3887216856e7f5de13a 100644
--- a/docs/dev/connectors/pubsub.md
+++ b/docs/dev/connectors/pubsub.md
@@ -143,13 +143,13 @@ env.addSource(pubsubSource)
-### Atleast once guarantee
+### At least once guarantee
#### SourceFunction
There are several reasons why a message might be send multiple times, such as failure scenarios on Google PubSub's side.
-Another reason is when the acknowledgement deadline has passed. This is the time between receiving the message and between acknowledging the message. The PubSubSource will only acknowledge a message on successful checkpoints to guarantee Atleast-Once. This does mean if the time between successful checkpoints is larger than the acknowledgment deadline of your subscription messages will most likely be processed multiple times.
+Another reason is when the acknowledgement deadline has passed. This is the time between receiving the message and acknowledging the message. The PubSubSource will only acknowledge a message on successful checkpoints to guarantee at-least-once. This does mean if the time between successful checkpoints is larger than the acknowledgment deadline of your subscription messages will most likely be processed multiple times.
For this reason it's recommended to have a (much) lower checkpoint interval than acknowledgement deadline.
diff --git a/docs/dev/connectors/pubsub.zh.md b/docs/dev/connectors/pubsub.zh.md
index 59b3e0096be8cfa4a2fa8378efd14bf4f0263b31..eaf5f582c8467cf0cc000cdd431054c949cc4c39 100644
--- a/docs/dev/connectors/pubsub.zh.md
+++ b/docs/dev/connectors/pubsub.zh.md
@@ -23,9 +23,7 @@ specific language governing permissions and limitations
under the License.
-->
-This connector provides a Source and Sink that can read from and write to
-[Google Cloud PubSub](https://cloud.google.com/pubsub). To use this connector, add the
-following dependency to your project:
+这个连接器可向 [Google Cloud PubSub](https://cloud.google.com/pubsub) 读取与写入数据。添加下面的依赖来使用此连接器:
{% highlight xml %}
@@ -36,27 +34,20 @@ following dependency to your project:
{% endhighlight %}
-Note: This connector has been added to Flink recently. It has not received widespread testing yet.
+注意:此连接器最近才加到 Flink 里,还未接受广泛测试。
-Note that the streaming connectors are currently not part of the binary
-distribution. See
-[here]({{ site.baseurl }}/dev/projectsetup/dependencies.html)
-for information about how to package the program with the libraries for
-cluster execution.
-
-
+注意连接器目前还不是二进制发行版的一部分,添加依赖、打包配置以及集群运行信息请参考[这里]({{ site.baseurl }}/zh/dev/projectsetup/dependencies.html)
## Consuming or Producing PubSubMessages
-The connector provides a connectors for receiving and sending messages from and to Google PubSub.
-Google PubSub has an `at-least-once` guarantee and as such the connector delivers the same guarantees.
+连接器可以接收和发送 Google PubSub 的信息。和 Google PubSub 一样,这个连接器能够保证`至少一次`的语义。
### PubSub SourceFunction
-The class `PubSubSource` has a builder to create PubSubsources: `PubSubSource.newBuilder(...)`
+`PubSubSource` 类的对象由构建类来构建: `PubSubSource.newBuilder(...)`
-There are several optional methods to alter how the PubSubSource is created, the bare minimum is to provide a Google project, Pubsub subscription and a way to deserialize the PubSubMessages.
+有多种可选的方法来创建 PubSubSource,但最低要求是要提供 Google Project、Pubsub 订阅和反序列化 PubSubMessages 的方法。
Example:
@@ -77,13 +68,13 @@ streamExecEnv.addSource(source);
-Currently the source functions [pulls](https://cloud.google.com/pubsub/docs/pull) messages from PubSub, [push endpoints](https://cloud.google.com/pubsub/docs/push) are not supported.
+当前还不支持 PubSub 的 source functions [pulls](https://cloud.google.com/pubsub/docs/pull) messages 和 [push endpoints](https://cloud.google.com/pubsub/docs/push)。
### PubSub Sink
-The class `PubSubSink` has a builder to create PubSubSinks. `PubSubSink.newBuilder(...)`
+`PubSubSink` 类的对象由构建类来构建: `PubSubSink.newBuilder(...)`
-This builder works in a similar way to the PubSubSource.
+构建类的使用方式与 PubSubSource 类似。
Example:
@@ -106,17 +97,18 @@ dataStream.addSink(pubsubSink);
### Google Credentials
-Google uses [Credentials](https://cloud.google.com/docs/authentication/production) to authenticate and authorize applications so that they can use Google Cloud Platform resources (such as PubSub).
+应用程序需要使用 [Credentials](https://cloud.google.com/docs/authentication/production) 来通过认证和授权才能使用 Google Cloud Platform 的资源,例如 PubSub。
+
+上述的两个构建类都允许你提供 Credentials, 但是连接器默认会通过环境变量: [GOOGLE_APPLICATION_CREDENTIALS](https://cloud.google.com/docs/authentication/production#obtaining_and_providing_service_account_credentials_manually) 来获取 Credentials 的路径。
-Both builders allow you to provide these credentials but by default the connectors will look for an environment variable: [GOOGLE_APPLICATION_CREDENTIALS](https://cloud.google.com/docs/authentication/production#obtaining_and_providing_service_account_credentials_manually) which should point to a file containing the credentials.
+如果你想手动提供 Credentials,例如你想从外部系统读取 Credentials,你可以使用 `PubSubSource.newBuilder(...).withCredentials(...)`。
-If you want to provide Credentials manually, for instance if you read the Credentials yourself from an external system, you can use `PubSubSource.newBuilder(...).withCredentials(...)`.
+### 集成测试
-### Integration testing
+在集成测试的时候,如果你不想直接连 PubSub 而是想读取和写入一个 docker container,可以参照 [PubSub testing locally](https://cloud.google.com/pubsub/docs/emulator)。
-When running integration tests you might not want to connect to PubSub directly but use a docker container to read and write to. (See: [PubSub testing locally](https://cloud.google.com/pubsub/docs/emulator))
+下面的例子展示了如何使用 source 来从仿真器读取信息并发送回去:
-The following example shows how you would create a source to read messages from the emulator and send them back:
-### Atleast once guarantee
+### 至少一次语义保证
#### SourceFunction
-There are several reasons why a message might be send multiple times, such as failure scenarios on Google PubSub's side.
+有很多原因导致会一个信息会被多次发出,例如 Google PubSub 的故障。
-Another reason is when the acknowledgement deadline has passed. This is the time between receiving the message and between acknowledging the message. The PubSubSource will only acknowledge a message on successful checkpoints to guarantee Atleast-Once. This does mean if the time between successful checkpoints is larger than the acknowledgment deadline of your subscription messages will most likely be processed multiple times.
+另一个可能的原因是超过了确认的截止时间,即收到与确认信息之间的时间间隔。PubSubSource 只有在信息被成功快照之后才会确认以保证至少一次的语义。这意味着,如果你的快照间隔大于信息确认的截止时间,那么你订阅的信息很有可能会被多次处理。
-For this reason it's recommended to have a (much) lower checkpoint interval than acknowledgement deadline.
+因此,我们建议把快照的间隔设置得比信息确认截止时间更短。
-See [PubSub](https://cloud.google.com/pubsub/docs/subscriber) for details on how to increase the acknowledgment deadline of your subscription.
+参照 [PubSub](https://cloud.google.com/pubsub/docs/subscriber) 来增加信息确认截止时间。
-Note: The metric `PubSubMessagesProcessedNotAcked` shows how many messages are waiting for the next checkpoint before they will be acknowledged.
+注意: `PubSubMessagesProcessedNotAcked` 显示了有多少信息正在等待下一个 checkpoint 还没被确认。
#### SinkFunction
-The sink function buffers messages that are to be send to PubSub for a short amount of time for performance reasons. Before each checkpoint this buffer is flushed and the checkpoint will not succeed unless the messages have been delivered to PubSub.
+Sink function 会把准备发到 PubSub 的信息短暂地缓存以提高性能。每次 checkpoint 前,它会刷新缓冲区,并且只有当所有信息成功发送到 PubSub 之后,checkpoint 才会成功完成。
{% top %}
diff --git a/docs/dev/connectors/streamfile_sink.md b/docs/dev/connectors/streamfile_sink.md
index bf25182179ee04bc108283e8b4022c193c1b3fc0..9fe047e0e7015ae637406de3f684fa35cca5f945 100644
--- a/docs/dev/connectors/streamfile_sink.md
+++ b/docs/dev/connectors/streamfile_sink.md
@@ -67,7 +67,7 @@ Row-encoded formats need to specify an [Encoder]({{ site.javadocs_baseurl }}/api
In addition to the bucket assigner the [RowFormatBuilder]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.RowFormatBuilder.html) allows the user to specify:
- - Custom [RollingPolicy]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy.html) : Rolling polciy to override the DefaultRollingPolicy
+ - Custom [RollingPolicy]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicy.html) : Rolling policy to override the DefaultRollingPolicy
- bucketCheckInterval (default = 1 min) : Millisecond interval for checking time based rolling policies
Basic usage for writing String elements thus looks like this:
@@ -84,7 +84,7 @@ import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.
DataStream input = ...;
final StreamingFileSink sink = StreamingFileSink
- .forRowFormat(new Path(outputPath), new SimpleStringEncoder<>("UTF-8"))
+ .forRowFormat(new Path(outputPath), new SimpleStringEncoder("UTF-8"))
.withRollingPolicy(
DefaultRollingPolicy.builder()
.withRolloverInterval(TimeUnit.MINUTES.toMillis(15))
diff --git a/docs/dev/connectors/streamfile_sink.zh.md b/docs/dev/connectors/streamfile_sink.zh.md
index fdf678e34e2cd24ca97937df837f8b613411081e..bd74bef5290533cc0f02b94227e8b10c6d6c1f1d 100644
--- a/docs/dev/connectors/streamfile_sink.zh.md
+++ b/docs/dev/connectors/streamfile_sink.zh.md
@@ -73,7 +73,7 @@ import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.
DataStream input = ...;
final StreamingFileSink sink = StreamingFileSink
- .forRowFormat(new Path(outputPath), new SimpleStringEncoder<>("UTF-8"))
+ .forRowFormat(new Path(outputPath), new SimpleStringEncoder("UTF-8"))
.withRollingPolicy(
DefaultRollingPolicy.builder()
.withRolloverInterval(TimeUnit.MINUTES.toMillis(15))
diff --git a/docs/dev/datastream_api.md b/docs/dev/datastream_api.md
index cb19977734e740b687ae02c68bd7dcfc578d5b55..0d59d63dfa66b2c466373ef3eb0aed1245dad9b3 100644
--- a/docs/dev/datastream_api.md
+++ b/docs/dev/datastream_api.md
@@ -1,6 +1,6 @@
---
title: "Flink DataStream API Programming Guide"
-nav-title: Streaming (DataStream API)
+nav-title: DataStream API
nav-id: streaming
nav-parent_id: dev
nav-show_overview: true
@@ -32,19 +32,221 @@ example write the data to files, or to standard output (for example the command
terminal). Flink programs run in a variety of contexts, standalone, or embedded in other programs.
The execution can happen in a local JVM, or on clusters of many machines.
-Please see [basic concepts]({{ site.baseurl }}/dev/api_concepts.html) for an introduction
-to the basic concepts of the Flink API.
-
-In order to create your own Flink DataStream program, we encourage you to start with
-[anatomy of a Flink Program]({{ site.baseurl }}/dev/api_concepts.html#anatomy-of-a-flink-program)
-and gradually add your own
-[stream transformations]({{ site.baseurl }}/dev/stream/operators/index.html). The remaining sections act as references for additional
-operations and advanced features.
+In order to create your own Flink DataStream program, we encourage you to start
+with [anatomy of a Flink Program](#anatomy-of-a-flink-program) and gradually
+add your own [stream transformations]({{ site.baseurl
+}}/dev/stream/operators/index.html). The remaining sections act as references
+for additional operations and advanced features.
* This will be replaced by the TOC
{:toc}
+What is a DataStream?
+----------------------
+
+The DataStream API gets its name from the special `DataStream` class that is
+used to represent a collection of data in a Flink program. You can think of
+them as immutable collections of data that can contain duplicates. This data
+can either be finite or unbounded, the API that you use to work on them is the
+same.
+
+A `DataStream` is similar to a regular Java `Collection` in terms of usage but
+is quite different in some key ways. They are immutable, meaning that once they
+are created you cannot add or remove elements. You can also not simply inspect
+the elements inside but only work on them using the `DataStream` API
+operations, which are also called transformations.
+
+You can create an initial `DataStream` by adding a source in a Flink program.
+Then you can derive new streams from this and combine them by using API methods
+such as `map`, `filter`, and so on.
+
+Anatomy of a Flink Program
+--------------------------
+
+Flink programs look like regular programs that transform `DataStreams`. Each
+program consists of the same basic parts:
+
+1. Obtain an `execution environment`,
+2. Load/create the initial data,
+3. Specify transformations on this data,
+4. Specify where to put the results of your computations,
+5. Trigger the program execution
+
+
+
+
+
+
+We will now give an overview of each of those steps, please refer to the
+respective sections for more details. Note that all core classes of the Java
+DataStream API can be found in {% gh_link
+/flink-streaming-java/src/main/java/org/apache/flink/streaming/api
+"org.apache.flink.streaming.api" %}.
+
+The `StreamExecutionEnvironment` is the basis for all Flink programs. You can
+obtain one using these static methods on `StreamExecutionEnvironment`:
+
+{% highlight java %}
+getExecutionEnvironment()
+
+createLocalEnvironment()
+
+createRemoteEnvironment(String host, int port, String... jarFiles)
+{% endhighlight %}
+
+Typically, you only need to use `getExecutionEnvironment()`, since this will do
+the right thing depending on the context: if you are executing your program
+inside an IDE or as a regular Java program it will create a local environment
+that will execute your program on your local machine. If you created a JAR file
+from your program, and invoke it through the [command line]({{ site.baseurl
+}}/ops/cli.html), the Flink cluster manager will execute your main method and
+`getExecutionEnvironment()` will return an execution environment for executing
+your program on a cluster.
+
+For specifying data sources the execution environment has several methods to
+read from files using various methods: you can just read them line by line, as
+CSV files, or using any of the other provided sources. To just read a text file
+as a sequence of lines, you can use:
+
+{% highlight java %}
+final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+DataStream text = env.readTextFile("file:///path/to/file");
+{% endhighlight %}
+
+This will give you a DataStream on which you can then apply transformations to create new
+derived DataStreams.
+
+You apply transformations by calling methods on DataStream with a
+transformation functions. For example, a map transformation looks like this:
+
+{% highlight java %}
+DataStream input = ...;
+
+DataStream parsed = input.map(new MapFunction() {
+ @Override
+ public Integer map(String value) {
+ return Integer.parseInt(value);
+ }
+});
+{% endhighlight %}
+
+This will create a new DataStream by converting every String in the original
+collection to an Integer.
+
+Once you have a DataStream containing your final results, you can write it to
+an outside system by creating a sink. These are just some example methods for
+creating a sink:
+
+{% highlight java %}
+writeAsText(String path)
+
+print()
+{% endhighlight %}
+
+
+
+
+We will now give an overview of each of those steps, please refer to the
+respective sections for more details. Note that all core classes of the Scala
+DataStream API can be found in {% gh_link
+/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala
+"org.apache.flink.streaming.api.scala" %}.
+
+The `StreamExecutionEnvironment` is the basis for all Flink programs. You can
+obtain one using these static methods on `StreamExecutionEnvironment`:
+
+{% highlight scala %}
+getExecutionEnvironment()
+
+createLocalEnvironment()
+
+createRemoteEnvironment(host: String, port: Int, jarFiles: String*)
+{% endhighlight %}
+
+Typically, you only need to use `getExecutionEnvironment()`, since this will do
+the right thing depending on the context: if you are executing your program
+inside an IDE or as a regular Java program it will create a local environment
+that will execute your program on your local machine. If you created a JAR file
+from your program, and invoke it through the [command line]({{ site.baseurl
+}}/ops/cli.html), the Flink cluster manager will execute your main method and
+`getExecutionEnvironment()` will return an execution environment for executing
+your program on a cluster.
+
+For specifying data sources the execution environment has several methods to
+read from files using various methods: you can just read them line by line, as
+CSV files, or using any of the other provided sources. To just read a text file
+as a sequence of lines, you can use:
+
+{% highlight scala %}
+val env = StreamExecutionEnvironment.getExecutionEnvironment()
+
+val text: DataStream[String] = env.readTextFile("file:///path/to/file")
+{% endhighlight %}
+
+This will give you a DataStream on which you can then apply transformations to
+create new derived DataStreams.
+
+You apply transformations by calling methods on DataStream with a
+transformation functions. For example, a map transformation looks like this:
+
+{% highlight scala %}
+val input: DataSet[String] = ...
+
+val mapped = input.map { x => x.toInt }
+{% endhighlight %}
+
+This will create a new DataStream by converting every String in the original
+collection to an Integer.
+
+Once you have a DataStream containing your final results, you can write it to
+an outside system by creating a sink. These are just some example methods for
+creating a sink:
+
+{% highlight scala %}
+writeAsText(path: String)
+
+print()
+{% endhighlight %}
+
+
+
+
+Once you specified the complete program you need to **trigger the program
+execution** by calling `execute()` on the `StreamExecutionEnvironment`.
+Depending on the type of the `ExecutionEnvironment` the execution will be
+triggered on your local machine or submit your program for execution on a
+cluster.
+
+The `execute()` method will wait for the job to finish and then return a
+`JobExecutionResult`, this contains execution times and accumulator results.
+
+If you don't want to wait for the job to finish, you can trigger asynchronous
+job execution by calling `executeAysnc()` on the `StreamExecutionEnvironment`.
+It will return a `JobClient` with which you can communicate with the job you
+just submitted. For instance, here is how to implement the semantics of
+`execute()` by using `executeAsync()`.
+
+{% highlight java %}
+final JobClient jobClient = env.executeAsync();
+
+final JobExecutionResult jobExecutionResult = jobClient.getJobExecutionResult(userClassloader).get();
+{% endhighlight %}
+
+That last part about program execution is crucial to understanding when and how
+Flink operations are executed. All Flink programs are executed lazily: When the
+program's main method is executed, the data loading and transformations do not
+happen directly. Rather, each operation is created and added to a dataflow
+graph. The operations are actually executed when the execution is explicitly
+triggered by an `execute()` call on the execution environment. Whether the
+program is executed locally or on a cluster depends on the type of execution
+environment
+
+The lazy evaluation lets you construct sophisticated programs that Flink
+executes as one holistically planned unit.
+
+{% top %}
Example Program
---------------
diff --git a/docs/dev/datastream_api.zh.md b/docs/dev/datastream_api.zh.md
index ed1afd83b7a04e8e14a33f1f4f189d58c7397e2a..c95a9101dcd01086d02070cbd7ff488c56f80b2b 100644
--- a/docs/dev/datastream_api.zh.md
+++ b/docs/dev/datastream_api.zh.md
@@ -1,6 +1,6 @@
---
title: "Flink DataStream API 编程指南"
-nav-title: Streaming (DataStream API)
+nav-title: DataStream API
nav-id: streaming
nav-parent_id: dev
nav-show_overview: true
diff --git a/docs/dev/event_time.md b/docs/dev/event_time.md
index 83cfd7459a8400bbdef3b9320d9e158741c26028..a25122f1bdee3a3950b09f76994ff1726d4e1a76 100644
--- a/docs/dev/event_time.md
+++ b/docs/dev/event_time.md
@@ -25,13 +25,13 @@ under the License.
-->
In this section you will learn about writing time-aware Flink programs. Please
-take a look at [Timely Stream Processing]({{site.baseurl}}{% link
+take a look at [Timely Stream Processing]({% link
concepts/timely-stream-processing.md %}) to learn about the concepts behind
timely stream processing.
For information about how to use time in Flink programs refer to
-[windowing]({{site.baseurl}}{% link dev/stream/operators/windows.md %}) and
-[ProcessFunction]({{ site.baseurl }}{% link
+[windowing]({% link dev/stream/operators/windows.md %}) and
+[ProcessFunction]({% link
dev/stream/operators/process_function.md %}).
* toc
diff --git a/docs/dev/event_time.zh.md b/docs/dev/event_time.zh.md
index fcd5a14ebb803f1caa427ae42b9f97597c55972c..dbe964fee2fb9cae10f54bc6c329b78ec47cdd1c 100644
--- a/docs/dev/event_time.zh.md
+++ b/docs/dev/event_time.zh.md
@@ -24,66 +24,20 @@ specific language governing permissions and limitations
under the License.
-->
-* toc
-{:toc}
-
-# Event Time / Processing Time / Ingestion Time
-
-Flink supports different notions of *time* in streaming programs.
-
-- **Processing time:** Processing time refers to the system time of the machine that is executing the
- respective operation.
-
- When a streaming program runs on processing time, all time-based operations (like time windows) will
- use the system clock of the machines that run the respective operator. An hourly
- processing time window will include all records that arrived at a specific operator between the
- times when the system clock indicated the full hour. For example, if an application
- begins running at 9:15am, the first hourly processing time window will include events
- processed between 9:15am and 10:00am, the next window will include events processed between 10:00am and 11:00am, and so on.
-
- Processing time is the simplest notion of time and requires no coordination between streams and machines.
- It provides the best performance and the lowest latency. However, in distributed and asynchronous
- environments processing time does not provide determinism, because it is susceptible to the speed at which
- records arrive in the system (for example from the message queue), to the speed at which the
- records flow between operators inside the system, and to outages (scheduled, or otherwise).
-
-- **Event time:** Event time is the time that each individual event occurred on its producing device.
- This time is typically embedded within the records before they enter Flink, and that *event timestamp*
- can be extracted from each record. In event time, the progress of time depends on the data,
- not on any wall clocks. Event time programs must specify how to generate *Event Time Watermarks*,
- which is the mechanism that signals progress in event time. This watermarking mechanism is
- described in a later section, [below](#event-time-and-watermarks).
-
- In a perfect world, event time processing would yield completely consistent and deterministic results, regardless of when events arrive, or their ordering.
- However, unless the events are known to arrive in-order (by timestamp), event time processing incurs some latency while waiting for out-of-order events. As it is only possible to wait for a finite period of time, this places a limit on how deterministic event time applications can be.
-
- Assuming all of the data has arrived, event time operations will behave as expected, and produce correct and consistent results even when working with out-of-order or late events, or when reprocessing historic data. For example, an hourly event time window will contain all records
- that carry an event timestamp that falls into that hour, regardless of the order in which they arrive, or when they are processed. (See the section on [late events](#late-elements) for more information.)
-
-
+In this section you will learn about writing time-aware Flink programs. Please
+take a look at [Timely Stream Processing]({% link
+concepts/timely-stream-processing.zh.md %}) to learn about the concepts behind
+timely stream processing.
- Note that sometimes when event time programs are processing live data in real-time, they will use some *processing time* operations in order to guarantee that they are progressing in a timely fashion.
-
-- **Ingestion time:** Ingestion time is the time that events enter Flink. At the source operator each
- record gets the source's current time as a timestamp, and time-based operations (like time windows)
- refer to that timestamp.
-
- *Ingestion time* sits conceptually in between *event time* and *processing time*. Compared to
- *processing time*, it is slightly more expensive, but gives more predictable results. Because
- *ingestion time* uses stable timestamps (assigned once at the source), different window operations
- over the records will refer to the same timestamp, whereas in *processing time* each window operator
- may assign the record to a different window (based on the local system clock and any transport delay).
-
- Compared to *event time*, *ingestion time* programs cannot handle any out-of-order events or late data,
- but the programs don't have to specify how to generate *watermarks*.
-
- Internally, *ingestion time* is treated much like *event time*, but with automatic timestamp assignment and
- automatic watermark generation.
-
-
+For information about how to use time in Flink programs refer to
+[windowing]({% link dev/stream/operators/windows.zh.md %}) and
+[ProcessFunction]({% link
+dev/stream/operators/process_function.zh.md %}).
+* toc
+{:toc}
-### Setting a Time Characteristic
+## Setting a Time Characteristic
The first part of a Flink DataStream program usually sets the base *time characteristic*. That setting
defines how data stream sources behave (for example, whether they will assign timestamps), and what notion of
@@ -144,7 +98,6 @@ env.set_stream_time_characteristic(TimeCharacteristic.ProcessingTime)
-
Note that in order to run this example in *event time*, the program needs to either use sources
that directly define event time for the data and emit watermarks themselves, or the program must
inject a *Timestamp Assigner & Watermark Generator* after the sources. Those functions describe how to access
@@ -154,77 +107,7 @@ The section below describes the general mechanism behind *timestamps* and *water
to use timestamp assignment and watermark generation in the Flink DataStream API, please refer to
[Generating Timestamps / Watermarks]({{ site.baseurl }}/dev/event_timestamps_watermarks.html).
-
-# Event Time and Watermarks
-
-*Note: Flink implements many techniques from the Dataflow Model. For a good introduction to event time and watermarks, have a look at the articles below.*
-
- - [Streaming 101](https://www.oreilly.com/ideas/the-world-beyond-batch-streaming-101) by Tyler Akidau
- - The [Dataflow Model paper](https://research.google.com/pubs/archive/43864.pdf)
-
-
-A stream processor that supports *event time* needs a way to measure the progress of event time.
-For example, a window operator that builds hourly windows needs to be notified when event time has passed beyond the
-end of an hour, so that the operator can close the window in progress.
-
-*Event time* can progress independently of *processing time* (measured by wall clocks).
-For example, in one program the current *event time* of an operator may trail slightly behind the *processing time*
-(accounting for a delay in receiving the events), while both proceed at the same speed.
-On the other hand, another streaming program might progress through weeks of event time with only a few seconds of processing,
-by fast-forwarding through some historic data already buffered in a Kafka topic (or another message queue).
-
-------
-
-The mechanism in Flink to measure progress in event time is **watermarks**.
-Watermarks flow as part of the data stream and carry a timestamp *t*. A *Watermark(t)* declares that event time has reached time
-*t* in that stream, meaning that there should be no more elements from the stream with a timestamp *t' <= t* (i.e. events with timestamps
-older or equal to the watermark).
-
-The figure below shows a stream of events with (logical) timestamps, and watermarks flowing inline. In this example the events are in order
-(with respect to their timestamps), meaning that the watermarks are simply periodic markers in the stream.
-
-
-
-Watermarks are crucial for *out-of-order* streams, as illustrated below, where the events are not ordered by their timestamps.
-In general a watermark is a declaration that by that point in the stream, all events up to a certain timestamp should have arrived.
-Once a watermark reaches an operator, the operator can advance its internal *event time clock* to the value of the watermark.
-
-
-
-Note that event time is inherited by a freshly created stream element (or elements) from either the event that produced them or
-from watermark that triggered creation of those elements.
-
-## Watermarks in Parallel Streams
-
-Watermarks are generated at, or directly after, source functions. Each parallel subtask of a source function usually
-generates its watermarks independently. These watermarks define the event time at that particular parallel source.
-
-As the watermarks flow through the streaming program, they advance the event time at the operators where they arrive. Whenever an
-operator advances its event time, it generates a new watermark downstream for its successor operators.
-
-Some operators consume multiple input streams; a union, for example, or operators following a *keyBy(...)* or *partition(...)* function.
-Such an operator's current event time is the minimum of its input streams' event times. As its input streams
-update their event times, so does the operator.
-
-The figure below shows an example of events and watermarks flowing through parallel streams, and operators tracking event time.
-
-
-
-Note that the Kafka source supports per-partition watermarking, which you can read more about [here]({{ site.baseurl }}/dev/event_timestamps_watermarks.html#timestamps-per-kafka-partition).
-
-
-## Late Elements
-
-It is possible that certain elements will violate the watermark condition, meaning that even after the *Watermark(t)* has occurred,
-more elements with timestamp *t' <= t* will occur. In fact, in many real world setups, certain elements can be arbitrarily
-delayed, making it impossible to specify a time by which all elements of a certain event timestamp will have occurred.
-Furthermore, even if the lateness can be bounded, delaying the watermarks by too much is often not desirable, because it
-causes too much delay in the evaluation of event time windows.
-
-For this reason, streaming programs may explicitly expect some *late* elements. Late elements are elements that
-arrive after the system's event time clock (as signaled by the watermarks) has already passed the time of the late element's
-timestamp. See [Allowed Lateness]({{ site.baseurl }}/dev/stream/operators/windows.html#allowed-lateness) for more information on how to work
-with late elements in event time windows.
+{% top %}
## Idling sources
diff --git a/docs/dev/java_lambdas.md b/docs/dev/java_lambdas.md
index bc9704b517d0bed23994e6f954c2218a803e1694..accca2fdab25089e253a41f03691da1dc86cf4d7 100644
--- a/docs/dev/java_lambdas.md
+++ b/docs/dev/java_lambdas.md
@@ -1,7 +1,7 @@
---
title: "Java Lambda Expressions"
-nav-parent_id: api-concepts
-nav-pos: 20
+nav-parent_id: streaming
+nav-pos: 300
---
-FlinkCEP is the Complex Event Processing (CEP) library implemented on top of Flink.
-It allows you to detect event patterns in an endless stream of events, giving you the opportunity to get hold of what's important in your
-data.
+FlinkCEP是在Flink上层实现的复杂事件处理库。
+它可以让你在无限事件流中检测出特定的事件模型,有机会掌握数据中重要的那部分。
-This page describes the API calls available in Flink CEP. We start by presenting the [Pattern API](#the-pattern-api),
-which allows you to specify the patterns that you want to detect in your stream, before presenting how you can
-[detect and act upon matching event sequences](#detecting-patterns). We then present the assumptions the CEP
-library makes when [dealing with lateness](#handling-lateness-in-event-time) in event time and how you can
-[migrate your job](#migrating-from-an-older-flink-versionpre-13) from an older Flink version to Flink-1.3.
+本页讲述了Flink CEP中可用的API,我们首先讲述[模式API](#模式api),它可以让你指定想在数据流中检测的模式,然后讲述如何[检测匹配的事件序列并进行处理](#检测模式)。
+再然后我们讲述Flink在按照事件时间[处理迟到事件](#按照事件时间处理迟到事件)时的假设,
+以及如何从旧版本的Flink向1.3之后的版本[迁移作业](#从旧版本迁移13之前)。
* This will be replaced by the TOC
{:toc}
-## Getting Started
+## 开始
-If you want to jump right in, [set up a Flink program]({{ site.baseurl }}/dev/projectsetup/dependencies.html) and
-add the FlinkCEP dependency to the `pom.xml` of your project.
+如果你想现在开始尝试,[创建一个Flink程序]({{ site.baseurl }}/zh/dev/projectsetup/dependencies.html),
+添加FlinkCEP的依赖到项目的`pom.xml`文件中。
@@ -63,13 +60,12 @@ add the FlinkCEP dependency to the `pom.xml` of your project.
-{% info %} FlinkCEP is not part of the binary distribution. See how to link with it for cluster execution [here]({{site.baseurl}}/dev/projectsetup/dependencies.html).
+{% info 提示 %} FlinkCEP不是二进制发布包的一部分。在集群上执行如何链接它可以看[这里]({{site.baseurl}}/zh/dev/projectsetup/dependencies.html)。
-Now you can start writing your first CEP program using the Pattern API.
+现在可以开始使用Pattern API写你的第一个CEP程序了。
-{% warn Attention %} The events in the `DataStream` to which
-you want to apply pattern matching must implement proper `equals()` and `hashCode()` methods
-because FlinkCEP uses them for comparing and matching events.
+{% warn 注意 %} `DataStream`中的事件,如果你想在上面进行模式匹配的话,必须实现合适的 `equals()`和`hashCode()`方法,
+因为FlinkCEP使用它们来比较和匹配事件。
@@ -136,140 +132,143 @@ val result: DataStream[Alert] = patternStream.process(
-## The Pattern API
+## 模式API
-The pattern API allows you to define complex pattern sequences that you want to extract from your input stream.
+模式API可以让你定义想从输入流中抽取的复杂模式序列。
-Each complex pattern sequence consists of multiple simple patterns, i.e. patterns looking for individual events with the same properties. From now on, we will call these simple patterns **patterns**, and the final complex pattern sequence we are searching for in the stream, the **pattern sequence**. You can see a pattern sequence as a graph of such patterns, where transitions from one pattern to the next occur based on user-specified
-*conditions*, e.g. `event.getName().equals("end")`. A **match** is a sequence of input events which visits all
-patterns of the complex pattern graph, through a sequence of valid pattern transitions.
+每个复杂的模式序列包括多个简单的模式,比如,寻找拥有相同属性事件序列的模式。从现在开始,我们把这些简单的模式称作**模式**,
+把我们在数据流中最终寻找的复杂模式序列称作**模式序列**,你可以把模式序列看作是这样的模式构成的图,
+这些模式基于用户指定的**条件**从一个转换到另外一个,比如 `event.getName().equals("end")`。
+一个**匹配**是输入事件的一个序列,这些事件通过一系列有效的模式转换,能够访问到复杂模式图中的所有模式。
-{% warn Attention %} Each pattern must have a unique name, which you use later to identify the matched events.
+{% warn 注意 %} 每个模式必须有一个独一无二的名字,你可以在后面使用它来识别匹配到的事件。
-{% warn Attention %} Pattern names **CANNOT** contain the character `":"`.
+{% warn 注意 %} 模式的名字不能包含字符`":"`.
-In the rest of this section we will first describe how to define [Individual Patterns](#individual-patterns), and then how you can combine individual patterns into [Complex Patterns](#combining-patterns).
+这一节的剩余部分我们会先讲述如何定义[单个模式](#单个模式),然后讲如何将单个模式组合成[复杂模式](#组合模式)。
-### Individual Patterns
+### 单个模式
-A **Pattern** can be either a *singleton* or a *looping* pattern. Singleton patterns accept a single
-event, while looping patterns can accept more than one. In pattern matching symbols, the pattern `"a b+ c? d"` (or `"a"`, followed by *one or more* `"b"`'s, optionally followed by a `"c"`, followed by a `"d"`), `a`, `c?`, and `d` are
-singleton patterns, while `b+` is a looping one. By default, a pattern is a singleton pattern and you can transform
-it to a looping one by using [Quantifiers](#quantifiers). Each pattern can have one or more
-[Conditions](#conditions) based on which it accepts events.
+一个**模式**可以是一个**单例**或者**循环**模式。单例模式只接受一个事件,循环模式可以接受多个事件。
+在模式匹配表达式中,模式`"a b+ c? d"`(或者`"a"`,后面跟着一个或者多个`"b"`,再往后可选择的跟着一个`"c"`,最后跟着一个`"d"`),
+`a`,`c?`,和 `d`都是单例模式,`b+`是一个循环模式。默认情况下,模式都是单例的,你可以通过使用[量词](#量词)把它们转换成循环模式。
+每个模式可以有一个或者多个[条件](#条件)来决定它接受哪些事件。
-#### Quantifiers
+#### 量词
-In FlinkCEP, you can specify looping patterns using these methods: `pattern.oneOrMore()`, for patterns that expect one or more occurrences of a given event (e.g. the `b+` mentioned before); and `pattern.times(#ofTimes)`, for patterns that
-expect a specific number of occurrences of a given type of event, e.g. 4 `a`'s; and `pattern.times(#fromTimes, #toTimes)`, for patterns that expect a specific minimum number of occurrences and a maximum number of occurrences of a given type of event, e.g. 2-4 `a`s.
+在FlinkCEP中,你可以通过这些方法指定循环模式:`pattern.oneOrMore()`,指定期望一个给定事件出现一次或者多次的模式(例如前面提到的`b+`模式);
+`pattern.times(#ofTimes)`,指定期望一个给定事件出现特定次数的模式,例如出现4次`a`;
+`pattern.times(#fromTimes, #toTimes)`,指定期望一个给定事件出现次数在一个最小值和最大值中间的模式,比如出现2-4次`a`。
-You can make looping patterns greedy using the `pattern.greedy()` method, but you cannot yet make group patterns greedy. You can make all patterns, looping or not, optional using the `pattern.optional()` method.
+你可以使用`pattern.greedy()`方法让循环模式变成贪心的,但现在还不能让模式组贪心。
+你可以使用`pattern.optional()`方法让所有的模式变成可选的,不管是否是循环模式。
-For a pattern named `start`, the following are valid quantifiers:
+对一个命名为`start`的模式,以下量词是有效的:
{% highlight java %}
-// expecting 4 occurrences
+// 期望出现4次
start.times(4);
-// expecting 0 or 4 occurrences
+// 期望出现0或者4次
start.times(4).optional();
-// expecting 2, 3 or 4 occurrences
+// 期望出现2、3或者4次
start.times(2, 4);
-// expecting 2, 3 or 4 occurrences and repeating as many as possible
+// 期望出现2、3或者4次,并且尽可能的重复次数多
start.times(2, 4).greedy();
-// expecting 0, 2, 3 or 4 occurrences
+// 期望出现0、2、3或者4次
start.times(2, 4).optional();
-// expecting 0, 2, 3 or 4 occurrences and repeating as many as possible
+// 期望出现0、2、3或者4次,并且尽可能的重复次数多
start.times(2, 4).optional().greedy();
-// expecting 1 or more occurrences
+// 期望出现1到多次
start.oneOrMore();
-// expecting 1 or more occurrences and repeating as many as possible
+// 期望出现1到多次,并且尽可能的重复次数多
start.oneOrMore().greedy();
-// expecting 0 or more occurrences
+// 期望出现0到多次
start.oneOrMore().optional();
-// expecting 0 or more occurrences and repeating as many as possible
+// 期望出现0到多次,并且尽可能的重复次数多
start.oneOrMore().optional().greedy();
-// expecting 2 or more occurrences
+// 期望出现2到多次
start.timesOrMore(2);
-// expecting 2 or more occurrences and repeating as many as possible
+// 期望出现2到多次,并且尽可能的重复次数多
start.timesOrMore(2).greedy();
-// expecting 0, 2 or more occurrences and repeating as many as possible
+// 期望出现0、2或多次
+start.timesOrMore(2).optional();
+
+// 期望出现0、2或多次,并且尽可能的重复次数多
start.timesOrMore(2).optional().greedy();
{% endhighlight %}
{% highlight scala %}
-// expecting 4 occurrences
+// 期望出现4次
start.times(4)
-// expecting 0 or 4 occurrences
+// 期望出现0或者4次
start.times(4).optional()
-// expecting 2, 3 or 4 occurrences
+// 期望出现2、3或者4次
start.times(2, 4)
-// expecting 2, 3 or 4 occurrences and repeating as many as possible
+// 期望出现2、3或者4次,并且尽可能的重复次数多
start.times(2, 4).greedy()
-// expecting 0, 2, 3 or 4 occurrences
+// 期望出现0、2、3或者4次
start.times(2, 4).optional()
-// expecting 0, 2, 3 or 4 occurrences and repeating as many as possible
+// 期望出现0、2、3或者4次,并且尽可能的重复次数多
start.times(2, 4).optional().greedy()
-// expecting 1 or more occurrences
+// 期望出现1到多次
start.oneOrMore()
-// expecting 1 or more occurrences and repeating as many as possible
+// 期望出现1到多次,并且尽可能的重复次数多
start.oneOrMore().greedy()
-// expecting 0 or more occurrences
+// 期望出现0到多次
start.oneOrMore().optional()
-// expecting 0 or more occurrences and repeating as many as possible
+// 期望出现0到多次,并且尽可能的重复次数多
start.oneOrMore().optional().greedy()
-// expecting 2 or more occurrences
+// 期望出现2到多次
start.timesOrMore(2)
-// expecting 2 or more occurrences and repeating as many as possible
+// 期望出现2到多次,并且尽可能的重复次数多
start.timesOrMore(2).greedy()
-// expecting 0, 2 or more occurrences
+// 期望出现0、2或多次
start.timesOrMore(2).optional()
-// expecting 0, 2 or more occurrences and repeating as many as possible
+// 期望出现0、2或多次,并且尽可能的重复次数多
start.timesOrMore(2).optional().greedy()
{% endhighlight %}
-#### Conditions
+#### 条件
-For every pattern you can specify a condition that an incoming event has to meet in order to be "accepted" into the pattern e.g. its value should be larger than 5,
-or larger than the average value of the previously accepted events.
-You can specify conditions on the event properties via the `pattern.where()`, `pattern.or()` or `pattern.until()` methods.
-These can be either `IterativeCondition`s or `SimpleCondition`s.
+对每个模式你可以指定一个条件来决定一个进来的事件是否被接受进入这个模式,例如,它的value字段应该大于5,或者大于前面接受的事件的平均值。
+指定判断事件属性的条件可以通过`pattern.where()`、`pattern.or()`或者`pattern.until()`方法。
+这些可以是`IterativeCondition`或者`SimpleCondition`。
-**Iterative Conditions:** This is the most general type of condition. This is how you can specify a condition that
-accepts subsequent events based on properties of the previously accepted events or a statistic over a subset of them.
+**迭代条件:** 这是最普遍的条件类型。使用它可以指定一个基于前面已经被接受的事件的属性或者它们的一个子集的统计数据来决定是否接受时间序列的条件。
-Below is the code for an iterative condition that accepts the next event for a pattern named "middle" if its name starts
-with "foo", and if the sum of the prices of the previously accepted events for that pattern plus the price of the current event do not exceed the value of 5.0. Iterative conditions can be powerful, especially in combination with looping patterns, e.g. `oneOrMore()`.
+下面是一个迭代条件的代码,它接受"middle"模式下一个事件的名称开头是"foo", 并且前面已经匹配到的事件加上这个事件的价格小于5.0。
+迭代条件非常强大,尤其是跟循环模式结合使用时。
@@ -307,14 +306,12 @@ middle.oneOrMore()
-{% warn Attention %} The call to `ctx.getEventsForPattern(...)` finds all the
-previously accepted events for a given potential match. The cost of this operation can vary, so when implementing
-your condition, try to minimize its use.
+{% warn 注意 %} 调用`ctx.getEventsForPattern(...)`可以获得所有前面已经接受作为可能匹配的事件。
+调用这个操作的代价可能很小也可能很大,所以在实现你的条件时,尽量少使用它。
-Described context gives one access to event time characteristics as well. For more info see [Time context](#time-context).
+描述的上下文提供了获取事件时间属性的方法。更多细节可以看[时间上下文](#时间上下文)。
-**Simple Conditions:** This type of condition extends the aforementioned `IterativeCondition` class and decides
-whether to accept an event or not, based *only* on properties of the event itself.
+**简单条件:** 这种类型的条件扩展了前面提到的`IterativeCondition`类,它决定是否接受一个事件只取决于事件自身的属性。
-Finally, you can also restrict the type of the accepted event to a subtype of the initial event type (here `Event`)
-via the `pattern.subtype(subClass)` method.
+最后,你可以通过`pattern.subtype(subClass)`方法限制接受的事件类型是初始事件的子类型。
@@ -344,7 +340,7 @@ via the `pattern.subtype(subClass)` method.
start.subtype(SubEvent.class).where(new SimpleCondition() {
@Override
public boolean filter(SubEvent value) {
- return ... // some condition
+ return ... // 一些判断条件
}
});
{% endhighlight %}
@@ -352,12 +348,13 @@ start.subtype(SubEvent.class).where(new SimpleCondition() {
-**Combining Conditions:** As shown above, you can combine the `subtype` condition with additional conditions. This holds for every condition. You can arbitrarily combine conditions by sequentially calling `where()`. The final result will be the logical **AND** of the results of the individual conditions. To combine conditions using **OR**, you can use the `or()` method, as shown below.
+**组合条件:** 如上所示,你可以把`subtype`条件和其他的条件结合起来使用。这适用于任何条件,你可以通过依次调用`where()`来组合条件。
+最终的结果是每个单一条件的结果的逻辑**AND**。如果想使用**OR**来组合条件,你可以像下面这样使用`or()`方法。
-**Stop condition:** In case of looping patterns (`oneOrMore()` and `oneOrMore().optional()`) you can
-also specify a stop condition, e.g. accept events with value larger than 5 until the sum of values is smaller than 50.
+**停止条件:** 如果使用循环模式(`oneOrMore()`和`oneOrMore().optional()`),你可以指定一个停止条件,例如,接受事件的值大于5直到值的和小于50。
-To better understand it, have a look at the following example. Given
+为了更好的理解它,看下面的例子。给定
-* pattern like `"(a+ until b)"` (one or more `"a"` until `"b"`)
+* 模式如`"(a+ until b)"` (一个或者更多的`"a"`直到`"b"`)
-* a sequence of incoming events `"a1" "c" "a2" "b" "a3"`
+* 到来的事件序列`"a1" "c" "a2" "b" "a3"`
-* the library will output results: `{a1 a2} {a1} {a2} {a3}`.
+* 输出结果会是: `{a1 a2} {a1} {a2} {a3}`.
-As you can see `{a1 a2 a3}` or `{a2 a3}` are not returned due to the stop condition.
+你可以看到`{a1 a2 a3}`和`{a2 a3}`由于停止条件没有被输出。
-
Pattern Operation
-
Description
+
模式操作
+
描述
where(condition)
-
Defines a condition for the current pattern. To match the pattern, an event must satisfy the condition.
- Multiple consecutive where() clauses lead to their conditions being ANDed:
Specifies a stop condition for a looping pattern. Meaning if event matching the given condition occurs, no more
- events will be accepted into the pattern.
-
Applicable only in conjunction with oneOrMore()
-
NOTE: It allows for cleaning state for corresponding pattern on event-based condition.
Specifies that this pattern is greedy, i.e. it will repeat as many as possible. This is only applicable
- to quantifiers and it does not support group pattern currently.
Defines a condition for the current pattern. To match the pattern, an event must satisfy the condition.
- Multiple consecutive where() clauses lead to their conditions being ANDed:
Specifies a stop condition for looping pattern. Meaning if event matching the given condition occurs, no more
- events will be accepted into the pattern.
-
Applicable only in conjunction with oneOrMore()
-
NOTE: It allows for cleaning state for corresponding pattern on event-based condition.
Specifies that this pattern is greedy, i.e. it will repeat as many as possible. This is only applicable
- to quantifiers and it does not support group pattern currently.
+
指定这个模式是贪心的,也就是说,它会重复尽可能多的次数。这只对量词适用,现在还不支持模式组。
{% highlight scala %}
pattern.oneOrMore().greedy()
{% endhighlight %}
@@ -665,12 +647,11 @@ pattern.oneOrMore().greedy()
-### Combining Patterns
+### 组合模式
-Now that you've seen what an individual pattern can look like, it is time to see how to combine them
-into a full pattern sequence.
+现在你已经看到单个的模式是什么样的了,该去看看如何把它们连接起来组成一个完整的模式序列。
-A pattern sequence has to start with an initial pattern, as shown below:
+模式序列由一个初始模式作为开头,如下所示:
-Next, you can append more patterns to your pattern sequence by specifying the desired *contiguity conditions* between
-them. FlinkCEP supports the following forms of contiguity between events:
+接下来,你可以增加更多的模式到模式序列中并指定它们之间所需的*连续条件*。FlinkCEP支持事件之间如下形式的连续策略:
- 1. **Strict Contiguity**: Expects all matching events to appear strictly one after the other, without any non-matching events in-between.
+ 1. **严格连续**: 期望所有匹配的事件严格的一个接一个出现,中间没有任何不匹配的事件。
- 2. **Relaxed Contiguity**: Ignores non-matching events appearing in-between the matching ones.
+ 2. **松散连续**: 忽略匹配的事件之间的不匹配的事件。
- 3. **Non-Deterministic Relaxed Contiguity**: Further relaxes contiguity, allowing additional matches
- that ignore some matching events.
+ 3. **不确定的松散连续**: 更进一步的松散连续,允许忽略掉一些匹配事件的附加匹配。
-To apply them between consecutive patterns, you can use:
+可以使用下面的方法来指定模式之间的连续策略:
-1. `next()`, for *strict*,
-2. `followedBy()`, for *relaxed*, and
-3. `followedByAny()`, for *non-deterministic relaxed* contiguity.
+1. `next()`,指定*严格连续*,
+2. `followedBy()`,指定*松散连续*,
+3. `followedByAny()`,指定*不确定的松散*连续。
-or
+或者
-1. `notNext()`, if you do not want an event type to directly follow another
-2. `notFollowedBy()`, if you do not want an event type to be anywhere between two other event types.
+1. `notNext()`,如果不想后面直接连着一个特定事件
+2. `notFollowedBy()`,如果不想一个特定事件发生在两个事件之间的任何地方。
-{% warn Attention %} A pattern sequence cannot end in `notFollowedBy()`.
+{% warn 注意 %} 模式序列不能以`notFollowedBy()`结尾。
-{% warn Attention %} A `NOT` pattern cannot be preceded by an optional one.
+{% warn 注意 %} 一个`NOT`模式前面不能是可选的模式。
{% highlight scala %}
-// strict contiguity
+// 严格连续
val strict: Pattern[Event, _] = start.next("middle").where(...)
-// relaxed contiguity
+// 松散连续
val relaxed: Pattern[Event, _] = start.followedBy("middle").where(...)
-// non-deterministic relaxed contiguity
+// 不确定的松散连续
val nonDetermin: Pattern[Event, _] = start.followedByAny("middle").where(...)
-// NOT pattern with strict contiguity
+// 严格连续的NOT模式
val strictNot: Pattern[Event, _] = start.notNext("not").where(...)
-// NOT pattern with relaxed contiguity
+// 松散连续的NOT模式
val relaxedNot: Pattern[Event, _] = start.notFollowedBy("not").where(...)
{% endhighlight %}
-Relaxed contiguity means that only the first succeeding matching event will be matched, while
-with non-deterministic relaxed contiguity, multiple matches will be emitted for the same beginning. As an example,
-a pattern `"a b"`, given the event sequence `"a", "c", "b1", "b2"`, will give the following results:
+松散连续意味着跟着的事件中,只有第一个可匹配的事件会被匹配上,而不确定的松散连接情况下,有着同样起始的多个匹配会被输出。
+举例来说,模式`"a b"`,给定事件序列`"a","c","b1","b2"`,会产生如下的结果:
-1. Strict Contiguity between `"a"` and `"b"`: `{}` (no match), the `"c"` after `"a"` causes `"a"` to be discarded.
+1. `"a"`和`"b"`之间严格连续: `{}` (没有匹配),`"a"`之后的`"c"`导致`"a"`被丢弃。
-2. Relaxed Contiguity between `"a"` and `"b"`: `{a b1}`, as relaxed continuity is viewed as "skip non-matching events
-till the next matching one".
+2. `"a"`和`"b"`之间松散连续: `{a b1}`,松散连续会"跳过不匹配的事件直到匹配上的事件"。
-3. Non-Deterministic Relaxed Contiguity between `"a"` and `"b"`: `{a b1}`, `{a b2}`, as this is the most general form.
+3. `"a"`和`"b"`之间不确定的松散连续: `{a b1}`, `{a b2}`,这是最常见的情况。
-It's also possible to define a temporal constraint for the pattern to be valid.
-For example, you can define that a pattern should occur within 10 seconds via the `pattern.within()` method.
-Temporal patterns are supported for both [processing and event time]({{site.baseurl}}/dev/event_time.html).
+也可以为模式定义一个有效时间约束。
+例如,你可以通过`pattern.within()`方法指定一个模式应该在10秒内发生。
+这种时间模式支持[处理时间和事件时间]({{site.baseurl}}/zh/dev/event_time.html).
-{% warn Attention %} A pattern sequence can only have one temporal constraint. If multiple such constraints are defined on different individual patterns, then the smallest is applied.
+{% warn 注意 %} 一个模式序列只能有一个时间限制。如果限制了多个时间在不同的单个模式上,会使用最小的那个时间限制。
-#### Contiguity within looping patterns
+#### 循环模式中的连续性
-You can apply the same contiguity condition as discussed in the previous [section](#combining-patterns) within a looping pattern.
-The contiguity will be applied between elements accepted into such a pattern.
-To illustrate the above with an example, a pattern sequence `"a b+ c"` (`"a"` followed by any(non-deterministic relaxed) sequence of one or more `"b"`'s followed by a `"c"`) with
-input `"a", "b1", "d1", "b2", "d2", "b3" "c"` will have the following results:
+你可以在循环模式中使用和前面[章节](#组合模式)讲过的同样的连续性。
+连续性会被运用在被接受进入模式的事件之间。
+用这个例子来说明上面所说的连续性,一个模式序列`"a b+ c"`(`"a"`后面跟着一个或者多个(不确定连续的)`"b"`,然后跟着一个`"c"`)
+输入为`"a","b1","d1","b2","d2","b3","c"`,输出结果如下:
- 1. **Strict Contiguity**: `{a b3 c}` -- the `"d1"` after `"b1"` causes `"b1"` to be discarded, the same happens for `"b2"` because of `"d2"`.
+ 1. **严格连续**: `{a b3 c}` -- `"b1"`之后的`"d1"`导致`"b1"`被丢弃,同样`"b2"`因为`"d2"`被丢弃。
- 2. **Relaxed Contiguity**: `{a b1 c}`, `{a b1 b2 c}`, `{a b1 b2 b3 c}`, `{a b2 c}`, `{a b2 b3 c}`, `{a b3 c}` - `"d"`'s are ignored.
+ 2. **松散连续**: `{a b1 c}`,`{a b1 b2 c}`,`{a b1 b2 b3 c}`,`{a b2 c}`,`{a b2 b3 c}`,`{a b3 c}` - `"d"`都被忽略了。
- 3. **Non-Deterministic Relaxed Contiguity**: `{a b1 c}`, `{a b1 b2 c}`, `{a b1 b3 c}`, `{a b1 b2 b3 c}`, `{a b2 c}`, `{a b2 b3 c}`, `{a b3 c}` -
- notice the `{a b1 b3 c}`, which is the result of relaxing contiguity between `"b"`'s.
+ 3. **不确定松散连续**: `{a b1 c}`,`{a b1 b2 c}`,`{a b1 b3 c}`,`{a b1 b2 b3 c}`,`{a b2 c}`,`{a b2 b3 c}`,`{a b3 c}` -
+ 注意`{a b1 b3 c}`,这是因为`"b"`之间是不确定松散连续产生的。
-For looping patterns (e.g. `oneOrMore()` and `times()`) the default is *relaxed contiguity*. If you want
-strict contiguity, you have to explicitly specify it by using the `consecutive()` call, and if you want
-*non-deterministic relaxed contiguity* you can use the `allowCombinations()` call.
+对于循环模式(例如`oneOrMore()`和`times()`)),默认是*松散连续*。如果想使用*严格连续*,你需要使用`consecutive()`方法明确指定,
+如果想使用*不确定松散连续*,你可以使用`allowCombinations()`方法。
-
Pattern Operation
-
Description
+
模式操作
+
描述
consecutive()
-
Works in conjunction with oneOrMore() and times() and imposes strict contiguity between the matching
- events, i.e. any non-matching element breaks the match (as in next()).
-
If not applied a relaxed contiguity (as in followedBy()) is used.
Works in conjunction with oneOrMore() and times() and imposes non-deterministic relaxed contiguity
- between the matching events (as in followedByAny()).
-
If not applied a relaxed contiguity (as in followedBy()) is used.
Works in conjunction with oneOrMore() and times() and imposes strict contiguity between the matching
- events, i.e. any non-matching element breaks the match (as in next()).
-
If not applied a relaxed contiguity (as in followedBy()) is used.
Works in conjunction with oneOrMore() and times() and imposes non-deterministic relaxed contiguity
- between the matching events (as in followedByAny()).
-
If not applied a relaxed contiguity (as in followedBy()) is used.
@@ -942,12 +918,11 @@ Pattern.begin("start").where(_.getName().equals("c"))
-### Groups of patterns
+### 模式组
-It's also possible to define a pattern sequence as the condition for `begin`, `followedBy`, `followedByAny` and
-`next`. The pattern sequence will be considered as the matching condition logically and a `GroupPattern` will be
-returned and it is possible to apply `oneOrMore()`, `times(#ofTimes)`, `times(#fromTimes, #toTimes)`, `optional()`,
-`consecutive()`, `allowCombinations()` to the `GroupPattern`.
+也可以定义一个模式序列作为`begin`,`followedBy`,`followedByAny`和`next`的条件。这个模式序列在逻辑上会被当作匹配的条件,
+并且返回一个`GroupPattern`,可以在`GroupPattern`上使用`oneOrMore()`,`times(#ofTimes)`,
+`times(#fromTimes, #toTimes)`,`optional()`,`consecutive()`,`allowCombinations()`。
Appends a new pattern. Other events can occur between a matching event and the previous
- matching event, and alternative matches will be presented for every alternative matching event
- (non-deterministic relaxed contiguity):
Appends a new pattern. Other events can occur between a sequence of matching events and the previous
- matching event, and alternative matches will be presented for every alternative sequence of matching events
- (non-deterministic relaxed contiguity):
Appends a new negative pattern. A matching (negative) event has to directly succeed the
- previous matching event (strict contiguity) for the partial match to be discarded:
Appends a new negative pattern. A partial matching event sequence will be discarded even
- if other events occur between the matching (negative) event and the previous matching event
- (relaxed contiguity):
{% highlight scala %}
val start = Pattern.begin[Event]("start")
{% endhighlight %}
@@ -1157,7 +1123,7 @@ val start = Pattern.begin[Event]("start")
begin(#pattern_sequence)
-
Defines a starting pattern:
+
定一个开始模式:
{% highlight scala %}
val start = Pattern.begin(
Pattern.begin[Event]("start").where(...).followedBy("middle").where(...)
@@ -1168,8 +1134,7 @@ val start = Pattern.begin(
next(#name)
-
Appends a new pattern. A matching event has to directly succeed the previous matching event
- (strict contiguity):
+
增加一个新的模式,匹配的事件必须是直接跟在前面匹配到的事件后面(严格连续):
{% highlight scala %}
val next = start.next("middle")
{% endhighlight %}
@@ -1178,8 +1143,7 @@ val next = start.next("middle")
next(#pattern_sequence)
-
Appends a new pattern. A sequence of matching events have to directly succeed the previous matching event
- (strict contiguity):
+
增加一个新的模式。匹配的事件序列必须是直接跟在前面匹配到的事件后面(严格连续):
{% highlight scala %}
val next = start.next(
Pattern.begin[Event]("start").where(...).followedBy("middle").where(...)
@@ -1190,8 +1154,7 @@ val next = start.next(
followedBy(#name)
-
Appends a new pattern. Other events can occur between a matching event and the previous
- matching event (relaxed contiguity) :
+
增加一个新的模式。可以有其他事件出现在匹配的事件和之前匹配到的事件中间(松散连续):
{% highlight scala %}
val followedBy = start.followedBy("middle")
{% endhighlight %}
@@ -1200,8 +1163,7 @@ val followedBy = start.followedBy("middle")
followedBy(#pattern_sequence)
-
Appends a new pattern. Other events can occur between a sequence of matching events and the previous
- matching event (relaxed contiguity) :
+
增加一个新的模式。可以有其他事件出现在匹配的事件和之前匹配到的事件中间(松散连续):
{% highlight scala %}
val followedBy = start.followedBy(
Pattern.begin[Event]("start").where(...).followedBy("middle").where(...)
@@ -1212,9 +1174,8 @@ val followedBy = start.followedBy(
followedByAny(#name)
-
Appends a new pattern. Other events can occur between a matching event and the previous
- matching event, and alternative matches will be presented for every alternative matching event
- (non-deterministic relaxed contiguity):
{% highlight scala %}
val followedByAny = start.followedByAny("middle")
{% endhighlight %}
@@ -1223,9 +1184,8 @@ val followedByAny = start.followedByAny("middle")
followedByAny(#pattern_sequence)
-
Appends a new pattern. Other events can occur between a sequence of matching events and the previous
- matching event, and alternative matches will be presented for every alternative sequence of matching events
- (non-deterministic relaxed contiguity):
{% highlight scala %}
val followedByAny = start.followedByAny(
Pattern.begin[Event]("start").where(...).followedBy("middle").where(...)
@@ -1237,8 +1197,8 @@ val followedByAny = start.followedByAny(
notNext()
-
Appends a new negative pattern. A matching (negative) event has to directly succeed the
- previous matching event (strict contiguity) for the partial match to be discarded:
{% highlight scala %}
val notNext = start.notNext("not")
{% endhighlight %}
@@ -1247,9 +1207,8 @@ val notNext = start.notNext("not")
notFollowedBy()
-
Appends a new negative pattern. A partial matching event sequence will be discarded even
- if other events occur between the matching (negative) event and the previous matching event
- (relaxed contiguity):
{% highlight scala %}
val notFollowedBy = start.notFollowedBy("not")
{% endhighlight %}
@@ -1259,8 +1218,7 @@ val notFollowedBy = start.notFollowedBy("not")
within(time)
-
Defines the maximum time interval for an event sequence to match the pattern. If a non-completed event
- sequence exceeds this time, it is discarded:
+
定义匹配模式的事件序列出现的最大时间间隔。如果未完成的事件序列超过了这个事件,就会被丢弃:
{% highlight scala %}
pattern.within(Time.seconds(10))
{% endhighlight %}
@@ -1272,25 +1230,26 @@ pattern.within(Time.seconds(10))
-### After Match Skip Strategy
+### 匹配后跳过策略
-For a given pattern, the same event may be assigned to multiple successful matches. To control to how many matches an event will be assigned, you need to specify the skip strategy called `AfterMatchSkipStrategy`. There are five types of skip strategies, listed as follows:
+对于一个给定的模式,同一个事件可能会分配到多个成功的匹配上。为了控制一个事件会分配到多少个匹配上,你需要指定跳过策略`AfterMatchSkipStrategy`。
+有五种跳过策略,如下:
-* *NO_SKIP*: Every possible match will be emitted.
-* *SKIP_TO_NEXT*: Discards every partial match that started with the same event, emitted match was started.
-* *SKIP_PAST_LAST_EVENT*: Discards every partial match that started after the match started but before it ended.
-* *SKIP_TO_FIRST*: Discards every partial match that started after the match started but before the first event of *PatternName* occurred.
-* *SKIP_TO_LAST*: Discards every partial match that started after the match started but before the last event of *PatternName* occurred.
+* *NO_SKIP*: 每个成功的匹配都会被输出。
+* *SKIP_TO_NEXT*: 丢弃以相同事件开始的所有部分匹配。
+* *SKIP_PAST_LAST_EVENT*: 丢弃起始在这个匹配的开始和结束之间的所有部分匹配。
+* *SKIP_TO_FIRST*: 丢弃起始在这个匹配的开始和第一个出现的名称为*PatternName*事件之间的所有部分匹配。
+* *SKIP_TO_LAST*: 丢弃起始在这个匹配的开始和最后一个出现的名称为*PatternName*事件之间的所有部分匹配。
-Notice that when using *SKIP_TO_FIRST* and *SKIP_TO_LAST* skip strategy, a valid *PatternName* should also be specified.
+注意当使用*SKIP_TO_FIRST*和*SKIP_TO_LAST*策略时,需要指定一个合法的*PatternName*.
-For example, for a given pattern `b+ c` and a data stream `b1 b2 b3 c`, the differences between these four skip strategies are as follows:
+例如,给定一个模式`b+ c`和一个数据流`b1 b2 b3 c`,不同跳过策略之间的不同如下:
-
Skip Strategy
-
Result
-
Description
+
跳过策略
+
结果
+
描述
NO_SKIP
@@ -1299,7 +1258,7 @@ For example, for a given pattern `b+ c` and a data stream `b1 b2 b3 c`, the diff
b2 b3 c b3 c
-
After found matching b1 b2 b3 c, the match process will not discard any result.
+
找到匹配b1 b2 b3 c之后,不会丢弃任何结果。
SKIP_TO_NEXT
@@ -1308,14 +1267,14 @@ For example, for a given pattern `b+ c` and a data stream `b1 b2 b3 c`, the diff
b2 b3 c b3 c
-
After found matching b1 b2 b3 c, the match process will not discard any result, because no other match could start at b1.
+
找到匹配b1 b2 b3 c之后,不会丢弃任何结果,因为没有以b1开始的其他匹配。
SKIP_PAST_LAST_EVENT
b1 b2 b3 c
-
After found matching b1 b2 b3 c, the match process will discard all started partial matches.
+
找到匹配b1 b2 b3 c之后,会丢弃其他所有的部分匹配。
SKIP_TO_FIRST[b]
@@ -1324,7 +1283,7 @@ For example, for a given pattern `b+ c` and a data stream `b1 b2 b3 c`, the diff
b2 b3 c b3 c
-
After found matching b1 b2 b3 c, the match process will try to discard all partial matches started before b1, but there are no such matches. Therefore nothing will be discarded.
-Have a look also at another example to better see the difference between NO_SKIP and SKIP_TO_FIRST:
-Pattern: `(a | b | c) (b | c) c+.greedy d` and sequence: `a b c1 c2 c3 d` Then the results will be:
+在看另外一个例子来说明NO_SKIP和SKIP_TO_FIRST之间的差别:
+模式: `(a | b | c) (b | c) c+.greedy d`,输入:`a b c1 c2 c3 d`,结果将会是:
-
Skip Strategy
-
Result
-
Description
+
跳过策略
+
结果
+
描述
NO_SKIP
@@ -1353,7 +1312,7 @@ Pattern: `(a | b | c) (b | c) c+.greedy d` and sequence: `a b c1 c2 c3 d` Then t
b c1 c2 c3 d c1 c2 c3 d
-
After found matching a b c1 c2 c3 d, the match process will not discard any result.
+
找到匹配a b c1 c2 c3 d之后,不会丢弃任何结果。
SKIP_TO_FIRST[c*]
@@ -1361,19 +1320,19 @@ Pattern: `(a | b | c) (b | c) c+.greedy d` and sequence: `a b c1 c2 c3 d` Then t
a b c1 c2 c3 d c1 c2 c3 d
-
After found matching a b c1 c2 c3 d, the match process will discard all partial matches started before c1. There is one such match b c1 c2 c3 d.
+
找到匹配a b c1 c2 c3 d之后,会丢弃所有在c1之前开始的部分匹配,有一个这样的b c1 c2 c3 d被丢弃。
-To better understand the difference between NO_SKIP and SKIP_TO_NEXT take a look at following example:
-Pattern: `a b+` and sequence: `a b1 b2 b3` Then the results will be:
+为了更好的理解NO_SKIP和SKIP_TO_NEXT之间的差别,看下面的例子:
+模式:`a b+`,输入:`a b1 b2 b3`,结果将会是:
-
Skip Strategy
-
Result
-
Description
+
跳过策略
+
结果
+
描述
NO_SKIP
@@ -1382,46 +1341,46 @@ Pattern: `a b+` and sequence: `a b1 b2 b3` Then the results will be:
a b1 b2 a b1 b2 b3
-
After found matching a b1, the match process will not discard any result.
+
找到匹配a b1之后,不会丢弃任何结果。
SKIP_TO_NEXT
a b1
-
After found matching a b1, the match process will discard all partial matches started at a. This means neither a b1 b2 nor a b1 b2 b3 could be generated.
-{% warn Attention %} For SKIP_TO_FIRST/LAST there are two options how to handle cases when there are no elements mapped to
-the specified variable. By default a NO_SKIP strategy will be used in this case. The other option is to throw exception in such situation.
-One can enable this option by:
+{% warn 注意 %} 使用SKIP_TO_FIRST/LAST时,有两个选项可以用来处理没有事件可以映射到对应的变量名上的情况。
+默认情况下会使用NO_SKIP策略,另外一个选项是抛出异常。
+可以使用如下的选项:
-## Detecting Patterns
+## 检测模式
-After specifying the pattern sequence you are looking for, it is time to apply it to your input stream to detect
-potential matches. To run a stream of events against your pattern sequence, you have to create a `PatternStream`.
-Given an input stream `input`, a pattern `pattern` and an optional comparator `comparator` used to sort events with the same timestamp in case of EventTime or that arrived at the same moment, you create the `PatternStream` by calling:
+在指定了要寻找的模式后,该把它们应用到输入流上来发现可能的匹配了。为了在事件流上运行你的模式,需要创建一个`PatternStream`。
+给定一个输入流`input`,一个模式`pattern`和一个可选的用来对使用事件时间时有同样时间戳或者同时到达的事件进行排序的比较器`comparator`,
+你可以通过调用如下方法来创建`PatternStream`:
-The input stream can be *keyed* or *non-keyed* depending on your use-case.
+输入流根据你的使用场景可以是*keyed*或者*non-keyed*。
-{% warn Attention %} Applying your pattern on a non-keyed stream will result in a job with parallelism equal to 1.
+{% warn 注意 %} 在*non-keyed*流上使用模式将会使你的作业并发度被设为1。
-### Selecting from Patterns
+### 从模式中选取
-Once you have obtained a `PatternStream` you can apply transformation to detected event sequences. The suggested way of doing that
-is by `PatternProcessFunction`.
+在获得到一个`PatternStream`之后,你可以应用各种转换来发现事件序列。推荐使用`PatternProcessFunction`。
-A `PatternProcessFunction` has a `processMatch` method which is called for each matching event sequence.
-It receives a match in the form of `Map>` where the key is the name of each pattern in your pattern
-sequence and the value is a list of all accepted events for that pattern (`IN` is the type of your input elements).
-The events for a given pattern are ordered by timestamp. The reason for returning a list of accepted events for each
-pattern is that when using looping patterns (e.g. `oneToMany()` and `times()`), more than one event may be accepted for a given pattern.
+`PatternProcessFunction`有一个`processMatch`的方法在每找到一个匹配的事件序列时都会被调用。
+它按照`Map>`的格式接收一个匹配,映射的键是你的模式序列中的每个模式的名称,值是被接受的事件列表(`IN`是输入事件的类型)。
+模式的输入事件按照时间戳进行排序。为每个模式返回一个接受的事件列表的原因是当使用循环模式(比如`oneToMany()`和`times()`)时,
+对一个模式会有不止一个事件被接受。
{% highlight java %}
class MyPatternProcessFunction extends PatternProcessFunction {
@@ -1509,18 +1466,17 @@ class MyPatternProcessFunction extends PatternProcessFunction
}
{% endhighlight %}
-The `PatternProcessFunction` gives access to a `Context` object. Thanks to it, one can access time related
-characteristics such as `currentProcessingTime` or `timestamp` of current match (which is the timestamp of the last element assigned to the match).
-For more info see [Time context](#time-context).
-Through this context one can also emit results to a [side-output]({{ site.baseurl }}/dev/stream/side_output.html).
+`PatternProcessFunction`可以访问`Context`对象。有了它之后,你可以访问时间属性,比如`currentProcessingTime`或者当前匹配的`timestamp`
+(最新分配到匹配上的事件的时间戳)。
+更多信息可以看[时间上下文](#时间上下文)。
+通过这个上下文也可以将结果输出到[侧输出]({{ site.baseurl }}/zh/dev/stream/side_output.html).
-#### Handling Timed Out Partial Patterns
+#### 处理超时的部分匹配
-Whenever a pattern has a window length attached via the `within` keyword, it is possible that partial event sequences
-are discarded because they exceed the window length. To act upon a timed out partial match one can use `TimedOutPartialMatchHandler` interface.
-The interface is supposed to be used in a mixin style. This mean you can additionally implement this interface with your `PatternProcessFunction`.
-The `TimedOutPartialMatchHandler` provides the additional `processTimedOutMatch` method which will be called for every timed out partial match.
+当一个模式上通过`within`加上窗口长度后,部分匹配的事件序列就可能因为超过窗口长度而被丢弃。可以使用`TimedOutPartialMatchHandler`接口
+来处理超时的部分匹配。这个接口可以和其它的混合使用。也就是说你可以在自己的`PatternProcessFunction`里另外实现这个接口。
+`TimedOutPartialMatchHandler`提供了另外的`processTimedOutMatch`方法,这个方法对每个超时的部分匹配都会调用。
{% highlight java %}
class MyPatternProcessFunction extends PatternProcessFunction implements TimedOutPartialMatchHandler {
@@ -1537,14 +1493,14 @@ class MyPatternProcessFunction extends PatternProcessFunction
}
{% endhighlight %}
-Note The `processTimedOutMatch` does not give one access to the main output. You can still emit results
-through [side-outputs]({{ site.baseurl }}/dev/stream/side_output.html) though, through the `Context` object.
+Note `processTimedOutMatch`不能访问主输出。
+但你可以通过`Context`对象把结果输出到[侧输出]({{ site.baseurl }}/zh/dev/stream/side_output.html)。
-#### Convenience API
+#### 便捷的API
-The aforementioned `PatternProcessFunction` was introduced in Flink 1.8 and since then it is the recommended way to interact with matches.
-One can still use the old style API like `select`/`flatSelect`, which internally will be translated into a `PatternProcessFunction`.
+前面提到的`PatternProcessFunction`是在Flink 1.8之后引入的,从那之后推荐使用这个接口来处理匹配到的结果。
+用户仍然可以使用像`select`/`flatSelect`这样旧格式的API,它们会在内部被转换为`PatternProcessFunction`。
@@ -1598,17 +1554,17 @@ val timeoutResult: DataStream[TimeoutEvent] = result.getSideOutput(outputTag)
-## Time in CEP library
+## CEP库中的时间
-### Handling Lateness in Event Time
+### 按照事件时间处理迟到事件
-In `CEP` the order in which elements are processed matters. To guarantee that elements are processed in the correct order when working in event time, an incoming element is initially put in a buffer where elements are *sorted in ascending order based on their timestamp*, and when a watermark arrives, all the elements in this buffer with timestamps smaller than that of the watermark are processed. This implies that elements between watermarks are processed in event-time order.
+在`CEP`中,事件的处理顺序很重要。在使用事件时间时,为了保证事件按照正确的顺序被处理,一个事件到来后会先被放到一个缓冲区中,
+在缓冲区里事件都按照时间戳从小到大排序,当水位线到达后,缓冲区中所有小于水位线的事件被处理。这意味着水位线之间的数据都按照时间戳被顺序处理。
-{% warn Attention %} The library assumes correctness of the watermark when working in event time.
+{% warn 注意 %} 这个库假定按照事件时间时水位线一定是正确的。
-To guarantee that elements across watermarks are processed in event-time order, Flink's CEP library assumes
-*correctness of the watermark*, and considers as *late* elements whose timestamp is smaller than that of the last
-seen watermark. Late elements are not further processed. Also, you can specify a sideOutput tag to collect the late elements come after the last seen watermark, you can use it like this.
+为了保证跨水位线的事件按照事件时间处理,Flink CEP库假定*水位线一定是正确的*,并且把时间戳小于最新水位线的事件看作是*晚到*的。
+晚到的事件不会被处理。你也可以指定一个侧输出标志来收集比最新水位线晚到的事件,你可以这样做:
@@ -1651,44 +1607,42 @@ val lateData: DataStream[String] = result.getSideOutput(lateDataOutputTag)
-### Time context
+### 时间上下文
-In [PatternProcessFunction](#selecting-from-patterns) as well as in [IterativeCondition](#conditions) user has access to a context
-that implements `TimeContext` as follows:
+在[PatternProcessFunction](#从模式中选取)中,用户可以和[IterativeCondition](#条件)中
+一样按照下面的方法使用实现了`TimeContext`的上下文:
{% highlight java %}
/**
- * Enables access to time related characteristics such as current processing time or timestamp of
- * currently processed element. Used in {@link PatternProcessFunction} and
- * {@link org.apache.flink.cep.pattern.conditions.IterativeCondition}
+ * 支持获取事件属性比如当前处理事件或当前正处理的事件的时间。
+ * 用在{@link PatternProcessFunction}和{@link org.apache.flink.cep.pattern.conditions.IterativeCondition}中
*/
@PublicEvolving
public interface TimeContext {
/**
- * Timestamp of the element currently being processed.
+ * 当前正处理的事件的时间戳。
*
- *
In case of {@link org.apache.flink.streaming.api.TimeCharacteristic#ProcessingTime} this
- * will be set to the time when event entered the cep operator.
+ *
如果是{@link org.apache.flink.streaming.api.TimeCharacteristic#ProcessingTime},这个值会被设置为事件进入CEP算子的时间。
*/
long timestamp();
- /** Returns the current processing time. */
+ /** 返回当前的处理时间。 */
long currentProcessingTime();
}
{% endhighlight %}
-This context gives user access to time characteristics of processed events (incoming records in case of `IterativeCondition` and matches in case of `PatternProcessFunction`).
-Call to `TimeContext#currentProcessingTime` always gives you the value of current processing time and this call should be preferred to e.g. calling `System.currentTimeMillis()`.
+这个上下文让用户可以获取处理的事件(在`IterativeCondition`时候是进来的记录,在`PatternProcessFunction`是匹配的结果)的时间属性。
+调用`TimeContext#currentProcessingTime`总是返回当前的处理时间,而且尽量去调用这个函数而不是调用其它的比如说`System.currentTimeMillis()`。
-In case of `TimeContext#timestamp()` the returned value is equal to assigned timestamp in case of `EventTime`. In `ProcessingTime` this will equal to the point of time when said event entered
-cep operator (or when the match was generated in case of `PatternProcessFunction`). This means that the value will be consistent across multiple calls to that method.
+使用`EventTime`时,`TimeContext#timestamp()`返回的值等于分配的时间戳。
+使用`ProcessingTime`时,这个值等于事件进入CEP算子的时间点(在`PatternProcessFunction`中是匹配产生的时间)。
+这意味着多次调用这个方法得到的值是一致的。
-## Examples
+## 例子
-The following example detects the pattern `start, middle(name = "error") -> end(name = "critical")` on a keyed data
-stream of `Events`. The events are keyed by their `id`s and a valid pattern has to occur within 10 seconds.
-The whole processing is done with event time.
+下面的例子在一个分片的`Events`流上检测模式`start, middle(name = "error") -> end(name = "critical")`。
+事件按照`id`分片,一个有效的模式需要发生在10秒内。
@@ -1750,31 +1704,25 @@ val alerts = patternStream.select(createAlert(_))
-## Migrating from an older Flink version(pre 1.3)
+## 从旧版本迁移(1.3之前)
-### Migrating to 1.4+
+### 迁移到1.4+
-In Flink-1.4 the backward compatibility of CEP library with <= Flink 1.2 was dropped. Unfortunately
-it is not possible to restore a CEP job that was once run with 1.2.x
+在Flink-1.4放弃了和<= Flink 1.2版本的兼容性。很不幸,不能再恢复用1.2.x运行过的CEP作业。
-### Migrating to 1.3.x
+### 迁移到1.3.x
-The CEP library in Flink-1.3 ships with a number of new features which have led to some changes in the API. Here we
-describe the changes that you need to make to your old CEP jobs, in order to be able to run them with Flink-1.3. After
-making these changes and recompiling your job, you will be able to resume its execution from a savepoint taken with the
-old version of your job, *i.e.* without having to re-process your past data.
+CEP库在Flink-1.3发布的一系列的新特性引入了一些API上的修改。这里我们描述你需要对旧的CEP作业所做的修改,以能够用Flink-1.3来运行它们。
+在做完这些修改并重新编译你的作业之后,可以从旧版本作业的保存点之后继续运行,*也就是说*不需要再重新处理旧的数据。
-The changes required are:
+需要的修改是:
-1. Change your conditions (the ones in the `where(...)` clause) to extend the `SimpleCondition` class instead of
-implementing the `FilterFunction` interface.
+1. 修改你的条件(在`where(...)`语句中的)来继承`SimpleCondition`类而不是实现`FilterFunction`接口。
-2. Change your functions provided as arguments to the `select(...)` and `flatSelect(...)` methods to expect a list of
-events associated with each pattern (`List` in `Java`, `Iterable` in `Scala`). This is because with the addition of
-the looping patterns, multiple input events can match a single (looping) pattern.
+2. 修改你作为`select(...)`和`flatSelect(...)`方法的参数的函数为期望每个模式关联一个事件列表(`Java`中`List`,`Scala`中`Iterable`)。
+这是因为增加了循环模式后,多个事件可能匹配一个单一的(循环)模式。
-3. The `followedBy()` in Flink 1.1 and 1.2 implied `non-deterministic relaxed contiguity` (see
-[here](#conditions-on-contiguity)). In Flink 1.3 this has changed and `followedBy()` implies `relaxed contiguity`,
-while `followedByAny()` should be used if `non-deterministic relaxed contiguity` is required.
+3. 在Flink 1.1和1.2中,`followedBy()`隐含了`不确定的松散连续` (参见[这里](#组合模式))。
+在Flink 1.3中,这里发生了变化, `followedBy()`隐含了`松散连续`,如果需要`不确定的松散连续`,应该使用`followedByAny()`。
{% top %}
diff --git a/docs/dev/libs/state_processor_api.md b/docs/dev/libs/state_processor_api.md
index bbaadde725f2862ffa6e5763f4ad55f5cecf22d4..bb23a5a296089a69724e77d923996432f593b823 100644
--- a/docs/dev/libs/state_processor_api.md
+++ b/docs/dev/libs/state_processor_api.md
@@ -417,7 +417,7 @@ public class SimpleBootstrapFunction extends StateBootstrapFunction {
}
}
-ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnviornment();
+ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet data = env.fromElements(1, 2, 3);
BootstrapTransformation transformation = OperatorTransformation
@@ -446,7 +446,7 @@ class SimpleBootstrapFunction extends StateBootstrapFunction[Integer] {
}
}
-val env = ExecutionEnvironment.getExecutionEnviornment
+val env = ExecutionEnvironment.getExecutionEnvironment
val data = env.fromElements(1, 2, 3)
BootstrapTransformation transformation = OperatorTransformation
diff --git a/docs/dev/libs/state_processor_api.zh.md b/docs/dev/libs/state_processor_api.zh.md
index bbaadde725f2862ffa6e5763f4ad55f5cecf22d4..bb23a5a296089a69724e77d923996432f593b823 100644
--- a/docs/dev/libs/state_processor_api.zh.md
+++ b/docs/dev/libs/state_processor_api.zh.md
@@ -417,7 +417,7 @@ public class SimpleBootstrapFunction extends StateBootstrapFunction {
}
}
-ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnviornment();
+ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet data = env.fromElements(1, 2, 3);
BootstrapTransformation transformation = OperatorTransformation
@@ -446,7 +446,7 @@ class SimpleBootstrapFunction extends StateBootstrapFunction[Integer] {
}
}
-val env = ExecutionEnvironment.getExecutionEnviornment
+val env = ExecutionEnvironment.getExecutionEnvironment
val data = env.fromElements(1, 2, 3)
BootstrapTransformation transformation = OperatorTransformation
diff --git a/docs/dev/parallel.md b/docs/dev/parallel.md
index 693f83f032ea7b7567770cb293c7eb175ee73640..54fa5a1d8b5f87718541f8ed6e22a909333fe2ba 100644
--- a/docs/dev/parallel.md
+++ b/docs/dev/parallel.md
@@ -82,7 +82,7 @@ env.execute("Word Count Example")
### Execution Environment Level
-As mentioned [here]({{ site.baseurl }}/dev/api_concepts.html#anatomy-of-a-flink-program) Flink
+As mentioned [here]({% link dev/datastream_api.md %}#anatomy-of-a-flink-program) Flink
programs are executed in the context of an execution environment. An
execution environment defines a default parallelism for all operators, data sources, and data sinks
it executes. Execution environment parallelism can be overwritten by explicitly configuring the
diff --git a/docs/dev/scala_api_extensions.md b/docs/dev/scala_api_extensions.md
index 69939c40ee1c827d47bb787bdf9a1adf7172a465..7fdafe6d4b42159307739b4e2b8065c334a9b1fb 100644
--- a/docs/dev/scala_api_extensions.md
+++ b/docs/dev/scala_api_extensions.md
@@ -1,7 +1,7 @@
---
title: "Scala API Extensions"
-nav-parent_id: api-concepts
-nav-pos: 10
+nav-parent_id: streaming
+nav-pos: 200
---
-Catalogs provide metadata, such as databases, tables, partitions, views, and functions and information needed to access data stored in a database or other external systems.
+Catalog 提供了元数据信息,例如数据库、表、分区、视图以及数据库或其他外部系统中存储的函数和信息。
-One of the most crucial aspects of data processing is managing metadata.
-It may be transient metadata like temporary tables, or UDFs registered against the table environment.
-Or permanent metadata, like that in a Hive Metastore. Catalogs provide a unified API for managing metadata and making it accessible from the Table API and SQL Queries.
+数据处理最关键的方面之一是管理元数据。
+元数据可以是临时的,例如临时表、或者通过 TableEnvironment 注册的 UDF。
+元数据也可以是持久化的,例如 Hive Metastore 中的元数据。Catalog 提供了一个统一的API,用于管理元数据,并使其可以从 Table API 和 SQL 查询语句中来访问。
* This will be replaced by the TOC
{:toc}
-## Catalog Types
+## Catalog 类型
### GenericInMemoryCatalog
-The `GenericInMemoryCatalog` is an in-memory implementation of a catalog. All objects will be available only for the lifetime of the session.
+`GenericInMemoryCatalog` 是基于内存实现的 Catalog,所有元数据只在 session 的生命周期内可用。
+
+### JDBCCatalog
+
+The `JDBCCatalog` enables users to connect Flink to relational databases over JDBC protocol.
+
+#### PostgresCatalog
+
+`PostgresCatalog` is the only implementation of JDBC Catalog at the moment.
+
+#### Usage of JDBCCatalog
+
+Set a `JDBCatalog` with the following parameters:
+
+- name: required, name of the catalog
+- default database: required, default database to connect to
+- username: required, username of Postgres account
+- password: required, password of the account
+- base url: required, should be of format "jdbc:postgresql://:", and should not contain database name here
+
+
+
+{% highlight java %}
+
+EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
+TableEnvironment tableEnv = TableEnvironment.create(settings);
+
+String name = "mypg";
+String defaultDatabase = "mydb";
+String username = "...";
+String password = "...";
+String baseUrl = "..."
+
+JDBCCatalog catalog = new JDBCCatalog(name, defaultDatabase, username, password, baseUrl);
+tableEnv.registerCatalog("mypg", catalog);
+
+// set the JDBCCatalog as the current catalog of the session
+tableEnv.useCatalog("mypg");
+{% endhighlight %}
+
+
+{% highlight scala %}
+
+val settings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build()
+val tableEnv = TableEnvironment.create(settings)
+
+val name = "mypg"
+val defaultDatabase = "mydb"
+val username = "..."
+val password = "..."
+val baseUrl = "..."
+
+val catalog = new JDBCCatalog(name, defaultDatabase, username, password, baseUrl)
+tableEnv.registerCatalog("mypg", catalog)
+
+// set the JDBCCatalog as the current catalog of the session
+tableEnv.useCatalog("mypg")
+{% endhighlight %}
+
+{% highlight yaml %}
+
+execution:
+ planner: blink
+ ...
+ current-catalog: mypg # set the JDBCCatalog as the current catalog of the session
+ current-database: mydb
+
+catalogs:
+ - name: mypg
+ type: jdbc
+ default-database: mydb
+ username: ...
+ password: ...
+ base-url: ...
+{% endhighlight %}
+
+
+
### HiveCatalog
-The `HiveCatalog` serves two purposes; as persistent storage for pure Flink metadata, and as an interface for reading and writing existing Hive metadata.
-Flink's [Hive documentation]({{ site.baseurl }}/dev/table/hive/index.html) provides full details on setting up the catalog and interfacing with an existing Hive installation.
+`HiveCatalog` 有两个用途:作为原生 Flink 元数据的持久化存储,以及作为读写现有 Hive 元数据的接口。
+Flink 的 [Hive 文档]({{ site.baseurl }}/zh/dev/table/hive/index.html) 提供了有关设置 `HiveCatalog` 以及访问现有 Hive 元数据的详细信息。
-{% warn %} The Hive Metastore stores all meta-object names in lower case. This is unlike `GenericInMemoryCatalog` which is case-sensitive
+警告 Hive Metastore 以小写形式存储所有元数据对象名称。而 `GenericInMemoryCatalog` 区分大小写。
-### User-Defined Catalog
+### 用户自定义 Catalog
-Catalogs are pluggable and users can develop custom catalogs by implementing the `Catalog` interface.
-To use custom catalogs in SQL CLI, users should develop both a catalog and its corresponding catalog factory by implementing the `CatalogFactory` interface.
+Catalog 是可扩展的,用户可以通过实现 `Catalog` 接口来开发自定义 Catalog。
+想要在 SQL CLI 中使用自定义 Catalog,用户除了需要实现自定义的 Catalog 之外,还需要为这个 Catalog 实现对应的 `CatalogFactory` 接口。
-The catalog factory defines a set of properties for configuring the catalog when the SQL CLI bootstraps.
-The set of properties will be passed to a discovery service where the service tries to match the properties to a `CatalogFactory` and initiate a corresponding catalog instance.
+`CatalogFactory` 定义了一组属性,用于 SQL CLI 启动时配置 Catalog。
+这组属性集将传递给发现服务,在该服务中,服务会尝试将属性关联到 `CatalogFactory` 并初始化相应的 Catalog 实例。
-## How to Create and Register Flink Tables to Catalog
+## 如何创建 Flink 表并将其注册到 Catalog
-### Using SQL DDL
+### 使用 SQL DDL
-Users can use SQL DDL to create tables in catalogs in both Table API and SQL.
+用户可以使用 DDL 通过 Table API 或者 SQL Client 在 Catalog 中创建表。
-For Table API:
+使用 Table API:
@@ -84,7 +175,7 @@ tableEnv.listTables(); // should return the tables in current catalog and databa
-For SQL Client:
+使用 SQL Client:
{% highlight sql %}
// the catalog should have been registered via yaml file
@@ -96,11 +187,11 @@ Flink SQL> SHOW TABLES;
mytable
{% endhighlight %}
-For detailed information, please check out [Flink SQL CREATE DDL]({{ site.baseurl }}/dev/table/sql/create.html).
+更多详细信息,请参考[Flink SQL CREATE DDL]({{ site.baseurl }}/zh/dev/table/sql/create.html)。
-### Using Java/Scala/Python API
+### 使用 Java/Scala/Python API
-Users can use Java, Scala, or Python API to create catalog tables programmatically.
+用户可以用编程的方式使用Java、Scala 或者 Python API 来创建 Catalog 表。
@@ -142,11 +233,11 @@ List tables = catalog.listTables("mydb"); // tables should contain "myta
## Catalog API
-Note: only catalog program APIs are listed here. Users can achieve many of the same funtionalities with SQL DDL.
-For detailed DDL information, please refer to [SQL CREATE DDL]({{ site.baseurl }}/dev/table/sql/create.html).
+注意:这里只列出了编程方式的 Catalog API,用户可以使用 SQL DDL 实现许多相同的功能。
+关于 DDL 的详细信息请参考 [SQL CREATE DDL]({{ site.baseurl }}/zh/dev/table/sql/create.html)。
-### Database operations
+### 数据库操作
-## Table API and SQL for Catalog
+## 通过 Table API 和 SQL Client 操作 Catalog
-### Registering a Catalog
+### 注册 Catalog
-Users have access to a default in-memory catalog named `default_catalog`, that is always created by default. This catalog by default has a single database called `default_database`.
-Users can also register additional catalogs into an existing Flink session.
+用户可以访问默认创建的内存 Catalog `default_catalog`,这个 Catalog 默认拥有一个默认数据库 `default_database`。
+用户也可以注册其他的 Catalog 到现有的 Flink 会话中。
-All catalogs defined using YAML must provide a `type` property that specifies the type of catalog.
-The following types are supported out of the box.
+使用 YAML 定义的 Catalog 必须提供 `type` 属性,以表示指定的 Catalog 类型。
+以下几种类型可以直接使用。
@@ -345,9 +436,9 @@ catalogs:
-### Changing the Current Catalog And Database
+### 修改当前的 Catalog 和数据库
-Flink will always search for tables, views, and UDF's in the current catalog and database.
+Flink 始终在当前的 Catalog 和数据库中寻找表、视图和 UDF。
@@ -364,7 +455,7 @@ Flink SQL> USE myDB;
-Metadata from catalogs that are not the current catalog are accessible by providing fully qualified names in the form `catalog.database.object`.
+通过提供全限定名 `catalog.database.object` 来访问不在当前 Catalog 中的元数据信息。
@@ -379,7 +470,7 @@ Flink SQL> SELECT * FROM not_the_current_catalog.not_the_current_db.my_table;
-### List Available Catalogs
+### 列出可用的 Catalog
@@ -395,7 +486,7 @@ Flink SQL> show catalogs;
-### List Available Databases
+### 列出可用的数据库
@@ -410,7 +501,7 @@ Flink SQL> show databases;
-### List Available Tables
+### 列出可用的表
diff --git a/docs/dev/table/common.md b/docs/dev/table/common.md
index bd015e3285d5da87a8ea893a587ced6ae8a400de..0d3b56142551c4ccf3140805132d50f80bde6a03 100644
--- a/docs/dev/table/common.md
+++ b/docs/dev/table/common.md
@@ -835,13 +835,18 @@ Table API and SQL queries are translated into [DataStream]({{ site.baseurl }}/de
1. Optimization of the logical plan
2. Translation into a DataStream or DataSet program
-A Table API or SQL query is translated when:
+For streaming, a Table API or SQL query is translated when:
+
+* `TableEnvironment.execute()` is called. A `Table` (emitted to a `TableSink` through `Table.insertInto()`) or a SQL update query (specified through `TableEnvironment.sqlUpdate()`) will be buffered in `TableEnvironment` first. Each sink will be optimized independently. The execution graph contains multiple independent sub-DAGs.
+* A `Table` is translated when it is converted into a `DataStream` (see [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api)). Once translated, it's a regular DataStream program and is executed when `StreamExecutionEnvironment.execute()` is called.
+
+For batch, a Table API or SQL query is translated when:
* a `Table` is emitted to a `TableSink`, i.e., when `Table.insertInto()` is called.
* a SQL update query is specified, i.e., when `TableEnvironment.sqlUpdate()` is called.
-* a `Table` is converted into a `DataStream` or `DataSet` (see [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api)).
+* a `Table` is converted into a `DataSet` (see [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api)).
-Once translated, a Table API or SQL query is handled like a regular DataStream or DataSet program and is executed when `StreamExecutionEnvironment.execute()` or `ExecutionEnvironment.execute()` is called.
+Once translated, a Table API or SQL query is handled like a regular DataSet program and is executed when `ExecutionEnvironment.execute()` is called.
@@ -851,17 +856,11 @@ Table API and SQL queries are translated into [DataStream]({{ site.baseurl }}/de
1. Optimization of the logical plan,
2. Translation into a DataStream program.
-The behavior of translating a query is different for `TableEnvironment` and `StreamTableEnvironment`.
-
-For `TableEnvironment`, a Table API or SQL query is translated when `TableEnvironment.execute()` is called, because `TableEnvironment` will optimize multiple-sinks into one DAG.
+a Table API or SQL query is translated when:
-While for `StreamTableEnvironment`, a Table API or SQL query is translated when:
-
-* a `Table` is emitted to a `TableSink`, i.e., when `Table.insertInto()` is called.
-* a SQL update query is specified, i.e., when `TableEnvironment.sqlUpdate()` is called.
-* a `Table` is converted into a `DataStream`.
+* `TableEnvironment.execute()` is called. A `Table` (emitted to a `TableSink` through `Table.insertInto()`) or a SQL update query (specified through `TableEnvironment.sqlUpdate()`) will be buffered in `TableEnvironment` first. All sinks will be optimized into one DAG.
+* A `Table` is translated when it is converted into a `DataStream` (see [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api)). Once translated, it's a regular DataStream program and is executed when `StreamExecutionEnvironment.execute()` is called.
-Once translated, a Table API or SQL query is handled like a regular DataStream program and is executed when `TableEnvironment.execute()` or `StreamExecutionEnvironment.execute()` is called.
@@ -1293,7 +1292,7 @@ val table: Table = tableEnv.fromDataStream(stream, 'age as 'myAge, 'name as 'myN
#### POJO (Java and Scala)
-Flink supports POJOs as composite types. The rules for what determines a POJO are documented [here]({{ site.baseurl }}/dev/api_concepts.html#pojos).
+Flink supports POJOs as composite types. The rules for what determines a POJO are documented [here]({% link dev/types_serialization.md %}#pojos).
When converting a POJO `DataStream` or `DataSet` into a `Table` without specifying field names, the names of the original POJO fields are used. The name mapping requires the original names and cannot be done by positions. Fields can be renamed using an alias (with the `as` keyword), reordered, and projected.
diff --git a/docs/dev/table/common.zh.md b/docs/dev/table/common.zh.md
index c2b579cf3ed0b9b5df68743045d599035941ce15..4670a205a47efa261e71d0ec446df026a9605afc 100644
--- a/docs/dev/table/common.zh.md
+++ b/docs/dev/table/common.zh.md
@@ -1,5 +1,5 @@
---
-title: "Concepts & Common API"
+title: "概念与通用 API"
nav-parent_id: tableapi
nav-pos: 0
---
@@ -22,28 +22,28 @@ specific language governing permissions and limitations
under the License.
-->
-The Table API and SQL are integrated in a joint API. The central concept of this API is a `Table` which serves as input and output of queries. This document shows the common structure of programs with Table API and SQL queries, how to register a `Table`, how to query a `Table`, and how to emit a `Table`.
+Table API 和 SQL 集成在同一套 API 中。这套 API 的核心概念是`Table`,用作查询的输入和输出。本文介绍了 Table API 和 SQL 查询程序的通用结构、如何注册 `Table` 、如何查询 `Table` 以及如何输出 `Table` 。
* This will be replaced by the TOC
{:toc}
-Main Differences Between the Two Planners
+两种计划器(Planner)的主要区别
-----------------------------------------
-1. Blink treats batch jobs as a special case of streaming. As such, the conversion between Table and DataSet is also not supported, and batch jobs will not be translated into `DateSet` programs but translated into `DataStream` programs, the same as the streaming jobs.
-2. The Blink planner does not support `BatchTableSource`, use bounded `StreamTableSource` instead of it.
-3. The Blink planner only support the brand new `Catalog` and does not support `ExternalCatalog` which is deprecated.
-4. The implementations of `FilterableTableSource` for the old planner and the Blink planner are incompatible. The old planner will push down `PlannerExpression`s into `FilterableTableSource`, while the Blink planner will push down `Expression`s.
-5. String based key-value config options (Please see the documentation about [Configuration]({{ site.baseurl }}/dev/table/config.html) for details) are only used for the Blink planner.
-6. The implementation(`CalciteConfig`) of `PlannerConfig` in two planners is different.
-7. The Blink planner will optimize multiple-sinks into one DAG (supported only on `TableEnvironment`, not on `StreamTableEnvironment`). The old planner will always optimize each sink into a new DAG, where all DAGs are independent of each other.
-8. The old planner does not support catalog statistics now, while the Blink planner does.
+1. Blink 将批处理作业视作流处理的一种特例。严格来说,`Table` 和 `DataSet` 之间不支持相互转换,并且批处理作业也不会转换成 `DataSet` 程序而是转换成 `DataStream` 程序,流处理作业也一样。
+2. Blink 计划器不支持 `BatchTableSource`,而是使用有界的 `StreamTableSource` 来替代。
+3. Blink 计划器仅支持全新的 `Catalog` 不支持被弃用的 `ExternalCatalog`。
+4. 旧计划器和 Blink 计划器中 `FilterableTableSource` 的实现是不兼容的。旧计划器会将 `PlannerExpression` 下推至 `FilterableTableSource`,而 Blink 计划器则是将 `Expression` 下推。
+5. 基于字符串的键值配置选项仅在 Blink 计划器中使用。(详情参见 [配置]({{ site.baseurl }}/zh/dev/table/config.html) )
+6. `PlannerConfig` 在两种计划器中的实现(`CalciteConfig`)是不同的。
+7. Blink 计划器会将多sink(multiple-sinks)优化成一张有向无环图(DAG)(仅支持 `TableEnvironment`,不支持 `StreamTableEnvironment`)。旧计划器总是将每个sink都优化成一个新的有向无环图,且所有图相互独立。
+8. 旧计划器目前不支持 catalog 统计数据,而 Blink 支持。
-Structure of Table API and SQL Programs
+Table API 和 SQL 程序的结构
---------------------------------------
-All Table API and SQL programs for batch and streaming follow the same pattern. The following code example shows the common structure of Table API and SQL programs.
+所有用于批处理和流处理的 Table API 和 SQL 程序都遵循相同的模式。下面的代码示例展示了 Table API 和 SQL 程序的通用结构。
-**Note:** Table API and SQL queries can be easily integrated with and embedded into DataStream or DataSet programs. Have a look at the [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api) section to learn how DataStreams and DataSets can be converted into Tables and vice versa.
+**注意:** Table API 和 SQL 查询可以很容易地集成并嵌入到 DataStream 或 DataSet 程序中。 请参阅[与 DataStream 和 DataSet API 结合](#integration-with-datastream-and-dataset-api) 章节了解如何将 DataSet 和 DataStream 与表之间的相互转化。
{% top %}
-Create a TableEnvironment
+创建 TableEnvironment
-------------------------
-The `TableEnvironment` is a central concept of the Table API and SQL integration. It is responsible for:
+`TableEnvironment` 是 Table API 和 SQL 的核心概念。它负责:
-* Registering a `Table` in the internal catalog
-* Registering catalogs
-* Loading pluggable modules
-* Executing SQL queries
-* Registering a user-defined (scalar, table, or aggregation) function
-* Converting a `DataStream` or `DataSet` into a `Table`
-* Holding a reference to an `ExecutionEnvironment` or `StreamExecutionEnvironment`
+* 在内部的 catalog 中注册 `Table`
+* 注册外部的 catalog
+* 加载可插拔模块
+* 执行 SQL 查询
+* 注册自定义函数 (scalar、table 或 aggregation)
+* 将 `DataStream` 或 `DataSet` 转换成 `Table`
+* 持有对 `ExecutionEnvironment` 或 `StreamExecutionEnvironment` 的引用
-A `Table` is always bound to a specific `TableEnvironment`. It is not possible to combine tables of different TableEnvironments in the same query, e.g., to join or union them.
+`Table` 总是与特定的 `TableEnvironment` 绑定。不能在同一条查询中使用不同 TableEnvironment 中的表,例如,对它们进行 join 或 union 操作。
-A `TableEnvironment` is created by calling the static `BatchTableEnvironment.create()` or `StreamTableEnvironment.create()` method with a `StreamExecutionEnvironment` or an `ExecutionEnvironment` and an optional `TableConfig`. The `TableConfig` can be used to configure the `TableEnvironment` or to customize the query optimization and translation process (see [Query Optimization](#query-optimization)).
+`TableEnvironment` 可以通过静态方法 `BatchTableEnvironment.create()` 或者 `StreamTableEnvironment.create()` 在 `StreamExecutionEnvironment` 或者 `ExecutionEnvironment` 中创建,`TableConfig` 是可选项。`TableConfig`可用于配置`TableEnvironment`或定制的查询优化和转换过程(参见 [查询优化](#query-optimization))。
-Make sure to choose the specific planner `BatchTableEnvironment`/`StreamTableEnvironment` that matches your programming language.
+请确保选择与你的编程语言匹配的特定的计划器`BatchTableEnvironment`/`StreamTableEnvironment`。
-If both planner jars are on the classpath (the default behavior), you should explicitly set which planner to use in the current program.
+如果两种计划器的 jar 包都在 classpath 中(默认行为),你应该明确地设置要在当前程序中使用的计划器。
-**Note:** If there is only one planner jar in `/lib` directory, you can use `useAnyPlanner` (`use_any_planner` for python) to create specific `EnvironmentSettings`.
+**注意:** 如果`/lib`目录中只有一种计划器的 jar 包,则可以使用`useAnyPlanner`(python 使用 `use any_u_planner`)创建 `EnvironmentSettings`。
{% top %}
-Create Tables in the Catalog
+在 Catalog 中创建表
-------------------------------
-A `TableEnvironment` maintains a map of catalogs of tables which are created with an identifier. Each
-identifier consists of 3 parts: catalog name, database name and object name. If a catalog or database is not
-specified, the current default value will be used (see examples in the [Table identifier expanding]({{ site.baseurl }}/dev/table/common.html#table-identifier-expanding) section).
+`TableEnvironment` 维护着一个由标识符(identifier)创建的表 catalog 的映射。标识符由三个部分组成:catalog 名称、数据库名称以及对象名称。如果 catalog 或者数据库没有指明,就会使用当前默认值(参见[表标识符扩展]({{ site.baseurl }}/zh/dev/table/common.html#table-identifier-expanding)章节中的例子)。
-Tables can be either virtual (`VIEWS`) or regular (`TABLES`). `VIEWS` can be created from an
-existing `Table` object, usually the result of a Table API or SQL query. `TABLES` describe
-external data, such as a file, database table, or message queue.
+`Table` 可以是虚拟的(视图 `VIEWS`)也可以是常规的(表 `TABLES`)。视图 `VIEWS`可以从已经存在的`Table`中创建,一般是 Table API 或者 SQL 的查询结果。 表`TABLES`描述的是外部数据,例如文件、数据库表或者消息队列。
-### Temporary vs Permanent tables.
+### 临时表(Temporary Table)和永久表(Permanent Table)
-Tables may either be temporary, and tied to the lifecycle of a single Flink session, or permanent,
-and visible across multiple Flink sessions and clusters.
+表可以是临时的,并与单个 Flink 会话(session)的生命周期相关,也可以是永久的,并且在多个 Flink 会话和群集(cluster)中可见。
-Permanent tables require a [catalog]({{ site.baseurl }}/dev/table/catalogs.html) (such as Hive Metastore)
-to maintain metadata about the table. Once a permanent table is created, it is visible to any Flink
-session that is connected to the catalog and will continue to exist until the table is explicitly
-dropped.
+永久表需要 [catalog]({{ site.baseurl }}/zh/dev/table/catalogs.html)(例如 Hive Metastore)以维护表的元数据。一旦永久表被创建,它将对任何连接到 catalog 的 Flink 会话可见且持续存在,直至被明确删除。
-On the other hand, temporary tables are always stored in memory and only exist for the duration of
-the Flink session they are created within. These tables are not visible to other sessions. They are
-not bound to any catalog or database but can be created in the namespace of one. Temporary tables
-are not dropped if their corresponding database is removed.
+另一方面,临时表通常保存于内存中并且仅在创建它们的 Flink 会话持续期间存在。这些表对于其它会话是不可见的。它们不与任何 catalog 或者数据库绑定但可以在一个命名空间(namespace)中创建。即使它们对应的数据库被删除,临时表也不会被删除。
-#### Shadowing
+#### 屏蔽(Shadowing)
-It is possible to register a temporary table with the same identifier as an existing permanent
-table. The temporary table shadows the permanent one and makes the permanent table inaccessible as
-long as the temporary one exists. All queries with that identifier will be executed against the
-temporary table.
+可以使用与已存在的永久表相同的标识符去注册临时表。临时表会屏蔽永久表,并且只要临时表存在,永久表就无法访问。所有使用该标识符的查询都将作用于临时表。
-This might be useful for experimentation. It allows running exactly the same query first against a
-temporary table that e.g. has just a subset of data, or the data is obfuscated. Once verified that
-the query is correct it can be run against the real production table.
+这可能对实验(experimentation)有用。它允许先对一个临时表进行完全相同的查询,例如只有一个子集的数据,或者数据是不确定的。一旦验证了查询的正确性,就可以对实际的生产表进行查询。
-### Create a Table
+### 创建表
-#### Virtual Tables
+#### 虚拟表
-A `Table` API object corresponds to a `VIEW` (virtual table) in a SQL terms. It encapsulates a logical
-query plan. It can be created in a catalog as follows:
+在 SQL 的术语中,Table API 的对象对应于`视图`(虚拟表)。它封装了一个逻辑查询计划。它可以通过以下方法在 catalog 中创建:
-**Note:** `Table` objects are similar to `VIEW`'s from relational database
-systems, i.e., the query that defines the `Table` is not optimized but will be inlined when another
-query references the registered `Table`. If multiple queries reference the same registered `Table`,
-it will be inlined for each referencing query and executed multiple times, i.e., the result of the
-registered `Table` will *not* be shared.
+**注意:** 从传统数据库系统的角度来看,`Table` 对象与 `VIEW` 视图非常像。也就是,定义了 `Table` 的查询是没有被优化的,
+而且会被内嵌到另一个引用了这个注册了的 `Table`的查询中。如果多个查询都引用了同一个注册了的`Table`,那么它会被内嵌每个查询中并被执行多次,
+也就是说注册了的`Table`的结果**不会**被共享(注:Blink 计划器的`TableEnvironment`会优化成只执行一次)。
{% top %}
#### Connector Tables
-It is also possible to create a `TABLE` as known from relational databases from a [connector]({{ site.baseurl }}/dev/table/connect.html) declaration.
-The connector describes the external system that stores the data of a table. Storage systems such as Apacha Kafka or a regular file system can be declared here.
+另外一个方式去创建 `TABLE` 是通过 [connector]({{ site.baseurl }}/dev/table/connect.html) 声明。Connector 描述了存储表数据的外部系统。存储系统例如 Apache Kafka 或者常规的文件系统都可以通过这种方式来声明。
-### Expanding Table identifiers
+### 扩展表标识符
+
-Tables are always registered with a 3-part identifier consisting of catalog, database, and table name.
+表总是通过三元标识符注册,包括 catalog 名、数据库名和表名。
-Users can set one catalog and one database inside it to be the “current catalog” and “current database”.
-With them, the first two parts in the 3-parts identifier mentioned above can be optional - if they are not provided,
-the current catalog and current database will be referred. Users can switch the current catalog and current database via
-table API or SQL.
+用户可以指定一个 catalog 和数据库作为 "当前catalog" 和"当前数据库"。有了这些,那么刚刚提到的三元标识符的前两个部分就可以被省略了。如果前两部分的标识符没有指定,
+那么会使用当前的 catalog 和当前数据库。用户也可以通过 Table API 或 SQL 切换当前的 catalog 和当前的数据库。
-Identifiers follow SQL requirements which means that they can be escaped with a backtick character (`` ` ``).
-Additionally all SQL reserved keywords must be escaped.
+标识符遵循 SQL 标准,因此使用时需要用反引号(`` ` ``)进行转义。此外,所有 SQL 保留关键字都必须转义。
-Query a Table
+查询表
-------------
### Table API
-The Table API is a language-integrated query API for Scala and Java. In contrast to SQL, queries are not specified as Strings but are composed step-by-step in the host language.
+Table API 是关于 Scala 和 Java 的集成语言式查询 API。与 SQL 相反,Table API 的查询不是由字符串指定,而是在宿主语言中逐步构建。
-The API is based on the `Table` class which represents a table (streaming or batch) and offers methods to apply relational operations. These methods return a new `Table` object, which represents the result of applying the relational operation on the input `Table`. Some relational operations are composed of multiple method calls such as `table.groupBy(...).select()`, where `groupBy(...)` specifies a grouping of `table`, and `select(...)` the projection on the grouping of `table`.
+Table API 是基于 `Table` 类的,该类表示一个表(流或批处理),并提供使用关系操作的方法。这些方法返回一个新的 Table 对象,该对象表示对输入 Table 进行关系操作的结果。 一些关系操作由多个方法调用组成,例如 `table.groupBy(...).select()`,其中 `groupBy(...)` 指定 `table` 的分组,而 `select(...)` 在 `table` 分组上的投影。
-The [Table API]({{ site.baseurl }}/dev/table/tableApi.html) document describes all Table API operations that are supported on streaming and batch tables.
+文档 [Table API]({{ site.baseurl }}/zh/dev/table/tableApi.html) 说明了所有流处理和批处理表支持的 Table API 算子。
-The following example shows a simple Table API aggregation query:
+以下示例展示了一个简单的 Table API 聚合查询:
@@ -589,11 +567,11 @@ revenue = orders \
### SQL
-Flink's SQL integration is based on [Apache Calcite](https://calcite.apache.org), which implements the SQL standard. SQL queries are specified as regular Strings.
+Flink SQL 是基于实现了SQL标准的 [Apache Calcite](https://calcite.apache.org) 的。SQL 查询由常规字符串指定。
-The [SQL]({{ site.baseurl }}/dev/table/sql/index.html) document describes Flink's SQL support for streaming and batch tables.
+文档 [SQL]({{ site.baseurl }}/zh/dev/table/sql.html) 描述了Flink对流处理和批处理表的SQL支持。
-The following example shows how to specify a query and return the result as a `Table`.
+下面的示例演示了如何指定查询并将结果作为 `Table` 对象返回。
-The following example shows how to specify an update query that inserts its result into a registered table.
+如下的示例展示了如何指定一个更新查询,将查询的结果插入到已注册的表中。
@@ -728,27 +706,27 @@ table_env.sql_update(
{% top %}
-### Mixing Table API and SQL
+### 混用 Table API 和 SQL
-Table API and SQL queries can be easily mixed because both return `Table` objects:
+Table API 和 SQL 查询的混用非常简单因为它们都返回 `Table` 对象:
-* A Table API query can be defined on the `Table` object returned by a SQL query.
-* A SQL query can be defined on the result of a Table API query by [registering the resulting Table](#register-a-table) in the `TableEnvironment` and referencing it in the `FROM` clause of the SQL query.
+* 可以在 SQL 查询返回的 `Table` 对象上定义 Table API 查询。
+* 在 `TableEnvironment` 中注册的[结果表](#register-a-table)可以在 SQL 查询的 `FROM` 子句中引用,通过这种方法就可以在 Table API 查询的结果上定义 SQL 查询。
{% top %}
-Emit a Table
+输出表
------------
-A `Table` is emitted by writing it to a `TableSink`. A `TableSink` is a generic interface to support a wide variety of file formats (e.g. CSV, Apache Parquet, Apache Avro), storage systems (e.g., JDBC, Apache HBase, Apache Cassandra, Elasticsearch), or messaging systems (e.g., Apache Kafka, RabbitMQ).
+`Table` 通过写入 `TableSink` 输出。`TableSink` 是一个通用接口,用于支持多种文件格式(如 CSV、Apache Parquet、Apache Avro)、存储系统(如 JDBC、Apache HBase、Apache Cassandra、Elasticsearch)或消息队列系统(如 Apache Kafka、RabbitMQ)。
-A batch `Table` can only be written to a `BatchTableSink`, while a streaming `Table` requires either an `AppendStreamTableSink`, a `RetractStreamTableSink`, or an `UpsertStreamTableSink`.
+批处理 `Table` 只能写入 `BatchTableSink`,而流处理 `Table` 需要指定写入 `AppendStreamTableSink`,`RetractStreamTableSink` 或者 `UpsertStreamTableSink`。
-Please see the documentation about [Table Sources & Sinks]({{ site.baseurl }}/dev/table/sourceSinks.html) for details about available sinks and instructions for how to implement a custom `TableSink`.
+请参考文档 [Table Sources & Sinks]({{ site.baseurl }}/zh/dev/table/sourceSinks.html) 以获取更多关于可用 Sink 的信息以及如何自定义 `TableSink`。
-The `Table.insertInto(String tableName)` method emits the `Table` to a registered `TableSink`. The method looks up the `TableSink` from the catalog by the name and validates that the schema of the `Table` is identical to the schema of the `TableSink`.
+方法 `Table.insertInto(String tableName)` 将 `Table` 发送至已注册的 `TableSink`。该方法通过名称在 catalog 中查找 `TableSink` 并确认`Table` schema 和 `TableSink` schema 一致。
-The following examples shows how to emit a `Table`:
+下面的示例演示如何输出 `Table`:
@@ -832,70 +810,69 @@ result.insert_into("CsvSinkTable")
{% top %}
-Translate and Execute a Query
+翻译与执行查询
-----------------------------
-The behavior of translating and executing a query is different for the two planners.
+两种计划器翻译和执行查询的方式是不同的。
-Table API and SQL queries are translated into [DataStream]({{ site.baseurl }}/dev/datastream_api.html) or [DataSet]({{ site.baseurl }}/dev/batch) programs depending on whether their input is a streaming or batch input. A query is internally represented as a logical query plan and is translated in two phases:
+Table API 和 SQL 查询会被翻译成 [DataStream]({{ site.baseurl }}/zh/dev/datastream_api.html) 或者 [DataSet]({{ site.baseurl }}/zh/dev/batch) 程序, 这取决于它们的输入数据源是流式的还是批式的。查询在内部表示为逻辑查询计划,并被翻译成两个阶段:
-1. Optimization of the logical plan
-2. Translation into a DataStream or DataSet program
+1. 优化逻辑执行计划
+2. 翻译成 DataStream 或 DataSet 程序
-A Table API or SQL query is translated when:
+对于 Streaming 而言,Table API 或者 SQL 查询在下列情况下会被翻译:
-* a `Table` is emitted to a `TableSink`, i.e., when `Table.insertInto()` is called.
-* a SQL update query is specified, i.e., when `TableEnvironment.sqlUpdate()` is called.
-* a `Table` is converted into a `DataStream` or `DataSet` (see [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api)).
+* 当 `TableEnvironment.execute()` 被调用时。`Table` (通过 `Table.insertInto()` 输出给 `TableSink`)和 SQL (通过调用 `TableEnvironment.sqlUpdate()`)会先被缓存到 `TableEnvironment` 中,每个 sink 会被单独优化。执行计划将包括多个独立的有向无环子图。
+* `Table` 被转换成 `DataStream` 时(参阅[与 DataStream 和 DataSet API 结合](#integration-with-datastream-and-dataset-api))。转换完成后,它就成为一个普通的 DataStream 程序,并且会在调用 `StreamExecutionEnvironment.execute()` 的时候被执行。
-Once translated, a Table API or SQL query is handled like a regular DataStream or DataSet program and is executed when `StreamExecutionEnvironment.execute()` or `ExecutionEnvironment.execute()` is called.
+对于 Batch 而言,Table API 或者 SQL 查询在下列情况下会被翻译:
-
-
-
-Table API and SQL queries are translated into [DataStream]({{ site.baseurl }}/dev/datastream_api.html) programs whether their input is streaming or batch. A query is internally represented as a logical query plan and is translated in two phases:
+* `Table` 被输出给 `TableSink`,即当调用 `Table.insertInto()` 时。
+* SQL 更新语句执行时,即,当调用 `TableEnvironment.sqlUpdate()` 时。
+* `Table` 被转换成 `DataSet` 时(参阅[与 DataStream 和 DataSet API 结合](#integration-with-datastream-and-dataset-api))。
-1. Optimization of the logical plan,
-2. Translation into a DataStream program.
+翻译完成后,Table API 或者 SQL 查询会被当做普通的 DataSet 程序对待并且会在调用 `ExecutionEnvironment.execute()` 的时候被执行。
-The behavior of translating a query is different for `TableEnvironment` and `StreamTableEnvironment`.
+
-For `TableEnvironment`, a Table API or SQL query is translated when `TableEnvironment.execute()` is called, because `TableEnvironment` will optimize multiple-sinks into one DAG.
+
+不论输入数据源是流式的还是批式的,Table API 和 SQL 查询都会被转换成 [DataStream]({{ site.baseurl }}/zh/dev/datastream_api.html) 程序。查询在内部表示为逻辑查询计划,并被翻译成两个阶段:
-While for `StreamTableEnvironment`, a Table API or SQL query is translated when:
+1. 优化逻辑执行计划
+2. 翻译成 DataStream 程序
-* a `Table` is emitted to a `TableSink`, i.e., when `Table.insertInto()` is called.
-* a SQL update query is specified, i.e., when `TableEnvironment.sqlUpdate()` is called.
-* a `Table` is converted into a `DataStream`.
+Table API 或者 SQL 查询在下列情况下会被翻译:
-Once translated, a Table API or SQL query is handled like a regular DataStream program and is executed when `TableEnvironment.execute()` or `StreamExecutionEnvironment.execute()` is called.
+* 当 `TableEnvironment.execute()` 被调用时。`Table` (通过 `Table.insertInto()` 输出给 `TableSink`)和 SQL (通过调用 `TableEnvironment.sqlUpdate()`)会先被缓存到 `TableEnvironment` 中,所有的 sink 会被优化成一张有向无环图。
+* `Table` 被转换成 `DataStream` 时(参阅[与 DataStream 和 DataSet API 结合](#integration-with-datastream-and-dataset-api))。转换完成后,它就成为一个普通的 DataStream 程序,并且会在调用 `StreamExecutionEnvironment.execute()` 的时候被执行。
{% top %}
-Integration with DataStream and DataSet API
+与 DataStream 和 DataSet API 结合
-------------------------------------------
-Both planners on stream can integrate with the `DataStream` API. Only old planner can integrate with the `DataSet API`, Blink planner on batch could not be combined with both.
-**Note:** The `DataSet` API discussed below is only relevant for the old planner on batch.
+在流处理方面两种计划器都可以与 `DataStream` API 结合。只有旧计划器可以与 `DataSet API` 结合。在批处理方面,Blink 计划器不能同两种计划器中的任何一个结合。
+
+**注意:** 下文讨论的 `DataSet` API 只与旧计划起有关。
-Table API and SQL queries can be easily integrated with and embedded into [DataStream]({{ site.baseurl }}/dev/datastream_api.html) and [DataSet]({{ site.baseurl }}/dev/batch) programs. For instance, it is possible to query an external table (for example from a RDBMS), do some pre-processing, such as filtering, projecting, aggregating, or joining with meta data, and then further process the data with either the DataStream or DataSet API (and any of the libraries built on top of these APIs, such as CEP or Gelly). Inversely, a Table API or SQL query can also be applied on the result of a DataStream or DataSet program.
+Table API 和 SQL 可以被很容易地集成并嵌入到 [DataStream]({{ site.baseurl }}/zh/dev/datastream_api.html) 和 [DataSet]({{ site.baseurl }}/zh/dev/batch) 程序中。例如,可以查询外部表(例如从 RDBMS),进行一些预处理,例如过滤,投影,聚合或与元数据 join,然后使用 DataStream 或 DataSet API(以及在这些 API 之上构建的任何库,例如 CEP 或 Gelly)。相反,也可以将 Table API 或 SQL 查询应用于 DataStream 或 DataSet 程序的结果。
-This interaction can be achieved by converting a `DataStream` or `DataSet` into a `Table` and vice versa. In this section, we describe how these conversions are done.
+这种交互可以通过 `DataStream` 或 `DataSet` 与 `Table` 的相互转化实现。本节我们会介绍这些转化是如何实现的。
-### Implicit Conversion for Scala
+### Scala 隐式转换
-The Scala Table API features implicit conversions for the `DataSet`, `DataStream`, and `Table` classes. These conversions are enabled by importing the package `org.apache.flink.table.api.scala._` in addition to `org.apache.flink.api.scala._` for the Scala DataStream API.
+Scala Table API 含有对 `DataSet`、`DataStream` 和 `Table` 类的隐式转换。 通过为 Scala DataStream API 导入 `org.apache.flink.table.api.scala._` 包以及 `org.apache.flink.api.scala._` 包,可以启用这些转换。
-### Create a View from a DataStream or DataSet
+### 通过 DataSet 或 DataStream 创建`视图`
-A `DataStream` or `DataSet` can be registered in a `TableEnvironment` as a View. The schema of the resulting view depends on the data type of the registered `DataStream` or `DataSet`. Please check the section about [mapping of data types to table schema](#mapping-of-data-types-to-table-schema) for details.
+在 `TableEnvironment` 中可以将 `DataStream` 或 `DataSet` 注册成视图。结果视图的 schema 取决于注册的 `DataStream` 或 `DataSet` 的数据类型。请参阅文档 [数据类型到 table schema 的映射](#mapping-of-data-types-to-table-schema)获取详细信息。
-**Note:** Views created from a `DataStream` or `DataSet` can be registered as temporary views only.
+**注意:** 通过 `DataStream` 或 `DataSet` 创建的视图只能注册成临时视图。
@@ -933,9 +910,9 @@ tableEnv.createTemporaryView("myTable2", stream, 'myLong, 'myString)
{% top %}
-### Convert a DataStream or DataSet into a Table
+### 将 DataStream 或 DataSet 转换成表
-Instead of registering a `DataStream` or `DataSet` in a `TableEnvironment`, it can also be directly converted into a `Table`. This is convenient if you want to use the Table in a Table API query.
+与在 `TableEnvironment` 注册 `DataStream` 或 `DataSet` 不同,DataStream 和 DataSet 还可以直接转换成 `Table`。如果你想在 Table API 的查询中使用表,这将非常便捷。
@@ -973,26 +950,28 @@ val table2: Table = tableEnv.fromDataStream(stream, 'myLong, 'myString)
{% top %}
-### Convert a Table into a DataStream or DataSet
+
+
+### 将表转换成 DataStream 或 DataSet
-A `Table` can be converted into a `DataStream` or `DataSet`. In this way, custom DataStream or DataSet programs can be run on the result of a Table API or SQL query.
+`Table` 可以被转换成 `DataStream` 或 `DataSet`。通过这种方式,定制的 DataSet 或 DataStream 程序就可以在 Table API 或者 SQL 的查询结果上运行了。
-When converting a `Table` into a `DataStream` or `DataSet`, you need to specify the data type of the resulting `DataStream` or `DataSet`, i.e., the data type into which the rows of the `Table` are to be converted. Often the most convenient conversion type is `Row`. The following list gives an overview of the features of the different options:
+将 `Table` 转换为 `DataStream` 或者 `DataSet` 时,你需要指定生成的 `DataStream` 或者 `DataSet` 的数据类型,即,`Table` 的每行数据要转换成的数据类型。通常最方便的选择是转换成 `Row` 。以下列表概述了不同选项的功能:
-- **Row**: fields are mapped by position, arbitrary number of fields, support for `null` values, no type-safe access.
-- **POJO**: fields are mapped by name (POJO fields must be named as `Table` fields), arbitrary number of fields, support for `null` values, type-safe access.
-- **Case Class**: fields are mapped by position, no support for `null` values, type-safe access.
-- **Tuple**: fields are mapped by position, limitation to 22 (Scala) or 25 (Java) fields, no support for `null` values, type-safe access.
-- **Atomic Type**: `Table` must have a single field, no support for `null` values, type-safe access.
+- **Row**: 字段按位置映射,字段数量任意,支持 `null` 值,无类型安全(type-safe)检查。
+- **POJO**: 字段按名称映射(POJO 必须按`Table` 中字段名称命名),字段数量任意,支持 `null` 值,无类型安全检查。
+- **Case Class**: 字段按位置映射,不支持 `null` 值,有类型安全检查。
+- **Tuple**: 字段按位置映射,字段数量少于 22(Scala)或者 25(Java),不支持 `null` 值,无类型安全检查。
+- **Atomic Type**: `Table` 必须有一个字段,不支持 `null` 值,有类型安全检查。
-#### Convert a Table into a DataStream
+#### 将表转换成 DataStream
-A `Table` that is the result of a streaming query will be updated dynamically, i.e., it is changing as new records arrive on the query's input streams. Hence, the `DataStream` into which such a dynamic query is converted needs to encode the updates of the table.
+流式查询(streaming query)的结果表会动态更新,即,当新纪录到达查询的输入流时,查询结果会改变。因此,像这样将动态查询结果转换成 DataStream 需要对表的更新方式进行编码。
-There are two modes to convert a `Table` into a `DataStream`:
+将 `Table` 转换为 `DataStream` 有两种模式:
-1. **Append Mode**: This mode can only be used if the dynamic `Table` is only modified by `INSERT` changes, i.e, it is append-only and previously emitted results are never updated.
-2. **Retract Mode**: This mode can always be used. It encodes `INSERT` and `DELETE` changes with a `boolean` flag.
+1. **Append Mode**: 仅当动态 `Table` 仅通过`INSERT`更改进行修改时,才可以使用此模式,即,它仅是追加操作,并且之前输出的结果永远不会更新。
+2. **Retract Mode**: 任何情形都可以使用此模式。它使用 boolean 值对 `INSERT` 和 `DELETE` 操作的数据进行标记。
@@ -1049,11 +1028,11 @@ val retractStream: DataStream[(Boolean, Row)] = tableEnv.toRetractStream[Row](ta
-**Note:** A detailed discussion about dynamic tables and their properties is given in the [Dynamic Tables](streaming/dynamic_tables.html) document.
+**注意:** 文档[动态表](streaming/dynamic_tables.html)给出了有关动态表及其属性的详细讨论。
-#### Convert a Table into a DataSet
+#### 将表转换成 DataSet
-A `Table` is converted into a `DataSet` as follows:
+将 `Table` 转换成 `DataSet` 的过程如下:
@@ -1096,17 +1075,17 @@ val dsTuple: DataSet[(String, Int)] = tableEnv.toDataSet[(String, Int)](table)
{% top %}
-### Mapping of Data Types to Table Schema
+### 数据类型到 Table Schema 的映射
-Flink's DataStream and DataSet APIs support very diverse types. Composite types such as Tuples (built-in Scala and Flink Java tuples), POJOs, Scala case classes, and Flink's Row type allow for nested data structures with multiple fields that can be accessed in table expressions. Other types are treated as atomic types. In the following, we describe how the Table API converts these types into an internal row representation and show examples of converting a `DataStream` into a `Table`.
+Flink 的 DataStream 和 DataSet APIs 支持多样的数据类型。例如 Tuple(Scala 内置以及Flink Java tuple)、POJO 类型、Scala case class 类型以及 Flink 的 Row 类型等允许嵌套且有多个可在表的表达式中访问的字段的复合数据类型。其他类型被视为原子类型。下面,我们讨论 Table API 如何将这些数据类型类型转换为内部 row 表示形式,并提供将 `DataStream` 转换成 `Table` 的样例。
-The mapping of a data type to a table schema can happen in two ways: **based on the field positions** or **based on the field names**.
+数据类型到 table schema 的映射有两种方式:**基于字段位置**或**基于字段名称**。
-**Position-based Mapping**
+**基于位置映射**
-Position-based mapping can be used to give fields a more meaningful name while keeping the field order. This mapping is available for composite data types *with a defined field order* as well as atomic types. Composite data types such as tuples, rows, and case classes have such a field order. However, fields of a POJO must be mapped based on the field names (see next section). Fields can be projected out but can't be renamed using an alias `as`.
+基于位置的映射可在保持字段顺序的同时为字段提供更有意义的名称。这种映射方式可用于*具有特定的字段顺序*的复合数据类型以及原子类型。如 tuple、row 以及 case class 这些复合数据类型都有这样的字段顺序。然而,POJO 类型的字段则必须通过名称映射(参见下一章)。可以将字段投影出来,但不能使用`as`重命名。
-When defining a position-based mapping, the specified names must not exist in the input data type, otherwise the API will assume that the mapping should happen based on the field names. If no field names are specified, the default field names and field order of the composite type are used or `f0` for atomic types.
+定义基于位置的映射时,输入数据类型中一定不能存在指定的名称,否则 API 会假定应该基于字段名称进行映射。如果未指定任何字段名称,则使用默认的字段名称和复合数据类型的字段顺序,或者使用 `f0` 表示原子类型。
-**Name-based Mapping**
+**基于名称的映射**
-Name-based mapping can be used for any data type including POJOs. It is the most flexible way of defining a table schema mapping. All fields in the mapping are referenced by name and can be possibly renamed using an alias `as`. Fields can be reordered and projected out.
+基于名称的映射适用于任何数据类型包括 POJO 类型。这是定义 table schema 映射最灵活的方式。映射中的所有字段均按名称引用,并且可以通过 `as` 重命名。字段可以被重新排序和映射。
-If no field names are specified, the default field names and field order of the composite type are used or `f0` for atomic types.
+若果没有指定任何字段名称,则使用默认的字段名称和复合数据类型的字段顺序,或者使用 `f0` 表示原子类型。
@@ -1196,9 +1175,9 @@ val table: Table = tableEnv.fromDataStream(stream, '_2 as 'myInt, '_1 as 'myLong
-#### Atomic Types
+#### 原子类型
-Flink treats primitives (`Integer`, `Double`, `String`) or generic types (types that cannot be analyzed and decomposed) as atomic types. A `DataStream` or `DataSet` of an atomic type is converted into a `Table` with a single attribute. The type of the attribute is inferred from the atomic type and the name of the attribute can be specified.
+Flink 将基础数据类型(`Integer`、`Double`、`String`)或者通用数据类型(不可再拆分的数据类型)视为原子类型。原子类型的 `DataStream` 或者 `DataSet` 会被转换成只有一条属性的 `Table`。属性的数据类型可以由原子类型推断出,还可以重新命名属性。
@@ -1232,9 +1211,9 @@ val table: Table = tableEnv.fromDataStream(stream, 'myLong)
-#### Tuples (Scala and Java) and Case Classes (Scala only)
+#### Tuple类型(Scala 和 Java)和 Case Class类型(仅 Scala)
-Flink supports Scala's built-in tuples and provides its own tuple classes for Java. DataStreams and DataSets of both kinds of tuples can be converted into tables. Fields can be renamed by providing names for all fields (mapping based on position). If no field names are specified, the default field names are used. If the original field names (`f0`, `f1`, ... for Flink Tuples and `_1`, `_2`, ... for Scala Tuples) are referenced, the API assumes that the mapping is name-based instead of position-based. Name-based mapping allows for reordering fields and projection with alias (`as`).
+Flink 支持 Scala 的内置 tuple 类型并给 Java 提供自己的 tuple 类型。两种 tuple 的 DataStream 和 DataSet 都能被转换成表。可以通过提供所有字段名称来重命名字段(基于位置映射)。如果没有指明任何字段名称,则会使用默认的字段名称。如果引用了原始字段名称(对于 Flink tuple 为`f0`、`f1` ... ...,对于 Scala tuple 为`_1`、`_2` ... ...),则 API 会假定映射是基于名称的而不是基于位置的。基于名称的映射可以通过 `as` 对字段和投影进行重新排序。
@@ -1300,11 +1279,11 @@ val table: Table = tableEnv.fromDataStream(stream, 'age as 'myAge, 'name as 'myN
-#### POJO (Java and Scala)
+#### POJO 类型 (Java 和 Scala)
-Flink supports POJOs as composite types. The rules for what determines a POJO are documented [here]({{ site.baseurl }}/dev/api_concepts.html#pojos).
+Flink 支持 POJO 类型作为复合类型。确定 POJO 类型的规则记录在[这里]({{ site.baseurl }}/zh/dev/api_concepts.html#pojos).
-When converting a POJO `DataStream` or `DataSet` into a `Table` without specifying field names, the names of the original POJO fields are used. The name mapping requires the original names and cannot be done by positions. Fields can be renamed using an alias (with the `as` keyword), reordered, and projected.
+在不指定字段名称的情况下将 POJO 类型的 `DataStream` 或 `DataSet` 转换成 `Table` 时,将使用原始 POJO 类型字段的名称。名称映射需要原始名称,并且不能按位置进行。字段可以使用别名(带有 `as` 关键字)来重命名,重新排序和投影。
@@ -1352,9 +1331,9 @@ val table: Table = tableEnv.fromDataStream(stream, 'name as 'myName)
-#### Row
+#### Row类型
-The `Row` data type supports an arbitrary number of fields and fields with `null` values. Field names can be specified via a `RowTypeInfo` or when converting a `Row` `DataStream` or `DataSet` into a `Table`. The row type supports mapping of fields by position and by name. Fields can be renamed by providing names for all fields (mapping based on position) or selected individually for projection/ordering/renaming (mapping based on name).
+`Row` 类型支持任意数量的字段以及具有 `null` 值的字段。字段名称可以通过 `RowTypeInfo` 指定,也可以在将 `Row` 的 `DataStream` 或 `DataSet` 转换为 `Table` 时指定。Row 类型的字段映射支持基于名称和基于位置两种方式。字段可以通过提供所有字段的名称的方式重命名(基于位置映射)或者分别选择进行投影/排序/重命名(基于名称映射)。
@@ -1411,54 +1390,54 @@ val table: Table = tableEnv.fromDataStream(stream, 'name as 'myName)
{% top %}
-Query Optimization
+查询优化
------------------
-Apache Flink leverages Apache Calcite to optimize and translate queries. The optimization currently performed include projection and filter push-down, subquery decorrelation, and other kinds of query rewriting. Old planner does not yet optimize the order of joins, but executes them in the same order as defined in the query (order of Tables in the `FROM` clause and/or order of join predicates in the `WHERE` clause).
+Apache Flink 利用 Apache Calcite 来优化和翻译查询。当前执行的优化包括投影和过滤器下推,子查询消除以及其他类型的查询重写。原版计划程序尚未优化 join 的顺序,而是按照查询中定义的顺序执行它们(FROM 子句中的表顺序和/或 WHERE 子句中的 join 谓词顺序)。
-It is possible to tweak the set of optimization rules which are applied in different phases by providing a `CalciteConfig` object. This can be created via a builder by calling `CalciteConfig.createBuilder())` and is provided to the TableEnvironment by calling `tableEnv.getConfig.setPlannerConfig(calciteConfig)`.
+通过提供一个 `CalciteConfig` 对象,可以调整在不同阶段应用的优化规则集合。这个对象可以通过调用构造器 `CalciteConfig.createBuilder()` 创建,并通过调用 `tableEnv.getConfig.setPlannerConfig(calciteConfig)` 提供给 TableEnvironment。
-Apache Flink leverages and extends Apache Calcite to perform sophisticated query optimization.
-This includes a series of rule and cost-based optimizations such as:
+Apache Flink 使用并扩展了 Apache Calcite 来执行复杂的查询优化。
+这包括一系列基于规则和成本的优化,例如:
-* Subquery decorrelation based on Apache Calcite
-* Project pruning
-* Partition pruning
-* Filter push-down
-* Sub-plan deduplication to avoid duplicate computation
-* Special subquery rewriting, including two parts:
- * Converts IN and EXISTS into left semi-joins
- * Converts NOT IN and NOT EXISTS into left anti-join
-* Optional join reordering
- * Enabled via `table.optimizer.join-reorder-enabled`
+* 基于 Apache Calcite 的子查询解相关
+* 投影剪裁
+* 分区剪裁
+* 过滤器下推
+* 子计划消除重复数据以避免重复计算
+* 特殊子查询重写,包括两部分:
+ * 将 IN 和 EXISTS 转换为 left semi-joins
+ * 将 NOT IN 和 NOT EXISTS 转换为 left anti-join
+* 可选 join 重新排序
+ * 通过 `table.optimizer.join-reorder-enabled` 启用
-**Note:** IN/EXISTS/NOT IN/NOT EXISTS are currently only supported in conjunctive conditions in subquery rewriting.
+**注意:** 当前仅在子查询重写的结合条件下支持 IN / EXISTS / NOT IN / NOT EXISTS。
-The optimizer makes intelligent decisions, based not only on the plan but also rich statistics available from the data sources and fine-grain costs for each operator such as io, cpu, network, and memory.
+优化器不仅基于计划,而且还基于可从数据源获得的丰富统计信息以及每个算子(例如 io,cpu,网络和内存)的细粒度成本来做出明智的决策。
-Advanced users may provide custom optimizations via a `CalciteConfig` object that can be provided to the table environment by calling `TableEnvironment#getConfig#setPlannerConfig`.
+高级用户可以通过 `CalciteConfig` 对象提供自定义优化,可以通过调用 `TableEnvironment#getConfig#setPlannerConfig` 将其提供给 TableEnvironment。
-### Explaining a Table
+### 解释表
-The Table API provides a mechanism to explain the logical and optimized query plans to compute a `Table`.
-This is done through the `TableEnvironment.explain(table)` method or `TableEnvironment.explain()` method. `explain(table)` returns the plan of a given `Table`. `explain()` returns the result of a multiple sinks plan and is mainly used for the Blink planner. It returns a String describing three plans:
+Table API 提供了一种机制来解释计算 `Table` 的逻辑和优化查询计划。
+这是通过 `TableEnvironment.explain(table)` 或者 `TableEnvironment.explain()` 完成的。`explain(table)` 返回给定 `Table` 的计划。 `explain()` 返回多 sink 计划的结果并且主要用于 Blink 计划器。它返回一个描述三种计划的字符串:
-1. the Abstract Syntax Tree of the relational query, i.e., the unoptimized logical query plan,
-2. the optimized logical query plan, and
-3. the physical execution plan.
+1. 关系查询的抽象语法树(the Abstract Syntax Tree),即未优化的逻辑查询计划,
+2. 优化的逻辑查询计划,以及
+3. 物理执行计划。
-The following code shows an example and the corresponding output for given `Table` using `explain(table)`:
+以下代码展示了一个示例以及对给定 `Table` 使用 `explain(table)` 的相应输出:
@@ -1636,7 +1615,7 @@ Stage 4 : Data Source
-The following code shows an example and the corresponding output for multiple-sinks plan using `explain()`:
+以下代码展示了一个示例以及使用 `explain()` 的多 sink 计划的相应输出:
@@ -1754,7 +1733,7 @@ print(explanation)
-the result of multiple-sinks plan is
+多 sink 计划的结果是:
{% highlight text %}
diff --git a/docs/dev/table/config.md b/docs/dev/table/config.md
index a9e9b7119c64abf5e1e2eac5111c62ee4639c8d1..dd312d6fc558ed45e8c0ed6f50eeb3231df5b355 100644
--- a/docs/dev/table/config.md
+++ b/docs/dev/table/config.md
@@ -104,7 +104,3 @@ The following options can be used to tune the performance of the query execution
The following options can be used to adjust the behavior of the query optimizer to get a better execution plan.
{% include generated/optimizer_config_configuration.html %}
-
-### Python Options
-
-{% include generated/python_configuration.html %}
diff --git a/docs/dev/table/config.zh.md b/docs/dev/table/config.zh.md
index 0ef01a1a6227a794bb04e754b3907bcff5a82957..1947ab8b54b6b1fbef9a470dbde5feec0fa519d8 100644
--- a/docs/dev/table/config.zh.md
+++ b/docs/dev/table/config.zh.md
@@ -104,7 +104,3 @@ The following options can be used to tune the performance of the query execution
The following options can be used to adjust the behavior of the query optimizer to get a better execution plan.
{% include generated/optimizer_config_configuration.html %}
-
-### Python Options
-
-{% include generated/python_configuration.html %}
diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md
index d8c3274e0741be738686fd51decdc8e3393fe2a2..0af84d82679c519ddf4eae136f9ea06854aecdd0 100644
--- a/docs/dev/table/connect.md
+++ b/docs/dev/table/connect.md
@@ -1,7 +1,7 @@
---
-title: "Connect to External Systems"
-nav-parent_id: tableapi
-nav-pos: 50
+title: "Table API Connectors"
+nav-parent_id: connectors-root
+nav-pos: 2
---
-Flink Table API & SQL empowers users to do data transformations with functions.
+Flink 允许用户在 Table API 和 SQL 中使用函数进行数据的转换。
* This will be replaced by the TOC
{:toc}
-Types of Functions
+函数类型
------------------
-There are two dimensions to classify functions in Flink.
+Flink 中的函数有两个划分标准。
-One dimension is system (or built-in) functions v.s. catalog functions. System functions have no namespace and can be
-referenced with just their names. Catalog functions belong to a catalog and database therefore they have catalog and database
-namespaces, they can be referenced by either fully/partially qualified name (`catalog.db.func` or `db.func`) or just the
-function name.
+一个划分标准是:系统(内置)函数和 Catalog 函数。系统函数没有名称空间,只能通过其名称来进行引用。
+Catalog 函数属于 Catalog 和数据库,因此它们拥有 Catalog 和数据库命名空间。
+用户可以通过全/部分限定名(`catalog.db.func` 或 `db.func`)或者函数名
+来对 Catalog 函数进行引用。
-The other dimension is temporary functions v.s. persistent functions. Temporary functions are volatile and only live up to
- lifespan of a session, they are always created by users. Persistent functions live across lifespan of sessions, they are either
- provided by the system or persisted in catalogs.
-
-The two dimensions give Flink users 4 categories of functions:
+另一个划分标准是:临时函数和持久化函数。
+临时函数始终由用户创建,它容易改变并且仅在会话的生命周期内有效。
+持久化函数不是由系统提供,就是存储在 Catalog 中,它在会话的整个生命周期内都有效。
-1. Temporary system functions
-2. System functions
-3. Temporary catalog functions
-4. Catalog functions
+这两个划分标准给 Flink 用户提供了 4 种函数:
-Note that system functions always precede catalog's, and temporary functions always precede persistent on their own dimension
-in function resolution order explained below.
+1. 临时性系统函数
+2. 系统函数
+3. 临时性 Catalog 函数
+4. Catalog 函数
-Referencing Functions
+请注意,系统函数始终优先于 Catalog 函数解析,临时函数始终优先于持久化函数解析,
+函数解析优先级如下所述。
+
+函数引用
---------------------
-There are two ways users can reference a function in Flink - referencing function precisely or ambiguously.
+用户在 Flink 中可以通过精确、模糊两种引用方式引用函数。
-## Precise Function Reference
+### 精确函数引用
-Precise function reference empowers users to use catalog functions specifically, and across catalog and across database,
-e.g. `select mycatalog.mydb.myfunc(x) from mytable` and `select mydb.myfunc(x) from mytable`.
+精确函数引用允许用户跨 Catalog,跨数据库调用 Catalog 函数。
+例如:`select mycatalog.mydb.myfunc(x) from mytable` 和 `select mydb.myfunc(x) from mytable`。
-This is only supported starting from Flink 1.10.
+仅 Flink 1.10 以上版本支持。
-## Ambiguous Function Reference
+### 模糊函数引用
-In ambiguous function reference, users just specify the function's name in SQL query, e.g. `select myfunc(x) from mytable`.
+在模糊函数引用中,用户只需在 SQL 查询中指定函数名,例如: `select myfunc(x) from mytable`。
-Function Resolution Order
+函数解析顺序
-------------------------
-The resolution order only matters when there are functions of different types but the same name,
-e.g. when there’re three functions all named “myfunc” but are of temporary catalog, catalog, and system function respectively.
-If there’s no function name collision, functions will just be resolved to the sole one.
+当函数名相同,函数类型不同时,函数解析顺序才有意义。
+例如:当有三个都名为 "myfunc" 的临时性 Catalog 函数,Catalog 函数,和系统函数时,
+如果没有命名冲突,三个函数将会被解析为一个函数。
-## Precise Function Reference
+### 精确函数引用
-Because system functions don’t have namespaces, a precise function reference in Flink must be pointing to either a temporary catalog
-function or a catalog function.
+由于系统函数没有命名空间,Flink 中的精确函数引用必须
+指向临时性 Catalog 函数或 Catalog 函数。
-The resolution order is:
+解析顺序如下:
-1. Temporary catalog function
-2. Catalog function
+1. 临时性 catalog 函数
+2. Catalog 函数
-## Ambiguous Function Reference
+### 模糊函数引用
-The resolution order is:
+解析顺序如下:
-1. Temporary system function
-2. System function
-3. Temporary catalog function, in the current catalog and current database of the session
-4. Catalog function, in the current catalog and current database of the session
+1. 临时性系统函数
+2. 系统函数
+3. 临时性 Catalog 函数, 在会话的当前 Catalog 和当前数据库中
+4. Catalog 函数, 在会话的当前 Catalog 和当前数据库中
diff --git a/docs/dev/table/functions/udfs.md b/docs/dev/table/functions/udfs.md
index 724f1ae39e91c313500420e2ab8ac8ad0991618d..456dbee6a9c76527b613f45744ea1e7a0fb5b0c3 100644
--- a/docs/dev/table/functions/udfs.md
+++ b/docs/dev/table/functions/udfs.md
@@ -134,35 +134,12 @@ object TimestampModifier extends ScalarFunction {
-Note Python 3.5+ and apache-beam==2.19.0 are required to run the Python scalar function.
+In order to define a Python scalar function, one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. The behavior of a Python scalar function is determined by the evaluation method which is named `eval`.
-Note By default PyFlink uses the command “python” to run the python udf workers. Before starting cluster, run the following command to confirm that it meets the requirements:
-
-{% highlight bash %}
-$ python --version
-# the version printed here must be 3.5+
-$ python -m pip install apache-beam==2.19.0
-{% endhighlight %}
-
-It supports to use both Java/Scala scalar functions and Python scalar functions in Python Table API and SQL. In order to define a Python scalar function, one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. The behavior of a Python scalar function is determined by the evaluation method. An evaluation method must be named `eval`. Evaluation method can also support variable arguments, such as `eval(*args)`.
-
-The following example shows how to define your own Java and Python hash code functions, register them in the TableEnvironment, and call them in a query. Note that you can configure your scalar function via a constructor before it is registered:
+The following example shows how to define your own Python hash code function, register it in the TableEnvironment, and call it in a query. Note that you can configure your scalar function via a constructor before it is registered:
{% highlight python %}
-'''
-Java code:
-
-// The Java class must have a public no-argument constructor and can be founded in current Java classloader.
-public class HashCode extends ScalarFunction {
- private int factor = 12;
-
- public int eval(String s) {
- return s.hashCode() * factor;
- }
-}
-'''
-
-class PyHashCode(ScalarFunction):
+class HashCode(ScalarFunction):
def __init__(self):
self.factor = 12
@@ -171,128 +148,18 @@ class PyHashCode(ScalarFunction):
table_env = BatchTableEnvironment.create(env)
-# register the Java function
-table_env.register_java_function("hashCode", "my.java.function.HashCode")
-
# register the Python function
-table_env.register_function("py_hash_code", udf(PyHashCode(), DataTypes.BIGINT(), DataTypes.BIGINT()))
+table_env.register_function("hash_code", udf(HashCode(), DataTypes.BIGINT(), DataTypes.BIGINT()))
# use the function in Python Table API
-my_table.select("string, bigint, string.hashCode(), hashCode(string), bigint.py_hash_code(), py_hash_code(bigint)")
+my_table.select("string, bigint, string.hash_code(), hash_code(string)")
# use the function in SQL API
-table_env.sql_query("SELECT string, bigint, hashCode(string), py_hash_code(bigint) FROM MyTable")
+table_env.sql_query("SELECT string, bigint, hash_code(bigint) FROM MyTable")
{% endhighlight %}
-There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`. The following example shows the different ways to define a Python scalar function which takes two columns of bigint as input parameters and returns the sum of them as the result.
-
-{% highlight python %}
-# option 1: extending the base class `ScalarFunction`
-class Add(ScalarFunction):
- def eval(self, i, j):
- return i + j
-
-add = udf(Add(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
-
-# option 2: Python function
-@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT())
-def add(i, j):
- return i + j
-
-# option 3: lambda function
-add = udf(lambda i, j: i + j, [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
-
-# option 4: callable function
-class CallableAdd(object):
- def __call__(self, i, j):
- return i + j
-
-add = udf(CallableAdd(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
-
-# option 5: partial function
-def partial_add(i, j, k):
- return i + j + k
-
-add = udf(functools.partial(partial_add, k=1), [DataTypes.BIGINT(), DataTypes.BIGINT()],
- DataTypes.BIGINT())
-
-# register the Python function
-table_env.register_function("add", add)
-# use the function in Python Table API
-my_table.select("add(a, b)")
-{% endhighlight %}
-
-If the python scalar function depends on third-party dependencies, you can specify the dependencies with the following table APIs or through command line directly when submitting the job.
-
-
-
-
-
APIs
-
Description
-
-
-
-
-
-
add_python_file(file_path)
-
-
Adds python file dependencies which could be python files, python packages or local directories. They will be added to the PYTHONPATH of the python UDF worker.
Specifies a requirements.txt file which defines the third-party dependencies. These dependencies will be installed to a temporary directory and added to the PYTHONPATH of the python UDF worker. For the dependencies which could not be accessed in the cluster, a directory which contains the installation packages of these dependencies could be specified using the parameter "requirements_cached_dir". It will be uploaded to the cluster to support offline installation.
Please make sure the installation packages matches the platform of the cluster and the python version used. These packages will be installed using pip, so also make sure the version of Pip (version >= 7.1.0) and the version of SetupTools (version >= 37.0.0).
-
-
-
-
add_python_archive(archive_path, target_dir=None)
-
-
Adds a python archive file dependency. The file will be extracted to the working directory of python UDF worker. If the parameter "target_dir" is specified, the archive file will be extracted to a directory named "target_dir". Otherwise, the archive file will be extracted to a directory with the same name of the archive file.
-{% highlight python %}
-# command executed in shell
-# assert the relative path of python interpreter is py_env/bin/python
-zip -r py_env.zip py_env
-
-# python code
-table_env.add_python_archive("py_env.zip")
-# or
-table_env.add_python_archive("py_env.zip", "myenv")
-
-# the files contained in the archive file can be accessed in UDF
-def my_udf():
- with open("myenv/py_env/data/data.txt") as f:
- ...
-{% endhighlight %}
-
Please make sure the uploaded python environment matches the platform that the cluster is running on. Currently only zip-format is supported. i.e. zip, jar, whl, egg, etc.
-
-
-
-
set_python_executable(python_exec)
-
-
Sets the path of the python interpreter which is used to execute the python udf workers, e.g., "/usr/local/bin/python3".
Please make sure that the specified environment matches the platform that the cluster is running on.
-
-
-
-
-
+There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`.
+Please refer to the [Python Scalar Function]({{ site.baseurl }}/dev/table/python/python_udfs.html#scalar-functions) documentation for more details.
@@ -303,14 +170,14 @@ Table Functions
Similar to a user-defined scalar function, a user-defined table function takes zero, one, or multiple scalar values as input parameters. However in contrast to a scalar function, it can return an arbitrary number of rows as output instead of a single value. The returned rows may consist of one or more columns.
+
+
In order to define a table function one has to extend the base class `TableFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a table function is determined by its evaluation methods. An evaluation method must be declared `public` and named `eval`. The `TableFunction` can be overloaded by implementing multiple methods named `eval`. The parameter types of the evaluation methods determine all valid parameters of the table function. Evaluation methods can also support variable arguments, such as `eval(String... strs)`. The type of the returned table is determined by the generic type of `TableFunction`. Evaluation methods emit output rows using the protected `collect(T)` method.
In the Table API, a table function is used with `.joinLateral` or `.leftOuterJoinLateral`. The `joinLateral` operator (cross) joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator). The `leftOuterJoinLateral` operator joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator) and preserves outer rows for which the table function returns an empty table. In SQL use `LATERAL TABLE()` with CROSS JOIN and LEFT JOIN with an ON TRUE join condition (see examples below).
The following example shows how to define table-valued function, register it in the TableEnvironment, and call it in a query. Note that you can configure your table function via a constructor before it is registered:
-
-
{% highlight java %}
// The generic type "Tuple2" determines the schema of the returned table as (String, Integer).
public class Split extends TableFunction> {
@@ -349,6 +216,12 @@ tableEnv.sqlQuery("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(s
+In order to define a table function one has to extend the base class `TableFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a table function is determined by its evaluation methods. An evaluation method must be declared `public` and named `eval`. The `TableFunction` can be overloaded by implementing multiple methods named `eval`. The parameter types of the evaluation methods determine all valid parameters of the table function. Evaluation methods can also support variable arguments, such as `eval(String... strs)`. The type of the returned table is determined by the generic type of `TableFunction`. Evaluation methods emit output rows using the protected `collect(T)` method.
+
+In the Table API, a table function is used with `.joinLateral` or `.leftOuterJoinLateral`. The `joinLateral` operator (cross) joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator). The `leftOuterJoinLateral` operator joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator) and preserves outer rows for which the table function returns an empty table. In SQL use `LATERAL TABLE()` with CROSS JOIN and LEFT JOIN with an ON TRUE join condition (see examples below).
+
+The following example shows how to define table-valued function, register it in the TableEnvironment, and call it in a query. Note that you can configure your table function via a constructor before it is registered:
+
{% highlight scala %}
// The generic type "(String, Int)" determines the schema of the returned table as (String, Integer).
class Split(separator: String) extends TableFunction[(String, Int)] {
@@ -380,53 +253,51 @@ tableEnv.sqlQuery("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(s
-{% highlight python %}
-'''
-Java code:
+In order to define a Python table function, one can extend the base class `TableFunction` in `pyflink.table.udtf` and Implement an evaluation method. The behavior of a Python table function is determined by the evaluation method which is named eval.
-// The generic type "Tuple2" determines the schema of the returned table as (String, Integer).
-// The java class must have a public no-argument constructor and can be founded in current java classloader.
-public class Split extends TableFunction> {
- private String separator = " ";
-
- public void eval(String str) {
- for (String s : str.split(separator)) {
- // use collect(...) to emit a row
- collect(new Tuple2(s, s.length()));
- }
- }
-}
-'''
+In the Python Table API, a Python table function is used with `.join_lateral` or `.left_outer_join_lateral`. The `join_lateral` operator (cross) joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator). The `left_outer_join_lateral` operator joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator) and preserves outer rows for which the table function returns an empty table. In SQL use `LATERAL TABLE()` with CROSS JOIN and LEFT JOIN with an ON TRUE join condition (see examples below).
-table_env = BatchTableEnvironment.create(env)
-my_table = ... # type: Table, table schema: [a: String]
+Note Currently, Python UDTF is supported in old planner both under streaming and batch mode while is only supported under streaming mode in Blink planner.
-# Register the java function.
-table_env.register_java_function("split", "my.java.function.Split")
+The following example shows how to define a Python table function, registered it in the TableEnvironment, and call it in a query. Note that you can configure your table function via a constructor before it is registered:
-# Use the table function in the Python Table API. "as" specifies the field names of the table.
-my_table.join_lateral("split(a) as (word, length)").select("a, word, length")
-my_table.left_outer_join_lateral("split(a) as (word, length)").select("a, word, length")
+{% highlight python %}
+class Split(TableFunction):
+ def eval(self, string):
+ for s in string.split(" "):
+ yield s, len(s)
-# Register the python function.
+env = StreamExecutionEnvironment.get_execution_environment()
+table_env = StreamTableEnvironment.create(env)
+my_table = ... # type: Table, table schema: [a: String]
-# Use the table function in SQL with LATERAL and TABLE keywords.
-# CROSS JOIN a table function (equivalent to "join" in Table API).
+# register the Python Table Function
+table_env.register_function("split", udtf(Split(), DataTypes.STRING(), [DataTypes.STRING(), DataTypes.INT()]))
+
+# use the Python Table Function in Python Table API
+my_table.join_lateral("split(a) as (word, length)")
+my_table.left_outer_join_lateral("split(a) as (word, length)")
+
+# use the Python Table function in SQL API
table_env.sql_query("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)")
-# LEFT JOIN a table function (equivalent to "left_outer_join" in Table API).
table_env.sql_query("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(split(a)) as T(word, length) ON TRUE")
+
{% endhighlight %}
+
+There are many ways to define a Python table function besides extending the base class `TableFunction`.
+Please refer to the [Python Table Function]({{ site.baseurl }}/dev/table/python/python_udfs.html#table-functions) documentation for more details.
+
+
+
Please note that POJO types do not have a deterministic field order. Therefore, you cannot rename the fields of POJO returned by a table function using `AS`.
By default the result type of a `TableFunction` is determined by Flink’s automatic type extraction facilities. This works well for basic types and simple POJOs but might be wrong for more complex, custom, or composite types. In such a case, the type of the result can be manually specified by overriding `TableFunction#getResultType()` which returns its `TypeInformation`.
The following example shows an example of a `TableFunction` that returns a `Row` type which requires explicit type information. We define that the returned table type should be `RowTypeInfo(String, Integer)` by overriding `TableFunction#getResultType()`.
-
-
{% highlight java %}
public class CustomTypeSplit extends TableFunction {
public void eval(String str) {
@@ -447,6 +318,12 @@ public class CustomTypeSplit extends TableFunction {
+Please note that POJO types do not have a deterministic field order. Therefore, you cannot rename the fields of POJO returned by a table function using `AS`.
+
+By default the result type of a `TableFunction` is determined by Flink’s automatic type extraction facilities. This works well for basic types and simple POJOs but might be wrong for more complex, custom, or composite types. In such a case, the type of the result can be manually specified by overriding `TableFunction#getResultType()` which returns its `TypeInformation`.
+
+The following example shows an example of a `TableFunction` that returns a `Row` type which requires explicit type information. We define that the returned table type should be `RowTypeInfo(String, Integer)` by overriding `TableFunction#getResultType()`.
+
{% highlight scala %}
class CustomTypeSplit extends TableFunction[Row] {
def eval(str: String): Unit = {
diff --git a/docs/dev/table/functions/udfs.zh.md b/docs/dev/table/functions/udfs.zh.md
index 5eec2f169233c10b6748b55f849fcee8852b9250..1f5ad8dcbf273aed20d44069c550f941d9a16ae0 100644
--- a/docs/dev/table/functions/udfs.zh.md
+++ b/docs/dev/table/functions/udfs.zh.md
@@ -134,35 +134,12 @@ object TimestampModifier extends ScalarFunction {
-Note Python 3.5+ and apache-beam==2.19.0 are required to run the Python scalar function.
+In order to define a Python scalar function, one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. The behavior of a Python scalar function is determined by the evaluation method which is named `eval`.
-Note By default PyFlink uses the command “python” to run the python udf workers. Before starting cluster, run the following command to confirm that it meets the requirements:
-
-{% highlight bash %}
-$ python --version
-# the version printed here must be 3.5+
-$ python -m pip install apache-beam==2.19.0
-{% endhighlight %}
-
-It supports to use both Java/Scala scalar functions and Python scalar functions in Python Table API and SQL. In order to define a Python scalar function, one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. The behavior of a Python scalar function is determined by the evaluation method. An evaluation method must be named `eval`. Evaluation method can also support variable arguments, such as `eval(*args)`.
-
-The following example shows how to define your own Java and Python hash code functions, register them in the TableEnvironment, and call them in a query. Note that you can configure your scalar function via a constructor before it is registered:
+The following example shows how to define your own Python hash code function, register it in the TableEnvironment, and call it in a query. Note that you can configure your scalar function via a constructor before it is registered:
{% highlight python %}
-'''
-Java code:
-
-// The Java class must have a public no-argument constructor and can be founded in current Java classloader.
-public class HashCode extends ScalarFunction {
- private int factor = 12;
-
- public int eval(String s) {
- return s.hashCode() * factor;
- }
-}
-'''
-
-class PyHashCode(ScalarFunction):
+class HashCode(ScalarFunction):
def __init__(self):
self.factor = 12
@@ -171,128 +148,18 @@ class PyHashCode(ScalarFunction):
table_env = BatchTableEnvironment.create(env)
-# register the Java function
-table_env.register_java_function("hashCode", "my.java.function.HashCode")
-
# register the Python function
-table_env.register_function("py_hash_code", udf(PyHashCode(), DataTypes.BIGINT(), DataTypes.BIGINT()))
+table_env.register_function("hash_code", udf(HashCode(), DataTypes.BIGINT(), DataTypes.BIGINT()))
# use the function in Python Table API
-my_table.select("string, bigint, string.hashCode(), hashCode(string), bigint.py_hash_code(), py_hash_code(bigint)")
+my_table.select("string, bigint, string.hash_code(), hash_code(string)")
# use the function in SQL API
-table_env.sql_query("SELECT string, bigint, hashCode(string), py_hash_code(bigint) FROM MyTable")
-{% endhighlight %}
-
-There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`. The following example shows the different ways to define a Python scalar function which takes two columns of bigint as input parameters and returns the sum of them as the result.
-
-{% highlight python %}
-# option 1: extending the base class `ScalarFunction`
-class Add(ScalarFunction):
- def eval(self, i, j):
- return i + j
-
-add = udf(Add(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
-
-# option 2: Python function
-@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT())
-def add(i, j):
- return i + j
-
-# option 3: lambda function
-add = udf(lambda i, j: i + j, [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
-
-# option 4: callable function
-class CallableAdd(object):
- def __call__(self, i, j):
- return i + j
-
-add = udf(CallableAdd(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
-
-# option 5: partial function
-def partial_add(i, j, k):
- return i + j + k
-
-add = udf(functools.partial(partial_add, k=1), [DataTypes.BIGINT(), DataTypes.BIGINT()],
- DataTypes.BIGINT())
-
-# register the Python function
-table_env.register_function("add", add)
-# use the function in Python Table API
-my_table.select("add(a, b)")
-{% endhighlight %}
-
-If the python scalar function depends on third-party dependencies, you can specify the dependencies with the following table APIs or through command line directly when submitting the job.
-
-
-
-
-
APIs
-
Description
-
-
-
-
-
-
add_python_file(file_path)
-
-
Adds python file dependencies which could be python files, python packages or local directories. They will be added to the PYTHONPATH of the python UDF worker.
Specifies a requirements.txt file which defines the third-party dependencies. These dependencies will be installed to a temporary directory and added to the PYTHONPATH of the python UDF worker. For the dependencies which could not be accessed in the cluster, a directory which contains the installation packages of these dependencies could be specified using the parameter "requirements_cached_dir". It will be uploaded to the cluster to support offline installation.
Please make sure the installation packages matches the platform of the cluster and the python version used. These packages will be installed using pip, so also make sure the version of Pip (version >= 7.1.0) and the version of SetupTools (version >= 37.0.0).
-
-
-
-
add_python_archive(archive_path, target_dir=None)
-
-
Adds a python archive file dependency. The file will be extracted to the working directory of python UDF worker. If the parameter "target_dir" is specified, the archive file will be extracted to a directory named "target_dir". Otherwise, the archive file will be extracted to a directory with the same name of the archive file.
-{% highlight python %}
-# command executed in shell
-# assert the relative path of python interpreter is py_env/bin/python
-zip -r py_env.zip py_env
-
-# python code
-table_env.add_python_archive("py_env.zip")
-# or
-table_env.add_python_archive("py_env.zip", "myenv")
-
-# the files contained in the archive file can be accessed in UDF
-def my_udf():
- with open("myenv/py_env/data/data.txt") as f:
- ...
-{% endhighlight %}
-
Please make sure the uploaded python environment matches the platform that the cluster is running on. Currently only zip-format is supported. i.e. zip, jar, whl, egg, etc.
-
-
-
-
set_python_executable(python_exec)
-
-
Sets the path of the python interpreter which is used to execute the python udf workers, e.g., "/usr/local/bin/python3".
Please make sure that the specified environment matches the platform that the cluster is running on.
-
-
-
-
+There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`.
+Please refer to the [Python Scalar Function]({{ site.baseurl }}/zh/dev/table/python/python_udfs.html#scalar-functions) documentation for more details.
@@ -303,14 +170,15 @@ Table Functions
Similar to a user-defined scalar function, a user-defined table function takes zero, one, or multiple scalar values as input parameters. However in contrast to a scalar function, it can return an arbitrary number of rows as output instead of a single value. The returned rows may consist of one or more columns.
+
+
+
In order to define a table function one has to extend the base class `TableFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a table function is determined by its evaluation methods. An evaluation method must be declared `public` and named `eval`. The `TableFunction` can be overloaded by implementing multiple methods named `eval`. The parameter types of the evaluation methods determine all valid parameters of the table function. Evaluation methods can also support variable arguments, such as `eval(String... strs)`. The type of the returned table is determined by the generic type of `TableFunction`. Evaluation methods emit output rows using the protected `collect(T)` method.
In the Table API, a table function is used with `.joinLateral` or `.leftOuterJoinLateral`. The `joinLateral` operator (cross) joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator). The `leftOuterJoinLateral` operator joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator) and preserves outer rows for which the table function returns an empty table. In SQL use `LATERAL TABLE()` with CROSS JOIN and LEFT JOIN with an ON TRUE join condition (see examples below).
The following example shows how to define table-valued function, register it in the TableEnvironment, and call it in a query. Note that you can configure your table function via a constructor before it is registered:
-
-
{% highlight java %}
// The generic type "Tuple2" determines the schema of the returned table as (String, Integer).
public class Split extends TableFunction> {
@@ -349,6 +217,13 @@ tableEnv.sqlQuery("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(s
+
+In order to define a table function one has to extend the base class `TableFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a table function is determined by its evaluation methods. An evaluation method must be declared `public` and named `eval`. The `TableFunction` can be overloaded by implementing multiple methods named `eval`. The parameter types of the evaluation methods determine all valid parameters of the table function. Evaluation methods can also support variable arguments, such as `eval(String... strs)`. The type of the returned table is determined by the generic type of `TableFunction`. Evaluation methods emit output rows using the protected `collect(T)` method.
+
+In the Table API, a table function is used with `.joinLateral` or `.leftOuterJoinLateral`. The `joinLateral` operator (cross) joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator). The `leftOuterJoinLateral` operator joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator) and preserves outer rows for which the table function returns an empty table. In SQL use `LATERAL TABLE()` with CROSS JOIN and LEFT JOIN with an ON TRUE join condition (see examples below).
+
+The following example shows how to define table-valued function, register it in the TableEnvironment, and call it in a query. Note that you can configure your table function via a constructor before it is registered:
+
{% highlight scala %}
// The generic type "(String, Int)" determines the schema of the returned table as (String, Integer).
class Split(separator: String) extends TableFunction[(String, Int)] {
@@ -380,53 +255,51 @@ tableEnv.sqlQuery("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(s
-{% highlight python %}
-'''
-Java code:
+In order to define a Python table function, one can extend the base class `TableFunction` in `pyflink.table.udtf` and Implement an evaluation method. The behavior of a Python table function is determined by the evaluation method which is named eval.
-// The generic type "Tuple2" determines the schema of the returned table as (String, Integer).
-// The java class must have a public no-argument constructor and can be founded in current java classloader.
-public class Split extends TableFunction> {
- private String separator = " ";
-
- public void eval(String str) {
- for (String s : str.split(separator)) {
- // use collect(...) to emit a row
- collect(new Tuple2(s, s.length()));
- }
- }
-}
-'''
+In the Python Table API, a Python table function is used with `.join_lateral` or `.left_outer_join_lateral`. The `join_lateral` operator (cross) joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator). The `left_outer_join_lateral` operator joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator) and preserves outer rows for which the table function returns an empty table. In SQL use `LATERAL TABLE()` with CROSS JOIN and LEFT JOIN with an ON TRUE join condition (see examples below).
-table_env = BatchTableEnvironment.create(env)
-my_table = ... # type: Table, table schema: [a: String]
+Note Currently, Python UDTF is supported in old planner both under streaming and batch mode while is only supported under streaming mode in Blink planner.
-# Register the java function.
-table_env.register_java_function("split", "my.java.function.Split")
+The following example shows how to define a Python table function, registered it in the TableEnvironment, and call it in a query. Note that you can configure your table function via a constructor before it is registered:
-# Use the table function in the Python Table API. "as" specifies the field names of the table.
-my_table.join_lateral("split(a) as (word, length)").select("a, word, length")
-my_table.left_outer_join_lateral("split(a) as (word, length)").select("a, word, length")
+{% highlight python %}
+class Split(TableFunction):
+ def eval(self, string):
+ for s in string.split(" "):
+ yield s, len(s)
-# Register the python function.
+env = StreamExecutionEnvironment.get_execution_environment()
+table_env = StreamTableEnvironment.create(env)
+my_table = ... # type: Table, table schema: [a: String]
+
+# register the Python Table Function
+table_env.register_function("split", udtf(Split(), DataTypes.STRING(), [DataTypes.STRING(), DataTypes.INT()]))
+
+# use the Python Table Function in Python Table API
+my_table.join_lateral("split(a) as (word, length)")
+my_table.left_outer_join_lateral("split(a) as (word, length)")
-# Use the table function in SQL with LATERAL and TABLE keywords.
-# CROSS JOIN a table function (equivalent to "join" in Table API).
+# use the Python Table function in SQL API
table_env.sql_query("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)")
-# LEFT JOIN a table function (equivalent to "left_outer_join" in Table API).
table_env.sql_query("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(split(a)) as T(word, length) ON TRUE")
+
{% endhighlight %}
+
+There are many ways to define a Python table function besides extending the base class `TableFunction`.
+Please refer to the [Python Table Function]({{ site.baseurl }}/zh/dev/table/python/python_udfs.html#table-functions) documentation for more details.
+
+
+
Please note that POJO types do not have a deterministic field order. Therefore, you cannot rename the fields of POJO returned by a table function using `AS`.
By default the result type of a `TableFunction` is determined by Flink’s automatic type extraction facilities. This works well for basic types and simple POJOs but might be wrong for more complex, custom, or composite types. In such a case, the type of the result can be manually specified by overriding `TableFunction#getResultType()` which returns its `TypeInformation`.
The following example shows an example of a `TableFunction` that returns a `Row` type which requires explicit type information. We define that the returned table type should be `RowTypeInfo(String, Integer)` by overriding `TableFunction#getResultType()`.
-
-
{% highlight java %}
public class CustomTypeSplit extends TableFunction {
public void eval(String str) {
@@ -447,6 +320,12 @@ public class CustomTypeSplit extends TableFunction {
+Please note that POJO types do not have a deterministic field order. Therefore, you cannot rename the fields of POJO returned by a table function using `AS`.
+
+By default the result type of a `TableFunction` is determined by Flink’s automatic type extraction facilities. This works well for basic types and simple POJOs but might be wrong for more complex, custom, or composite types. In such a case, the type of the result can be manually specified by overriding `TableFunction#getResultType()` which returns its `TypeInformation`.
+
+The following example shows an example of a `TableFunction` that returns a `Row` type which requires explicit type information. We define that the returned table type should be `RowTypeInfo(String, Integer)` by overriding `TableFunction#getResultType()`.
+
{% highlight scala %}
class CustomTypeSplit extends TableFunction[Row] {
def eval(str: String): Unit = {
diff --git a/docs/dev/table/hive/hive_catalog.md b/docs/dev/table/hive/hive_catalog.md
index 92b05cbbcbc2ebcba0b820e7808756dd88517b3d..d907703fe24b7e45bd5945dac92c0d26c52c74a2 100644
--- a/docs/dev/table/hive/hive_catalog.md
+++ b/docs/dev/table/hive/hive_catalog.md
@@ -153,7 +153,6 @@ catalogs:
- name: myhive
type: hive
hive-conf-dir: /opt/hive-conf # contains hive-site.xml
- hive-version: 2.3.4
{% endhighlight %}
@@ -191,7 +190,6 @@ Flink SQL> CREATE TABLE mykafka (name String, age Int) WITH (
'connector.type' = 'kafka',
'connector.version' = 'universal',
'connector.topic' = 'test',
- 'connector.properties.zookeeper.connect' = 'localhost:2181',
'connector.properties.bootstrap.servers' = 'localhost:9092',
'format.type' = 'csv',
'update-mode' = 'append'
@@ -228,7 +226,6 @@ Location: ......
Table Type: MANAGED_TABLE
Table Parameters:
flink.connector.properties.bootstrap.servers localhost:9092
- flink.connector.properties.zookeeper.connect localhost:2181
flink.connector.topic test
flink.connector.type kafka
flink.connector.version universal
diff --git a/docs/dev/table/hive/hive_catalog.zh.md b/docs/dev/table/hive/hive_catalog.zh.md
index 92b05cbbcbc2ebcba0b820e7808756dd88517b3d..d907703fe24b7e45bd5945dac92c0d26c52c74a2 100644
--- a/docs/dev/table/hive/hive_catalog.zh.md
+++ b/docs/dev/table/hive/hive_catalog.zh.md
@@ -153,7 +153,6 @@ catalogs:
- name: myhive
type: hive
hive-conf-dir: /opt/hive-conf # contains hive-site.xml
- hive-version: 2.3.4
{% endhighlight %}
@@ -191,7 +190,6 @@ Flink SQL> CREATE TABLE mykafka (name String, age Int) WITH (
'connector.type' = 'kafka',
'connector.version' = 'universal',
'connector.topic' = 'test',
- 'connector.properties.zookeeper.connect' = 'localhost:2181',
'connector.properties.bootstrap.servers' = 'localhost:9092',
'format.type' = 'csv',
'update-mode' = 'append'
@@ -228,7 +226,6 @@ Location: ......
Table Type: MANAGED_TABLE
Table Parameters:
flink.connector.properties.bootstrap.servers localhost:9092
- flink.connector.properties.zookeeper.connect localhost:2181
flink.connector.topic test
flink.connector.type kafka
flink.connector.version universal
diff --git a/docs/dev/table/hive/hive_functions.md b/docs/dev/table/hive/hive_functions.md
index e2b7ac51b6b6ca255f4f665ec3fefe721f65fd72..537d684aa37d71b4e15eaaf0fafc9a1e22652d51 100644
--- a/docs/dev/table/hive/hive_functions.md
+++ b/docs/dev/table/hive/hive_functions.md
@@ -54,7 +54,6 @@ modules:
type: core
- name: myhive
type: hive
- hive-version: 2.3.4
{% endhighlight %}
diff --git a/docs/dev/table/hive/index.md b/docs/dev/table/hive/index.md
index 42e825633f23fba0f5946b050d70c18042cafe63..3698df2571f84572545b955993b8c75b21b666be 100644
--- a/docs/dev/table/hive/index.md
+++ b/docs/dev/table/hive/index.md
@@ -92,6 +92,32 @@ to make the integration work in Table API program or SQL in SQL Client.
Alternatively, you can put these dependencies in a dedicated folder, and add them to classpath with the `-C`
or `-l` option for Table API program or SQL Client respectively.
+Apache Hive is built on Hadoop, so you need Hadoop dependency first, please refer to
+[Providing Hadoop classes]({{ site.baseurl }}/ops/deployment/hadoop.html#providing-hadoop-classes).
+
+There are two ways to add Hive dependencies. First is to use Flink's bundled Hive jars. You can choose a bundled Hive jar according to the version of the metastore you use. Second is to add each of the required jars separately. The second way can be useful if the Hive version you're using is not listed here.
+
+#### Using bundled hive jar
+
+The following tables list all available bundled hive jars. You can pick one to the `/lib/` directory in Flink distribution.
+
+{% if site.is_stable %}
+
+| Metastore version | Maven dependency | SQL Client JAR |
+| :---------------- | :--------------------------- | :----------------------|
+| 1.0.0 - 1.2.2 | `flink-connector-hive-1.2.2` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-1.2.2{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-1.2.2{{site.scala_version_suffix}}-{{site.version}}.jar) |
+| 2.0.0 - 2.2.0 | `flink-connector-hive-2.2.0` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-2.2.0{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-2.2.0{{site.scala_version_suffix}}-{{site.version}}.jar) |
+| 2.3.0 - 2.3.6 | `flink-connector-hive-2.3.6` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-2.3.6{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-2.3.6{{site.scala_version_suffix}}-{{site.version}}.jar) |
+| 3.0.0 - 3.1.2 | `flink-connector-hive-3.1.2` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-3.1.2{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-3.1.2{{site.scala_version_suffix}}-{{site.version}}.jar) |
+
+{% else %}
+
+These tables are only available for stable releases.
+
+{% endif %}
+
+#### User defined dependencies
+
Please find the required dependencies for different Hive major versions below.
@@ -105,12 +131,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector.Contains flink-hadoop-compatibility and flink-orc jars
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-exec-2.3.4.jar
@@ -125,12 +145,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-metastore-1.0.0.jar
hive-exec-1.0.0.jar
@@ -151,12 +165,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-metastore-1.1.0.jar
hive-exec-1.1.0.jar
@@ -177,12 +185,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-metastore-1.2.1.jar
hive-exec-1.2.1.jar
@@ -203,12 +205,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-exec-2.0.0.jar
@@ -223,12 +219,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-exec-2.1.0.jar
@@ -243,12 +233,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-exec-2.2.0.jar
@@ -267,12 +251,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.8.3-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-exec-3.1.0.jar
libfb303-0.9.3.jar // libfb303 is not packed into hive-exec in some versions, need to add it separately
@@ -281,6 +259,11 @@ Please find the required dependencies for different Hive major versions below.
+If you use the hive version of HDP or CDH, you need to refer to the dependency in the previous section and select a similar version.
+
+And you need to specify selected and supported "hive-version" in yaml, HiveCatalog and HiveModule.
+
+### Program maven
If you are building your own program, you need the following dependencies in your mvn file.
It's recommended not to include these dependencies in the resulting jar file.
@@ -374,7 +357,6 @@ catalogs:
- name: myhive
type: hive
hive-conf-dir: /opt/hive-conf
- hive-version: 2.3.4
{% endhighlight %}
@@ -387,4 +369,3 @@ DDL to create Hive tables, views, partitions, functions within Flink will be sup
## DML
Flink supports DML writing to Hive tables. Please refer to details in [Reading & Writing Hive Tables]({{ site.baseurl }}/dev/table/hive/read_write_hive.html)
-
diff --git a/docs/dev/table/hive/index.zh.md b/docs/dev/table/hive/index.zh.md
index 925dce1326bd37ae056d87ede40c56255e9b2c10..8136a3b7ce8df3501697dad6f04290ef5d6cd799 100644
--- a/docs/dev/table/hive/index.zh.md
+++ b/docs/dev/table/hive/index.zh.md
@@ -1,5 +1,5 @@
---
-title: "Hive Integration"
+title: "Hive 集成"
nav-id: hive_tableapi
nav-parent_id: tableapi
nav-pos: 100
@@ -24,27 +24,27 @@ specific language governing permissions and limitations
under the License.
-->
-[Apache Hive](https://hive.apache.org/) has established itself as a focal point of the data warehousing ecosystem.
-It serves as not only a SQL engine for big data analytics and ETL, but also a data management platform, where data is discovered, defined, and evolved.
+[Apache Hive](https://hive.apache.org/) 已经成为了数据仓库生态系统中的核心。
+它不仅仅是一个用于大数据分析和ETL场景的SQL引擎,同样它也是一个数据管理平台,可用于发现,定义,和演化数据。
-Flink offers a two-fold integration with Hive.
+Flink 与 Hive 的集成包含两个层面。
-The first is to leverage Hive's Metastore as a persistent catalog with Flink's `HiveCatalog` for storing Flink specific metadata across sessions.
-For example, users can store their Kafka or ElasticSearch tables in Hive Metastore by using `HiveCatalog`, and reuse them later on in SQL queries.
+一是利用了 Hive 的 MetaStore 作为持久化的 Catalog,用户可通过`HiveCatalog`将不同会话中的 Flink 元数据存储到 Hive Metastore 中。
+例如,用户可以使用`HiveCatalog`将其 Kafka 表或 Elasticsearch 表存储在 Hive Metastore 中,并后续在 SQL 查询中重新使用它们。
-The second is to offer Flink as an alternative engine for reading and writing Hive tables.
+二是利用 Flink 来读写 Hive 的表。
-The `HiveCatalog` is designed to be “out of the box” compatible with existing Hive installations.
-You do not need to modify your existing Hive Metastore or change the data placement or partitioning of your tables.
+`HiveCatalog`的设计提供了与 Hive 良好的兼容性,用户可以"开箱即用"的访问其已有的 Hive 数仓。
+您不需要修改现有的 Hive Metastore,也不需要更改表的数据位置或分区。
-* Note that we highly recommend users using the [blink planner]({{ site.baseurl }}/dev/table/#dependency-structure) with Hive integration.
+* 我们强烈建议用户使用 [Blink planner]({{ site.baseurl }}/zh/dev/table/#dependency-structure) 与 Hive 集成。
* This will be replaced by the TOC
{:toc}
-## Supported Hive Versions
+## 支持的Hive版本
-Flink supports the following Hive versions.
+Flink 支持一下的 Hive 版本。
- 1.0
- 1.0.0
@@ -77,22 +77,48 @@ Flink supports the following Hive versions.
- 3.1.1
- 3.1.2
-Please note Hive itself have different features available for different versions, and these issues are not caused by Flink:
+请注意,某些功能是否可用取决于您使用的 Hive 版本,这些限制不是由 Flink 所引起的:
-- Hive built-in functions are supported in 1.2.0 and later.
-- Column constraints, i.e. PRIMARY KEY and NOT NULL, are supported in 3.1.0 and later.
-- Altering table statistics is supported in 1.2.0 and later.
-- `DATE` column statistics are supported in 1.2.0 and later.
-- Writing to ORC tables is not supported in 2.0.x.
+- Hive 内置函数在使用 Hive-1.2.0 及更高版本时支持。
+- 列约束,也就是 PRIMARY KEY 和 NOT NULL,在使用 Hive-3.1.0 及更高版本时支持。
+- 更改表的统计信息,在使用 Hive-1.2.0 及更高版本时支持。
+- `DATE`列统计信息,在使用 Hive-1.2.0 及更高版时支持。
+- 使用 Hive-2.0.x 版本时不支持写入 ORC 表。
-### Dependencies
+### 依赖项
-To integrate with Hive, you need to add some extra dependencies to the `/lib/` directory in Flink distribution
-to make the integration work in Table API program or SQL in SQL Client.
-Alternatively, you can put these dependencies in a dedicated folder, and add them to classpath with the `-C`
-or `-l` option for Table API program or SQL Client respectively.
+要与 Hive 集成,您需要在 Flink 下的`/lib/`目录中添加一些额外的依赖包,
+以便通过 Table API 或 SQL Client 与 Hive 进行交互。
+或者,您可以将这些依赖项放在专用文件夹中,并分别使用 Table API 程序或 SQL Client 的`-C`或`-l`选项将它们添加到 classpath 中。
-Please find the required dependencies for different Hive major versions below.
+Apache Hive 是基于 Hadoop 之上构建的, 首先您需要 Hadoop 的依赖,请参考
+[Providing Hadoop classes]({{ site.baseurl }}/zh/ops/deployment/hadoop.html#providing-hadoop-classes).
+
+有两种添加 Hive 依赖项的方法。第一种是使用 Flink 提供的 Hive Jar包。您可以根据使用的 Metastore 的版本来选择对应的 Hive jar。第二个方式是分别添加每个所需的 jar 包。如果您使用的 Hive 版本尚未在此处列出,则第二种方法会更适合。
+
+#### 使用 Flink 提供的 Hive jar
+
+下表列出了所有可用的 Hive jar。您可以选择一个并放在 Flink 发行版的`/lib/` 目录中。
+
+
+{% if site.is_stable %}
+
+| Metastore 版本 | Maven 依赖 | SQL Client JAR |
+| :---------------- | :--------------------------- | :----------------------|
+| 1.0.0 - 1.2.2 | `flink-connector-hive-1.2.2` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-1.2.2{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-1.2.2{{site.scala_version_suffix}}-{{site.version}}.jar) |
+| 2.0.0 - 2.2.0 | `flink-connector-hive-2.2.0` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-2.2.0{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-2.2.0{{site.scala_version_suffix}}-{{site.version}}.jar) |
+| 2.3.0 - 2.3.6 | `flink-connector-hive-2.3.6` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-2.3.6{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-2.3.6{{site.scala_version_suffix}}-{{site.version}}.jar) |
+| 3.0.0 - 3.1.2 | `flink-connector-hive-3.1.2` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-3.1.2{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-3.1.2{{site.scala_version_suffix}}-{{site.version}}.jar) |
+
+{% else %}
+
+下表仅适用与稳定版本。
+
+{% endif %}
+
+#### 用户定义的依赖项
+
+您可以在下方找到不同Hive主版本所需要的依赖项。
@@ -105,12 +131,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector.Contains flink-hadoop-compatibility and flink-orc jars
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-exec-2.3.4.jar
@@ -125,12 +145,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-metastore-1.0.0.jar
hive-exec-1.0.0.jar
@@ -151,12 +165,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-metastore-1.1.0.jar
hive-exec-1.1.0.jar
@@ -177,12 +185,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-metastore-1.2.1.jar
hive-exec-1.2.1.jar
@@ -203,12 +205,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-exec-2.0.0.jar
@@ -223,12 +219,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-exec-2.1.0.jar
@@ -243,12 +233,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-exec-2.2.0.jar
@@ -267,12 +251,6 @@ Please find the required dependencies for different Hive major versions below.
// Flink's Hive connector
flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar
- // Hadoop dependencies
- // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively
- // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop
- // cluster and the Hive version you're using.
- flink-shaded-hadoop-2-uber-2.8.3-{{ site.shaded_version }}.jar
-
// Hive dependencies
hive-exec-3.1.0.jar
libfb303-0.9.3.jar // libfb303 is not packed into hive-exec in some versions, need to add it separately
@@ -281,10 +259,14 @@ Please find the required dependencies for different Hive major versions below.
+如果使用 Hive 的 HDP 或 CDH 版本,则需要参考上一节中的依赖项并选择一个类似的版本。
+
+并且您需要在定义 yaml 文件,或者创建 HiveCatalog 和 HiveModule 时,指定一个支持的 “hive-version”。
+
+### Maven 依赖
-If you are building your own program, you need the following dependencies in your mvn file.
-It's recommended not to include these dependencies in the resulting jar file.
-You're supposed to add dependencies as stated above at runtime.
+如果您在构建自己的应用程序,则需要在 mvn 文件中添加以下依赖项。
+您应该在运行时添加以上的这些依赖项,而不要在已生成的 jar 文件中去包含它们。
{% highlight xml %}
@@ -311,18 +293,15 @@ You're supposed to add dependencies as stated above at runtime.
{% endhighlight %}
-## Connecting To Hive
+## 连接到Hive
-Connect to an existing Hive installation using the [catalog interface]({{ site.baseurl }}/dev/table/catalogs.html)
-and [HiveCatalog]({{ site.baseurl }}/dev/table/hive/hive_catalog.html) through the table environment or YAML configuration.
+通过 TableEnvironment 或者 YAML 配置,使用 [Catalog 接口]({{ site.baseurl }}/zh/dev/table/catalogs.html) 和 [HiveCatalog]({{ site.baseurl }}/zh/dev/table/hive/hive_catalog.html)连接到现有的 Hive 集群。
-If the `hive-conf/hive-site.xml` file is stored in remote storage system, users should download
-the hive configuration file to their local environment first.
+如果`hive-conf/hive-site.xml`文件存储在远端存储系统,则用户首先应该将hive配置文件下载至其本地环境中。
-Please note while HiveCatalog doesn't require a particular planner, reading/writing Hive tables only works with blink planner.
-Therefore it's highly recommended that you use blink planner when connecting to your Hive warehouse.
+请注意,虽然 HiveCatalog 不需要特定的 planner,但读写Hive表仅适用于 Blink planner。因此,强烈建议您在连接到 Hive 仓库时使用 Blink planner。
-Take Hive version 2.3.4 for example:
+以Hive 2.3.4版本为例:
@@ -382,8 +360,8 @@ catalogs:
## DDL
-DDL to create Hive tables, views, partitions, functions within Flink will be supported soon.
+即将支持在 Flink 中创建 Hive 表,视图,分区和函数的DDL。
## DML
-Flink supports DML writing to Hive tables. Please refer to details in [Reading & Writing Hive Tables]({{ site.baseurl }}/dev/table/hive/read_write_hive.html)
+Flink 支持 DML 写入 Hive 表,请参考[读写 Hive 表]({{ site.baseurl }}/zh/dev/table/hive/read_write_hive.html)
diff --git a/docs/dev/table/hive/read_write_hive.md b/docs/dev/table/hive/read_write_hive.md
index 07ee3b34d579cdcf222c2e74b85c1af1eaeb11cc..8aa83b91a07f55434e77d842e73b246485e7e461 100644
--- a/docs/dev/table/hive/read_write_hive.md
+++ b/docs/dev/table/hive/read_write_hive.md
@@ -171,13 +171,14 @@ It is especially beneficial when a table contains many columns.
For queries with LIMIT clause, Flink will limit the number of output records wherever possible to minimize the
amount of data transferred across network.
-### ORC Vectorized Optimization upon Read
+### Vectorized Optimization upon Read
Optimization is used automatically when the following conditions are met:
+- Format: ORC or Parquet.
- Columns without complex data type, like hive types: List, Map, Struct, Union.
-This feature is turned on by default. If there is a problem, you can use this config option to close ORC Vectorized Optimization:
+This feature is turned on by default. If there is a problem, you can use this config option to close Vectorized Optimization:
{% highlight bash %}
table.exec.hive.fallback-mapred-reader=true
diff --git a/docs/dev/table/hive/read_write_hive.zh.md b/docs/dev/table/hive/read_write_hive.zh.md
index 07ee3b34d579cdcf222c2e74b85c1af1eaeb11cc..8aa83b91a07f55434e77d842e73b246485e7e461 100644
--- a/docs/dev/table/hive/read_write_hive.zh.md
+++ b/docs/dev/table/hive/read_write_hive.zh.md
@@ -171,13 +171,14 @@ It is especially beneficial when a table contains many columns.
For queries with LIMIT clause, Flink will limit the number of output records wherever possible to minimize the
amount of data transferred across network.
-### ORC Vectorized Optimization upon Read
+### Vectorized Optimization upon Read
Optimization is used automatically when the following conditions are met:
+- Format: ORC or Parquet.
- Columns without complex data type, like hive types: List, Map, Struct, Union.
-This feature is turned on by default. If there is a problem, you can use this config option to close ORC Vectorized Optimization:
+This feature is turned on by default. If there is a problem, you can use this config option to close Vectorized Optimization:
{% highlight bash %}
table.exec.hive.fallback-mapred-reader=true
diff --git a/docs/dev/table/modules.md b/docs/dev/table/modules.md
index ec39a08c05fa9775f4c5db9c38ad5e1e573df4c6..245dbe43bd719e99381beb916fc2a6ee2b4a614a 100644
--- a/docs/dev/table/modules.md
+++ b/docs/dev/table/modules.md
@@ -105,12 +105,11 @@ modules:
type: core
- name: myhive
type: hive
- hive-version: 1.2.1
{% endhighlight %}
-### List Available Catalogs
+### List Available Modules
-### List Available Catalogs
+### List Available Modules
diff --git a/docs/dev/table/python/dependency_management.md b/docs/dev/table/python/dependency_management.md
new file mode 100644
index 0000000000000000000000000000000000000000..0a68633ec61c9d7a99862735fbacc8ec7d46b63a
--- /dev/null
+++ b/docs/dev/table/python/dependency_management.md
@@ -0,0 +1,110 @@
+---
+title: "Dependency Management"
+nav-parent_id: python_tableapi
+nav-pos: 100
+---
+
+
+# Java Dependency
+
+If third-party Java dependencies are used, you can specify the dependencies with the following Python Table APIs or through command line arguments directly when submitting the job.
+
+{% highlight python %}
+# Specify a list of jar URLs via "pipeline.jars". The jars are separated by ";" and will be uploaded to the cluster.
+# NOTE: Only local file URLs (start with "file://") are supported.
+table_env.get_config().set_configuration("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar")
+
+# Specify a list of URLs via "pipeline.classpaths". The URLs are separated by ";" and will be added to the classpath of the cluster.
+# NOTE: The Paths must specify a protocol (e.g. file://) and users should ensure that the URLs are accessible on both the client and the cluster.
+table_env.get_config().set_configuration("pipeline.classpaths", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar")
+{% endhighlight %}
+
+# Python Dependency
+
+If third-party Python dependencies are used, you can specify the dependencies with the following Python Table APIs or through command line arguments directly when submitting the job.
+
+
+
+
+
APIs
+
Description
+
+
+
+
+
+
add_python_file(file_path)
+
+
Adds python file dependencies which could be python files, python packages or local directories. They will be added to the PYTHONPATH of the python UDF worker.
Specifies a requirements.txt file which defines the third-party dependencies. These dependencies will be installed to a temporary directory and added to the PYTHONPATH of the python UDF worker. For the dependencies which could not be accessed in the cluster, a directory which contains the installation packages of these dependencies could be specified using the parameter "requirements_cached_dir". It will be uploaded to the cluster to support offline installation.
Please make sure the installation packages matches the platform of the cluster and the python version used. These packages will be installed using pip, so also make sure the version of Pip (version >= 7.1.0) and the version of SetupTools (version >= 37.0.0).
+
+
+
+
add_python_archive(archive_path, target_dir=None)
+
+
Adds a python archive file dependency. The file will be extracted to the working directory of python UDF worker. If the parameter "target_dir" is specified, the archive file will be extracted to a directory named "target_dir". Otherwise, the archive file will be extracted to a directory with the same name of the archive file.
+{% highlight python %}
+# command executed in shell
+# assert the relative path of python interpreter is py_env/bin/python
+zip -r py_env.zip py_env
+
+# python code
+table_env.add_python_archive("py_env.zip")
+# or
+table_env.add_python_archive("py_env.zip", "myenv")
+
+# the files contained in the archive file can be accessed in UDF
+def my_udf():
+ with open("myenv/py_env/data/data.txt") as f:
+ ...
+{% endhighlight %}
+
Please make sure the uploaded python environment matches the platform that the cluster is running on. Currently only zip-format is supported. i.e. zip, jar, whl, egg, etc.
+
+
+
+
set_python_executable(python_exec)
+
+
Sets the path of the python interpreter which is used to execute the python udf workers, e.g., "/usr/local/bin/python3".
Please make sure that the specified environment matches the platform that the cluster is running on.
+
+
+
+
diff --git a/docs/dev/table/python/dependency_management.zh.md b/docs/dev/table/python/dependency_management.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..edf39d520919ce3220ceab8ab4aeda50acb61c4f
--- /dev/null
+++ b/docs/dev/table/python/dependency_management.zh.md
@@ -0,0 +1,110 @@
+---
+title: "依赖管理"
+nav-parent_id: python_tableapi
+nav-pos: 100
+---
+
+
+# Java Dependency
+
+If third-party Java dependencies are used, you can specify the dependencies with the following Python Table APIs or through command line arguments directly when submitting the job.
+
+{% highlight python %}
+# Specify a list of jar URLs via "pipeline.jars". The jars are separated by ";" and will be uploaded to the cluster.
+# NOTE: Only local file URLs (start with "file://") are supported.
+table_env.get_config().set_configuration("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar")
+
+# Specify a list of URLs via "pipeline.classpaths". The URLs are separated by ";" and will be added to the classpath of the cluster.
+# NOTE: The Paths must specify a protocol (e.g. file://) and users should ensure that the URLs are accessible on both the client and the cluster.
+table_env.get_config().set_configuration("pipeline.classpaths", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar")
+{% endhighlight %}
+
+# Python Dependency
+
+If third-party Python dependencies are used, you can specify the dependencies with the following Python Table APIs or through command line arguments directly when submitting the job.
+
+
+
+
+
APIs
+
Description
+
+
+
+
+
+
add_python_file(file_path)
+
+
Adds python file dependencies which could be python files, python packages or local directories. They will be added to the PYTHONPATH of the python UDF worker.
Specifies a requirements.txt file which defines the third-party dependencies. These dependencies will be installed to a temporary directory and added to the PYTHONPATH of the python UDF worker. For the dependencies which could not be accessed in the cluster, a directory which contains the installation packages of these dependencies could be specified using the parameter "requirements_cached_dir". It will be uploaded to the cluster to support offline installation.
Please make sure the installation packages matches the platform of the cluster and the python version used. These packages will be installed using pip, so also make sure the version of Pip (version >= 7.1.0) and the version of SetupTools (version >= 37.0.0).
+
+
+
+
add_python_archive(archive_path, target_dir=None)
+
+
Adds a python archive file dependency. The file will be extracted to the working directory of python UDF worker. If the parameter "target_dir" is specified, the archive file will be extracted to a directory named "target_dir". Otherwise, the archive file will be extracted to a directory with the same name of the archive file.
+{% highlight python %}
+# command executed in shell
+# assert the relative path of python interpreter is py_env/bin/python
+zip -r py_env.zip py_env
+
+# python code
+table_env.add_python_archive("py_env.zip")
+# or
+table_env.add_python_archive("py_env.zip", "myenv")
+
+# the files contained in the archive file can be accessed in UDF
+def my_udf():
+ with open("myenv/py_env/data/data.txt") as f:
+ ...
+{% endhighlight %}
+
Please make sure the uploaded python environment matches the platform that the cluster is running on. Currently only zip-format is supported. i.e. zip, jar, whl, egg, etc.
+
+
+
+
set_python_executable(python_exec)
+
+
Sets the path of the python interpreter which is used to execute the python udf workers, e.g., "/usr/local/bin/python3".
Please make sure that the specified environment matches the platform that the cluster is running on.
+
+
+
+
diff --git a/docs/dev/table/python/index.md b/docs/dev/table/python/index.md
new file mode 100644
index 0000000000000000000000000000000000000000..abf055ee0995f864481d8f3c5ed360a6fb1709e9
--- /dev/null
+++ b/docs/dev/table/python/index.md
@@ -0,0 +1,36 @@
+---
+title: "Python Table API"
+nav-id: python_tableapi
+nav-parent_id: tableapi
+nav-pos: 100
+nav-show_overview: true
+---
+
+
+Python Table API allows users to develop [Table API]({{ site.baseurl }}/dev/table/tableApi.html) programs using the Python language.
+Apache Flink has provided Python Table API support since 1.9.0.
+
+## Where to go next?
+
+- [Installation]({{ site.baseurl }}/dev/table/python/installation.html): Introduction of how to set up the Python Table API execution environment.
+- [User-defined Functions]({{ site.baseurl }}/dev/table/python/python_udfs.html): Explanation of how to define Python user-defined functions.
+- [Vectorized User-defined Functions]({{ site.baseurl }}/dev/table/python/vectorized_python_udfs.html): Explanation of how to define vectorized Python user-defined functions.
+- [Dependency Management]({{ site.baseurl }}/dev/table/python/dependency_management.html): Specification of how to use third-part dependencies in a Python Table API program.
+- [Configuration]({{ site.baseurl }}/dev/table/python/python_config.html): Description of the config options available for Python Table API programs.
diff --git a/docs/dev/table/python/index.zh.md b/docs/dev/table/python/index.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..e293f1ce149f41592c390fecb7cd9ee2aa5d1bf8
--- /dev/null
+++ b/docs/dev/table/python/index.zh.md
@@ -0,0 +1,36 @@
+---
+title: "Python Table API"
+nav-id: python_tableapi
+nav-parent_id: tableapi
+nav-pos: 100
+nav-show_overview: true
+---
+
+
+Python Table API allows users to develop [Table API]({{ site.baseurl }}/zh/dev/table/tableApi.html) programs using the Python language.
+Apache Flink has provided Python Table API support since 1.9.0.
+
+## Where to go next?
+
+- [环境安装]({{ site.baseurl }}/zh/dev/table/python/installation.html): Introduction of how to set up the Python Table API execution environment.
+- [自定义函数]({{ site.baseurl }}/zh/dev/table/python/python_udfs.html): Explanation of how to define Python user-defined functions.
+- [自定义向量化函数]({{ site.baseurl }}/zh/dev/table/python/vectorized_python_udfs.html): Explanation of how to define vectorized Python user-defined functions.
+- [依赖管理]({{ site.baseurl }}/zh/dev/table/python/dependency_management.html): Specification of how to use third-part dependencies in a Python Table API program.
+- [配置]({{ site.baseurl }}/zh/dev/table/python/python_config.html): Description of the config options available for Python Table API programs.
diff --git a/docs/dev/table/python/installation.md b/docs/dev/table/python/installation.md
new file mode 100644
index 0000000000000000000000000000000000000000..0010ca989d8ef175d62e88176698bf3d382c4989
--- /dev/null
+++ b/docs/dev/table/python/installation.md
@@ -0,0 +1,44 @@
+---
+title: "Installation"
+nav-parent_id: python_tableapi
+nav-pos: 10
+---
+
+
+* This will be replaced by the TOC
+{:toc}
+
+## Environment Requirements
+Note Python version (3.5, 3.6 or 3.7) is required for PyFlink. Please run the following command to make sure that it meets the requirements:
+
+{% highlight bash %}
+$ python --version
+# the version printed here must be 3.5, 3.6 or 3.7
+{% endhighlight %}
+
+## Installation of PyFlink
+
+PyFlink has already been deployed to PyPi and can be installed as following:
+
+{% highlight bash %}
+$ python -m pip install apache-flink
+{% endhighlight %}
+
+You can also build PyFlink from source by following the [development guide]({{ site.baseurl }}/flinkDev/building.html#build-pyflink).
diff --git a/docs/dev/table/python/installation.zh.md b/docs/dev/table/python/installation.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..43ccca8967b972c77a80a480a578ea7bd127eeae
--- /dev/null
+++ b/docs/dev/table/python/installation.zh.md
@@ -0,0 +1,44 @@
+---
+title: "环境安装"
+nav-parent_id: python_tableapi
+nav-pos: 10
+---
+
+
+* This will be replaced by the TOC
+{:toc}
+
+## Environment Requirements
+Note Python version (3.5, 3.6 or 3.7) is required for PyFlink. Please run the following command to make sure that it meets the requirements:
+
+{% highlight bash %}
+$ python --version
+# the version printed here must be 3.5, 3.6 or 3.7
+{% endhighlight %}
+
+## Installation of PyFlink
+
+PyFlink has already been deployed to PyPi and can be installed as following:
+
+{% highlight bash %}
+$ python -m pip install apache-flink
+{% endhighlight %}
+
+You can also build PyFlink from source by following the [development guide]({{ site.baseurl }}/zh/flinkDev/building.html#build-pyflink).
diff --git a/docs/dev/table/python/metrics.md b/docs/dev/table/python/metrics.md
new file mode 100644
index 0000000000000000000000000000000000000000..df98b79bff7cf7126a04e3e2f1e114579faa0318
--- /dev/null
+++ b/docs/dev/table/python/metrics.md
@@ -0,0 +1,207 @@
+---
+title: "Metrics"
+nav-parent_id: python_tableapi
+nav-pos: 110
+---
+
+
+PyFlink exposes a metric system that allows gathering and exposing metrics to external systems.
+
+* This will be replaced by the TOC
+{:toc}
+
+## Registering metrics
+
+You can access the metric system from a [User-defined Function]({{ site.baseurl }}/dev/table/python/python_udfs.html) by calling `function_context.get_metric_group()` in the `open` method.
+The `get_metric_group()` method returns a `MetricGroup` object on which you can create and register new metrics.
+
+### Metric types
+
+PyFlink supports `Counters`, `Gauges`, `Distribution` and `Meters`.
+
+#### Counter
+
+A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(n: int)` or `dec()/dec(n: int)`.
+You can create and register a `Counter` by calling `counter(name: str)` on a `MetricGroup`.
+
+
+
+#### Gauge
+
+A `Gauge` provides a value on demand. You can register a gauge by calling `gauge(name: str, obj: Callable[[], int])` on a MetricGroup. The Callable object will be used to report the values. Gauge metrics are restricted to integer-only values.
+
+
+
+#### Distribution
+
+A metric that reports information(sum, count, min, max and mean) about the distribution of reported values. The value can be updated using `update(n: int)`. You can register a distribution by calling `distribution(name: str)` on a MetricGroup. Distribution metrics are restricted to integer-only distributions.
+
+
+
+#### Meter
+
+A Meter measures an average throughput. An occurrence of an event can be registered with the `mark_event()` method. The occurrence of multiple events at the same time can be registered with mark_event(n: int) method. You can register a meter by calling `meter(self, name: str, time_span_in_seconds: int = 60)` on a MetricGroup. The default value of time_span_in_seconds is 60.
+
+
+
+{% highlight python %}
+from pyflink.table.udf import ScalarFunction
+
+class MyUDF(ScalarFunction):
+
+ def __init__(self):
+ self.meter = None
+
+ def open(self, function_context):
+ super().open(function_context)
+ # an average rate of events per second over 120s, default is 60s.
+ self.meter = function_context.get_metric_group().meter("my_meter", time_span_in_seconds=120)
+
+ def eval(self, i):
+ self.meter.mark_event(i)
+ return i - 1
+{% endhighlight %}
+
+
+
+
+## Scope
+
+You can refer to the Java metric document for more details on [Scope definition]({{ site.baseurl }}/monitoring/metrics.html#Scope).
+
+### User Scope
+
+You can define a user scope by calling `MetricGroup.add_group(key: str, value: str = None)`. If extra is not None, creates a new key-value MetricGroup pair. The key group is added to this group's sub-groups, while the value group is added to the key group's sub-groups. In this case, the value group will be returned and a user variable will be defined.
+
+
+
+### System Scope
+
+You can refer to the Java metric document for more details on [System Scope]({{ site.baseurl }}/monitoring/metrics.html#system-scope).
+
+### List of all Variables
+
+You can refer to the Java metric document for more details on [List of all Variables]({{ site.baseurl }}/monitoring/metrics.html#list-of-all-variables).
+
+### User Variables
+
+You can define a user variable by calling `MetricGroup.addGroup(key: str, value: str = None)` and specifying the value parameter.
+
+**Important:** User variables cannot be used in scope formats.
+
+
+
+## Common part between PyFlink and Flink
+
+You can refer to the Java metric document for more details on the following sections:
+
+- [Reporter]({{ site.baseurl }}/monitoring/metrics.html#reporter).
+- [System metrics]({{ site.baseurl }}/monitoring/metrics.html#system-metrics).
+- [Latency tracking]({{ site.baseurl }}/monitoring/metrics.html#latency-tracking).
+- [REST API integration]({{ site.baseurl }}/monitoring/metrics.html#rest-api-integration).
+- [Dashboard integration]({{ site.baseurl }}/monitoring/metrics.html#dashboard-integration).
+
+
+{% top %}
diff --git a/docs/dev/table/python/metrics.zh.md b/docs/dev/table/python/metrics.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..0f6fc138061e166a90109f6d598e6076099bc1c3
--- /dev/null
+++ b/docs/dev/table/python/metrics.zh.md
@@ -0,0 +1,207 @@
+---
+title: "指标"
+nav-parent_id: python_tableapi
+nav-pos: 110
+---
+
+
+PyFlink exposes a metric system that allows gathering and exposing metrics to external systems.
+
+* This will be replaced by the TOC
+{:toc}
+
+## Registering metrics
+
+You can access the metric system from a [User-defined Function]({{ site.baseurl }}/zh/dev/table/python/python_udfs.html) by calling `function_context.get_metric_group()` in the `open` method.
+The `get_metric_group()` method returns a `MetricGroup` object on which you can create and register new metrics.
+
+### Metric types
+
+PyFlink supports `Counters`, `Gauges`, `Distribution` and `Meters`.
+
+#### Counter
+
+A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(n: int)` or `dec()/dec(n: int)`.
+You can create and register a `Counter` by calling `counter(name: str)` on a `MetricGroup`.
+
+
+
+#### Gauge
+
+A `Gauge` provides a value on demand. You can register a gauge by calling `gauge(name: str, obj: Callable[[], int])` on a MetricGroup. The Callable object will be used to report the values. Gauge metrics are restricted to integer-only values.
+
+
+
+#### Distribution
+
+A metric that reports information(sum, count, min, max and mean) about the distribution of reported values. The value can be updated using `update(n: int)`. You can register a distribution by calling `distribution(name: str)` on a MetricGroup. Distribution metrics are restricted to integer-only distributions.
+
+
+
+#### Meter
+
+A Meter measures an average throughput. An occurrence of an event can be registered with the `mark_event()` method. The occurrence of multiple events at the same time can be registered with mark_event(n: int) method. You can register a meter by calling `meter(self, name: str, time_span_in_seconds: int = 60)` on a MetricGroup. The default value of time_span_in_seconds is 60.
+
+
+
+{% highlight python %}
+from pyflink.table.udf import ScalarFunction
+
+class MyUDF(ScalarFunction):
+
+ def __init__(self):
+ self.meter = None
+
+ def open(self, function_context):
+ super().open(function_context)
+ # an average rate of events per second over 120s, default is 60s.
+ self.meter = function_context.get_metric_group().meter("my_meter", time_span_in_seconds=120)
+
+ def eval(self, i):
+ self.meter.mark_event(i)
+ return i - 1
+{% endhighlight %}
+
+
+
+
+## Scope
+
+You can refer to the Java metric document for more details on [Scope definition]({{ site.baseurl }}/zh/monitoring/metrics.html#Scope).
+
+### User Scope
+
+You can define a user scope by calling `MetricGroup.add_group(key: str, value: str = None)`. If extra is not None, creates a new key-value MetricGroup pair. The key group is added to this group's sub-groups, while the value group is added to the key group's sub-groups. In this case, the value group will be returned and a user variable will be defined.
+
+
+
+### System Scope
+
+You can refer to the Java metric document for more details on [System Scope]({{ site.baseurl }}/zh/monitoring/metrics.html#system-scope).
+
+### List of all Variables
+
+You can refer to the Java metric document for more details on [List of all Variables]({{ site.baseurl }}/zh/monitoring/metrics.html#list-of-all-variables).
+
+### User Variables
+
+You can define a user variable by calling `MetricGroup.addGroup(key: str, value: str = None)` and specifying the value parameter.
+
+**Important:** User variables cannot be used in scope formats.
+
+
+
+## Common part between PyFlink and Flink
+
+You can refer to the Java metric document for more details on the following sections:
+
+- [Reporter]({{ site.baseurl }}/zh/monitoring/metrics.html#reporter).
+- [System metrics]({{ site.baseurl }}/zh/monitoring/metrics.html#system-metrics).
+- [Latency tracking]({{ site.baseurl }}/zh/monitoring/metrics.html#latency-tracking).
+- [REST API integration]({{ site.baseurl }}/zh/monitoring/metrics.html#rest-api-integration).
+- [Dashboard integration]({{ site.baseurl }}/zh/monitoring/metrics.html#dashboard-integration).
+
+
+{% top %}
diff --git a/docs/dev/table/python/python_config.md b/docs/dev/table/python/python_config.md
new file mode 100644
index 0000000000000000000000000000000000000000..e25fbf54037b786ceb783624a818930a08f4fdf5
--- /dev/null
+++ b/docs/dev/table/python/python_config.md
@@ -0,0 +1,32 @@
+---
+title: "Configuration"
+nav-parent_id: python_tableapi
+nav-pos: 120
+---
+
+
+Depending on the requirements of a Python Table API program, it might be necessary to adjust certain parameters for optimization.
+All the config options available for Java/Scala Table API program could also be used in the Python Table API program.
+You could refer to the [Table API Configuration]({{ site.baseurl }}/dev/table/config.html) for more details on all the available config options for Java/Scala Table API programs.
+It has also provided examples on how to set the config options in a Table API program.
+
+## Python Options
+
+{% include generated/python_configuration.html %}
diff --git a/docs/dev/table/python/python_config.zh.md b/docs/dev/table/python/python_config.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..fa4f59d83255ecb3cf2d57095dfb70aedc4d7726
--- /dev/null
+++ b/docs/dev/table/python/python_config.zh.md
@@ -0,0 +1,32 @@
+---
+title: "配置"
+nav-parent_id: python_tableapi
+nav-pos: 120
+---
+
+
+Depending on the requirements of a Python Table API program, it might be necessary to adjust certain parameters for optimization.
+All the config options available for Java/Scala Table API program could also be used in the Python Table API program.
+You could refer to the [Table API Configuration]({{ site.baseurl }}/zh/dev/table/config.html) for more details on all the available config options for Java/Scala Table API programs.
+It has also provided examples on how to set the config options in a Table API program.
+
+## Python Options
+
+{% include generated/python_configuration.html %}
diff --git a/docs/dev/table/python/python_udfs.md b/docs/dev/table/python/python_udfs.md
new file mode 100644
index 0000000000000000000000000000000000000000..a14b6cdc240b03bf03d310bcd1719723be08e90f
--- /dev/null
+++ b/docs/dev/table/python/python_udfs.md
@@ -0,0 +1,229 @@
+---
+title: "User-defined Functions"
+nav-parent_id: python_tableapi
+nav-pos: 20
+---
+
+
+User-defined functions are important features, because they significantly extend the expressiveness of Python Table API programs.
+
+**NOTE:** Python UDF execution requires Python version (3.5, 3.6 or 3.7) with PyFlink installed. It's required on both the client side and the cluster side.
+
+* This will be replaced by the TOC
+{:toc}
+
+## Scalar Functions
+It supports to use Python scalar functions in Python Table API programs. In order to define a Python scalar function,
+one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method.
+The behavior of a Python scalar function is defined by the evaluation method which is named `eval`.
+The evaluation method can support variable arguments, such as `eval(*args)`.
+
+The following example shows how to define your own Python hash code function, register it in the TableEnvironment, and call it in a query.
+Note that you can configure your scalar function via a constructor before it is registered:
+
+{% highlight python %}
+class HashCode(ScalarFunction):
+ def __init__(self):
+ self.factor = 12
+
+ def eval(self, s):
+ return hash(s) * self.factor
+
+table_env = BatchTableEnvironment.create(env)
+
+# register the Python function
+table_env.register_function("hash_code", udf(HashCode(), DataTypes.BIGINT(), DataTypes.BIGINT()))
+
+# use the Python function in Python Table API
+my_table.select("string, bigint, bigint.hash_code(), hash_code(bigint)")
+
+# use the Python function in SQL API
+table_env.sql_query("SELECT string, bigint, hash_code(bigint) FROM MyTable")
+{% endhighlight %}
+
+It also supports to use Java/Scala scalar functions in Python Table API programs.
+
+{% highlight python %}
+'''
+Java code:
+
+// The Java class must have a public no-argument constructor and can be founded in current Java classloader.
+public class HashCode extends ScalarFunction {
+ private int factor = 12;
+
+ public int eval(String s) {
+ return s.hashCode() * factor;
+ }
+}
+'''
+
+table_env = BatchTableEnvironment.create(env)
+
+# register the Java function
+table_env.register_java_function("hash_code", "my.java.function.HashCode")
+
+# use the Java function in Python Table API
+my_table.select("string.hash_code(), hash_code(string)")
+
+# use the Java function in SQL API
+table_env.sql_query("SELECT string, bigint, hash_code(string) FROM MyTable")
+{% endhighlight %}
+
+There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`.
+The following examples show the different ways to define a Python scalar function which takes two columns of
+bigint as the input parameters and returns the sum of them as the result.
+
+{% highlight python %}
+# option 1: extending the base class `ScalarFunction`
+class Add(ScalarFunction):
+ def eval(self, i, j):
+ return i + j
+
+add = udf(Add(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
+
+# option 2: Python function
+@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT())
+def add(i, j):
+ return i + j
+
+# option 3: lambda function
+add = udf(lambda i, j: i + j, [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
+
+# option 4: callable function
+class CallableAdd(object):
+ def __call__(self, i, j):
+ return i + j
+
+add = udf(CallableAdd(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
+
+# option 5: partial function
+def partial_add(i, j, k):
+ return i + j + k
+
+add = udf(functools.partial(partial_add, k=1), [DataTypes.BIGINT(), DataTypes.BIGINT()],
+ DataTypes.BIGINT())
+
+# register the Python function
+table_env.register_function("add", add)
+# use the function in Python Table API
+my_table.select("add(a, b)")
+{% endhighlight %}
+
+## Table Functions
+Similar to a Python user-defined scalar function, a user-defined table function takes zero, one, or
+multiple scalar values as input parameters. However in contrast to a scalar function, it can return
+an arbitrary number of rows as output instead of a single value. The return type of a Python UDTF
+could be of types Iterable, Iterator or generator.
+
+Note Currently, Python UDTF is supported in old planner both under streaming and batch mode while is only supported under streaming mode in Blink planner.
+
+The following example shows how to define your own Python multi emit function, register it in the
+TableEnvironment, and call it in a query.
+
+{% highlight python %}
+class Split(TableFunction):
+ def eval(self, string):
+ for s in string.split(" "):
+ yield s, len(s)
+
+env = StreamExecutionEnvironment.get_execution_environment()
+table_env = StreamTableEnvironment.create(env)
+my_table = ... # type: Table, table schema: [a: String]
+
+# register the Python Table Function
+table_env.register_function("split", udtf(Split(), DataTypes.STRING(), [DataTypes.STRING(), DataTypes.INT()]))
+
+# use the Python Table Function in Python Table API
+my_table.join_lateral("split(a) as (word, length)")
+my_table.left_outer_join_lateral("split(a) as (word, length)")
+
+# use the Python Table function in SQL API
+table_env.sql_query("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)")
+table_env.sql_query("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(split(a)) as T(word, length) ON TRUE")
+
+{% endhighlight %}
+
+
+It also supports to use Java/Scala table functions in Python Table API programs.
+{% highlight python %}
+'''
+Java code:
+
+// The generic type "Tuple2" determines the schema of the returned table as (String, Integer).
+// The java class must have a public no-argument constructor and can be founded in current java classloader.
+public class Split extends TableFunction> {
+ private String separator = " ";
+
+ public void eval(String str) {
+ for (String s : str.split(separator)) {
+ // use collect(...) to emit a row
+ collect(new Tuple2(s, s.length()));
+ }
+ }
+}
+'''
+
+env = StreamExecutionEnvironment.get_execution_environment()
+table_env = StreamTableEnvironment.create(env)
+my_table = ... # type: Table, table schema: [a: String]
+
+# Register the java function.
+table_env.register_java_function("split", "my.java.function.Split")
+
+# Use the table function in the Python Table API. "as" specifies the field names of the table.
+my_table.join_lateral("split(a) as (word, length)").select("a, word, length")
+my_table.left_outer_join_lateral("split(a) as (word, length)").select("a, word, length")
+
+# Register the python function.
+
+# Use the table function in SQL with LATERAL and TABLE keywords.
+# CROSS JOIN a table function (equivalent to "join" in Table API).
+table_env.sql_query("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)")
+# LEFT JOIN a table function (equivalent to "left_outer_join" in Table API).
+table_env.sql_query("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(split(a)) as T(word, length) ON TRUE")
+{% endhighlight %}
+
+Like Python scalar functions, you can use the above five ways to define Python TableFunctions.
+
+Note The only difference is that the return type of Python Table Functions needs to be an iterable, iterator or generator.
+
+{% highlight python %}
+# option 1: generator function
+@udtf(input_types=DataTypes.BIGINT(), result_types=DataTypes.BIGINT())
+def generator_func(x):
+ yield 1
+ yield 2
+
+# option 2: return iterator
+@udtf(input_types=DataTypes.BIGINT(), result_types=DataTypes.BIGINT())
+def iterator_func(x):
+ return range(5)
+
+# option 3: return iterable
+@udtf(input_types=DataTypes.BIGINT(), result_types=DataTypes.BIGINT())
+def iterable_func(x):
+ result = [1, 2, 3]
+ return result
+
+table_env.register_function("iterable_func", iterable_func)
+table_env.register_function("iterator_func", iterator_func)
+table_env.register_function("generator_func", generator_func)
+
+{% endhighlight %}
diff --git a/docs/dev/table/python/python_udfs.zh.md b/docs/dev/table/python/python_udfs.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..9a6fadba6eea06d3e54233aa0b31a98dae4ffd69
--- /dev/null
+++ b/docs/dev/table/python/python_udfs.zh.md
@@ -0,0 +1,229 @@
+---
+title: "自定义函数"
+nav-parent_id: python_tableapi
+nav-pos: 20
+---
+
+
+User-defined functions are important features, because they significantly extend the expressiveness of Python Table API programs.
+
+**NOTE:** Python UDF execution requires Python version (3.5, 3.6 or 3.7) with PyFlink installed. It's required on both the client side and the cluster side.
+
+* This will be replaced by the TOC
+{:toc}
+
+## Scalar Functions
+It supports to use Python scalar functions in Python Table API programs. In order to define a Python scalar function,
+one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method.
+The behavior of a Python scalar function is defined by the evaluation method which is named `eval`.
+The evaluation method can support variable arguments, such as `eval(*args)`.
+
+The following example shows how to define your own Python hash code function, register it in the TableEnvironment, and call it in a query.
+Note that you can configure your scalar function via a constructor before it is registered:
+
+{% highlight python %}
+class HashCode(ScalarFunction):
+ def __init__(self):
+ self.factor = 12
+
+ def eval(self, s):
+ return hash(s) * self.factor
+
+table_env = BatchTableEnvironment.create(env)
+
+# register the Python function
+table_env.register_function("hash_code", udf(HashCode(), DataTypes.BIGINT(), DataTypes.BIGINT()))
+
+# use the Python function in Python Table API
+my_table.select("string, bigint, bigint.hash_code(), hash_code(bigint)")
+
+# use the Python function in SQL API
+table_env.sql_query("SELECT string, bigint, hash_code(bigint) FROM MyTable")
+{% endhighlight %}
+
+It also supports to use Java/Scala scalar functions in Python Table API programs.
+
+{% highlight python %}
+'''
+Java code:
+
+// The Java class must have a public no-argument constructor and can be founded in current Java classloader.
+public class HashCode extends ScalarFunction {
+ private int factor = 12;
+
+ public int eval(String s) {
+ return s.hashCode() * factor;
+ }
+}
+'''
+
+table_env = BatchTableEnvironment.create(env)
+
+# register the Java function
+table_env.register_java_function("hash_code", "my.java.function.HashCode")
+
+# use the Java function in Python Table API
+my_table.select("string.hash_code(), hash_code(string)")
+
+# use the Java function in SQL API
+table_env.sql_query("SELECT string, bigint, hash_code(string) FROM MyTable")
+{% endhighlight %}
+
+There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`.
+The following examples show the different ways to define a Python scalar function which takes two columns of
+bigint as the input parameters and returns the sum of them as the result.
+
+{% highlight python %}
+# option 1: extending the base class `ScalarFunction`
+class Add(ScalarFunction):
+ def eval(self, i, j):
+ return i + j
+
+add = udf(Add(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
+
+# option 2: Python function
+@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT())
+def add(i, j):
+ return i + j
+
+# option 3: lambda function
+add = udf(lambda i, j: i + j, [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
+
+# option 4: callable function
+class CallableAdd(object):
+ def __call__(self, i, j):
+ return i + j
+
+add = udf(CallableAdd(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())
+
+# option 5: partial function
+def partial_add(i, j, k):
+ return i + j + k
+
+add = udf(functools.partial(partial_add, k=1), [DataTypes.BIGINT(), DataTypes.BIGINT()],
+ DataTypes.BIGINT())
+
+# register the Python function
+table_env.register_function("add", add)
+# use the function in Python Table API
+my_table.select("add(a, b)")
+{% endhighlight %}
+
+## Table Functions
+Similar to a Python user-defined scalar function, a user-defined table function takes zero, one, or
+multiple scalar values as input parameters. However in contrast to a scalar function, it can return
+an arbitrary number of rows as output instead of a single value. The return type of a Python UDTF
+could be of types Iterable, Iterator or generator.
+
+Note Currently, Python UDTF is supported in old planner both under streaming and batch mode while is only supported under streaming mode in Blink planner.
+
+The following example shows how to define your own Python multi emit function, register it in the
+TableEnvironment, and call it in a query.
+
+{% highlight python %}
+class Split(TableFunction):
+ def eval(self, string):
+ for s in string.split(" "):
+ yield s, len(s)
+
+env = StreamExecutionEnvironment.get_execution_environment()
+table_env = StreamTableEnvironment.create(env)
+my_table = ... # type: Table, table schema: [a: String]
+
+# register the Python Table Function
+table_env.register_function("split", udtf(Split(), DataTypes.STRING(), [DataTypes.STRING(), DataTypes.INT()]))
+
+# use the Python Table Function in Python Table API
+my_table.join_lateral("split(a) as (word, length)")
+my_table.left_outer_join_lateral("split(a) as (word, length)")
+
+# use the Python Table function in SQL API
+table_env.sql_query("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)")
+table_env.sql_query("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(split(a)) as T(word, length) ON TRUE")
+
+{% endhighlight %}
+
+
+It also supports to use Java/Scala table functions in Python Table API programs.
+{% highlight python %}
+'''
+Java code:
+
+// The generic type "Tuple2" determines the schema of the returned table as (String, Integer).
+// The java class must have a public no-argument constructor and can be founded in current java classloader.
+public class Split extends TableFunction> {
+ private String separator = " ";
+
+ public void eval(String str) {
+ for (String s : str.split(separator)) {
+ // use collect(...) to emit a row
+ collect(new Tuple2(s, s.length()));
+ }
+ }
+}
+'''
+
+env = StreamExecutionEnvironment.get_execution_environment()
+table_env = StreamTableEnvironment.create(env)
+my_table = ... # type: Table, table schema: [a: String]
+
+# Register the java function.
+table_env.register_java_function("split", "my.java.function.Split")
+
+# Use the table function in the Python Table API. "as" specifies the field names of the table.
+my_table.join_lateral("split(a) as (word, length)").select("a, word, length")
+my_table.left_outer_join_lateral("split(a) as (word, length)").select("a, word, length")
+
+# Register the python function.
+
+# Use the table function in SQL with LATERAL and TABLE keywords.
+# CROSS JOIN a table function (equivalent to "join" in Table API).
+table_env.sql_query("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)")
+# LEFT JOIN a table function (equivalent to "left_outer_join" in Table API).
+table_env.sql_query("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(split(a)) as T(word, length) ON TRUE")
+{% endhighlight %}
+
+Like Python scalar functions, you can use the above five ways to define Python TableFunctions.
+
+Note The only difference is that the return type of Python Table Functions needs to be an iterable, iterator or generator.
+
+{% highlight python %}
+# option 1: generator function
+@udtf(input_types=DataTypes.BIGINT(), result_types=DataTypes.BIGINT())
+def generator_func(x):
+ yield 1
+ yield 2
+
+# option 2: return iterator
+@udtf(input_types=DataTypes.BIGINT(), result_types=DataTypes.BIGINT())
+def iterator_func(x):
+ return range(5)
+
+# option 3: return iterable
+@udtf(input_types=DataTypes.BIGINT(), result_types=DataTypes.BIGINT())
+def iterable_func(x):
+ result = [1, 2, 3]
+ return result
+
+table_env.register_function("iterable_func", iterable_func)
+table_env.register_function("iterator_func", iterator_func)
+table_env.register_function("generator_func", generator_func)
+
+{% endhighlight %}
diff --git a/docs/dev/table/python/vectorized_python_udfs.md b/docs/dev/table/python/vectorized_python_udfs.md
new file mode 100644
index 0000000000000000000000000000000000000000..8eec40bf497bc4d1eed8e140b8d4154e6c09dc05
--- /dev/null
+++ b/docs/dev/table/python/vectorized_python_udfs.md
@@ -0,0 +1,65 @@
+---
+title: "Vectorized User-defined Functions"
+nav-parent_id: python_tableapi
+nav-pos: 30
+---
+
+
+Vectorized Python user-defined functions are functions which are executed by transferring a batch of elements between JVM and Python VM in Arrow columnar format.
+The performance of vectorized Python user-defined functions are usually much higher than non-vectorized Python user-defined functions as the serialization/deserialization
+overhead and invocation overhead are much reduced. Besides, users could leverage the popular Python libraries such as Pandas, Numpy, etc for the vectorized Python user-defined functions implementation.
+These Python libraries are highly optimized and provide high-performance data structures and functions. It shares the similar way as the
+[non-vectorized user-defined functions]({{ site.baseurl }}/dev/table/python/python_udfs.html) on how to define vectorized user-defined functions.
+Users only need to add an extra parameter `udf_type="pandas"` in the decorator `udf` to mark it as a vectorized user-defined function.
+
+**NOTE:** Python UDF execution requires Python version (3.5, 3.6 or 3.7) with PyFlink installed. It's required on both the client side and the cluster side.
+
+* This will be replaced by the TOC
+{:toc}
+
+## Vectorized Scalar Functions
+
+Vectorized Python scalar functions take `pandas.Series` as the inputs and return a `pandas.Series` of the same length as the output.
+Internally, Flink will split the input elements into batches, convert a batch of input elements into `Pandas.Series`
+and then call user-defined vectorized Python scalar functions for each batch of input elements. Please refer to the config option
+[python.fn-execution.arrow.batch.size]({{ site.baseurl }}/dev/table/python/python_config.html#python-fn-execution-arrow-batch-size) for more details
+on how to configure the batch size.
+
+Vectorized Python scalar function could be used in any places where non-vectorized Python scalar functions could be used.
+
+The following example shows how to define your own vectorized Python scalar function which computes the sum of two columns,
+and use it in a query:
+
+{% highlight python %}
+@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT(), udf_type="pandas")
+def add(i, j):
+ return i + j
+
+table_env = BatchTableEnvironment.create(env)
+
+# register the vectorized Python scalar function
+table_env.register_function("add", add)
+
+# use the vectorized Python scalar function in Python Table API
+my_table.select("add(bigint, bigint)")
+
+# use the vectorized Python scalar function in SQL API
+table_env.sql_query("SELECT add(bigint, bigint) FROM MyTable")
+{% endhighlight %}
diff --git a/docs/dev/table/python/vectorized_python_udfs.zh.md b/docs/dev/table/python/vectorized_python_udfs.zh.md
new file mode 100644
index 0000000000000000000000000000000000000000..c50bc5b0de10023e4ef4b95321818c7d20ec7132
--- /dev/null
+++ b/docs/dev/table/python/vectorized_python_udfs.zh.md
@@ -0,0 +1,65 @@
+---
+title: "自定义向量化函数"
+nav-parent_id: python_tableapi
+nav-pos: 30
+---
+
+
+Vectorized Python user-defined functions are functions which are executed by transferring a batch of elements between JVM and Python VM in Arrow columnar format.
+The performance of vectorized Python user-defined functions are usually much higher than non-vectorized Python user-defined functions as the serialization/deserialization
+overhead and invocation overhead are much reduced. Besides, users could leverage the popular Python libraries such as Pandas, Numpy, etc for the vectorized Python user-defined functions implementation.
+These Python libraries are highly optimized and provide high-performance data structures and functions. It shares the similar way as the
+[non-vectorized user-defined functions]({{ site.baseurl }}/zh/dev/table/python/python_udfs.html) on how to define vectorized user-defined functions.
+Users only need to add an extra parameter `udf_type="pandas"` in the decorator `udf` to mark it as a vectorized user-defined function.
+
+**NOTE:** Python UDF execution requires Python version (3.5, 3.6 or 3.7) with PyFlink installed. It's required on both the client side and the cluster side.
+
+* This will be replaced by the TOC
+{:toc}
+
+## Vectorized Scalar Functions
+
+Vectorized Python scalar functions take `pandas.Series` as the inputs and return a `pandas.Series` of the same length as the output.
+Internally, Flink will split the input elements into batches, convert a batch of input elements into `Pandas.Series`
+and then call user-defined vectorized Python scalar functions for each batch of input elements. Please refer to the config option
+[python.fn-execution.arrow.batch.size]({{ site.baseurl }}/zh/dev/table/python/python_config.html#python-fn-execution-arrow-batch-size) for more details
+on how to configure the batch size.
+
+Vectorized Python scalar function could be used in any places where non-vectorized Python scalar functions could be used.
+
+The following example shows how to define your own vectorized Python scalar function which computes the sum of two columns,
+and use it in a query:
+
+{% highlight python %}
+@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT(), udf_type="pandas")
+def add(i, j):
+ return i + j
+
+table_env = BatchTableEnvironment.create(env)
+
+# register the vectorized Python scalar function
+table_env.register_function("add", add)
+
+# use the vectorized Python scalar function in Python Table API
+my_table.select("add(bigint, bigint)")
+
+# use the vectorized Python scalar function in SQL API
+table_env.sql_query("SELECT add(bigint, bigint) FROM MyTable")
+{% endhighlight %}
diff --git a/docs/dev/table/sourceSinks.md b/docs/dev/table/sourceSinks.md
index dcd6829d5b6f8d7070ba5f8f96ad01e55f85dfd8..7b3647f0673669d838275f8ddd03c58ed506b1e5 100644
--- a/docs/dev/table/sourceSinks.md
+++ b/docs/dev/table/sourceSinks.md
@@ -452,7 +452,7 @@ The interface looks as follows:
{% highlight java %}
AppendStreamTableSink implements TableSink {
- public void emitDataStream(DataStream dataStream);
+ public DataStreamSink> consumeDataStream(DataStream dataStream);
}
{% endhighlight %}
diff --git a/docs/dev/table/sql/create.md b/docs/dev/table/sql/create.md
index 6e41eac277e517eb44face8977fc9b392e9663e3..e44e2bc74e0bdeba06a3418ac05b0f54af069cec 100644
--- a/docs/dev/table/sql/create.md
+++ b/docs/dev/table/sql/create.md
@@ -210,6 +210,24 @@ The key and value of expression `key1=val1` should both be string literal. See d
{% top %}
+## CREATE CATALOG
+
+{% highlight sql %}
+CREATE CATALOG catalog_name
+ WITH (key1=val1, key2=val2, ...)
+{% endhighlight %}
+
+Create a catalog with the given catalog properties. If a catalog with the same name already exists, an exception is thrown.
+
+**WITH OPTIONS**
+
+Catalog properties used to store extra information related to this catalog.
+The key and value of expression `key1=val1` should both be string literal.
+
+Check out more details at [Catalogs]({{ site.baseurl }}/dev/table/catalogs.html).
+
+{% top %}
+
## CREATE DATABASE
{% highlight sql %}
diff --git a/docs/dev/table/sql/create.zh.md b/docs/dev/table/sql/create.zh.md
index a69691202243d46f1e3a4625095db138d8e34788..5c6099ae82ba53c4124d4f89b7433149fb10b2ba 100644
--- a/docs/dev/table/sql/create.zh.md
+++ b/docs/dev/table/sql/create.zh.md
@@ -210,6 +210,24 @@ CREATE TABLE Orders (
{% top %}
+## CREATE CATALOG
+
+{% highlight sql %}
+CREATE CATALOG catalog_name
+ WITH (key1=val1, key2=val2, ...)
+{% endhighlight %}
+
+Create a catalog with the given catalog properties. If a catalog with the same name already exists, an exception is thrown.
+
+**WITH OPTIONS**
+
+Catalog properties used to store extra information related to this catalog.
+The key and value of expression `key1=val1` should both be string literal.
+
+Check out more details at [Catalogs]({{ site.baseurl }}/dev/table/catalogs.html).
+
+{% top %}
+
## CREATE DATABASE
{% highlight sql %}
diff --git a/docs/dev/table/sql/queries.md b/docs/dev/table/sql/queries.md
index de8d90622d0690dc98634a9e5b06bf29d203a986..eba61350e6bde159cbbffbd7eb4fab80ec0d1137 100644
--- a/docs/dev/table/sql/queries.md
+++ b/docs/dev/table/sql/queries.md
@@ -471,7 +471,8 @@ SELECT DISTINCT users FROM Orders
@@ -869,7 +871,7 @@ WHERE rownum <= N [AND conditions]
- `WHERE rownum <= N`: Flink 需要 `rownum <= N` 才能识别一个查询是否为 Top-N 查询。 其中, N 代表最大或最小的 N 条记录会被保留。
- `[AND conditions]`: 在 where 语句中,可以随意添加其他的查询条件,但其他条件只允许通过 `AND` 与 `rownum <= N` 结合使用。
-流处理模式需注意 TopN 查询 可自动更新结果。 Flink SQL 会根据排序键对输入的流进行排序;若 top N 的记录发生了变化,变化的部分会以撤销、更新记录的形式发送到下游。
+流处理模式需注意 TopN 查询的结果会带有更新。 Flink SQL 会根据排序键对输入的流进行排序;若 top N 的记录发生了变化,变化的部分会以撤销、更新记录的形式发送到下游。
推荐使用一个支持更新的存储作为 Top-N 查询的 sink 。另外,若 top N 记录需要存储到外部存储,则结果表需要拥有相同与 Top-N 查询相同的唯一键。
Top-N 的唯一键是分区列和 rownum 列的结合,另外 Top-N 查询也可以获得上游的唯一键。以下面的任务为例,`product_id` 是 `ShopSales` 的唯一键,然后 Top-N 的唯一键是 [`category`, `rownum`] 和 [`product_id`] 。
diff --git a/docs/dev/table/sqlClient.md b/docs/dev/table/sqlClient.md
index d35a054b16571d431ba339f02a684d07ddea66fa..daa14191935ae4732102f669a19cea67417ff6c5 100644
--- a/docs/dev/table/sqlClient.md
+++ b/docs/dev/table/sqlClient.md
@@ -211,7 +211,6 @@ catalogs:
property-version: 1
default-database: mydb2
hive-conf-dir: ...
- hive-version: 1.2.1
# Properties that change the fundamental execution behavior of a table program.
@@ -318,7 +317,6 @@ tables:
topic: TaxiRides
startup-mode: earliest-offset
properties:
- zookeeper.connect: localhost:2181
bootstrap.servers: localhost:9092
group.id: testGroup
format:
@@ -354,30 +352,42 @@ Both `connector` and `format` allow to define a property version (which is curre
### User-defined Functions
-The SQL Client allows users to create custom, user-defined functions to be used in SQL queries. Currently, these functions are restricted to be defined programmatically in Java/Scala classes.
+The SQL Client allows users to create custom, user-defined functions to be used in SQL queries. Currently, these functions are restricted to be defined programmatically in Java/Scala classes or Python files.
-In order to provide a user-defined function, you need to first implement and compile a function class that extends `ScalarFunction`, `AggregateFunction` or `TableFunction` (see [User-defined Functions]({{ site.baseurl }}/dev/table/functions/udfs.html)). One or more functions can then be packaged into a dependency JAR for the SQL Client.
+In order to provide a Java/Scala user-defined function, you need to first implement and compile a function class that extends `ScalarFunction`, `AggregateFunction` or `TableFunction` (see [User-defined Functions]({{ site.baseurl }}/dev/table/functions/udfs.html)). One or more functions can then be packaged into a dependency JAR for the SQL Client.
+
+In order to provide a Python user-defined function, you need to write a Python function and decorate it with the `pyflink.table.udf.udf` or `pyflink.table.udf.udtf` decorator (see [Python UDFs]({{ site.baseurl }}/dev/table/python/python_udfs.html)). One or more functions can then be placed into a Python file. The Python file and related dependencies need to be specified via the configuration (see [Python Configuration]({{ site.baseurl }}/dev/table/python/python_config.html)) in environment file or the command line options (see [Command Line Usage]({{ site.baseurl }}/ops/cli.html#usage)).
All functions must be declared in an environment file before being called. For each item in the list of `functions`, one must specify
- a `name` under which the function is registered,
-- the source of the function using `from` (restricted to be `class` for now),
+- the source of the function using `from` (restricted to be `class` (Java/Scala UDF) or `python` (Python UDF) for now),
+
+The Java/Scala UDF must specify:
+
- the `class` which indicates the fully qualified class name of the function and an optional list of `constructor` parameters for instantiation.
+The Python UDF must specify:
+
+- the `fully-qualified-name` which indicates the fully qualified name, i.e the "[module name].[object name]" of the function.
+
{% highlight yaml %}
functions:
- - name: ... # required: name of the function
- from: class # required: source of the function (can only be "class" for now)
- class: ... # required: fully qualified class name of the function
- constructor: # optional: constructor parameters of the function class
- - ... # optional: a literal parameter with implicit type
- - class: ... # optional: full class name of the parameter
- constructor: # optional: constructor parameters of the parameter's class
- - type: ... # optional: type of the literal parameter
- value: ... # optional: value of the literal parameter
+ - name: java_udf # required: name of the function
+ from: class # required: source of the function
+ class: ... # required: fully qualified class name of the function
+ constructor: # optional: constructor parameters of the function class
+ - ... # optional: a literal parameter with implicit type
+ - class: ... # optional: full class name of the parameter
+ constructor: # optional: constructor parameters of the parameter's class
+ - type: ... # optional: type of the literal parameter
+ value: ... # optional: value of the literal parameter
+ - name: python_udf # required: name of the function
+ from: python # required: source of the function
+ fully-qualified-name: ... # required: fully qualified class name of the function
{% endhighlight %}
-Make sure that the order and types of the specified parameters strictly match one of the constructors of your function class.
+For Java/Scala UDF, make sure that the order and types of the specified parameters strictly match one of the constructors of your function class.
#### Constructor Parameters
@@ -448,7 +458,6 @@ catalogs:
type: hive
property-version: 1
default-database: mydb2
- hive-version: 1.2.1
hive-conf-dir:
- name: catalog_2
type: hive
@@ -485,7 +494,6 @@ tables:
version: "0.11"
topic: OutputTopic
properties:
- zookeeper.connect: localhost:2181
bootstrap.servers: localhost:9092
group.id: testGroup
format:
diff --git a/docs/dev/table/sqlClient.zh.md b/docs/dev/table/sqlClient.zh.md
index bce976afa936f96e61d8aac04c1c5c876ca9f9a7..bac97aab0ff41953fe952fb2f87ce2d12457ba7c 100644
--- a/docs/dev/table/sqlClient.zh.md
+++ b/docs/dev/table/sqlClient.zh.md
@@ -211,7 +211,6 @@ catalogs:
property-version: 1
default-database: mydb2
hive-conf-dir: ...
- hive-version: 1.2.1
# Properties that change the fundamental execution behavior of a table program.
@@ -318,7 +317,6 @@ tables:
topic: TaxiRides
startup-mode: earliest-offset
properties:
- zookeeper.connect: localhost:2181
bootstrap.servers: localhost:9092
group.id: testGroup
format:
@@ -353,31 +351,42 @@ Both `connector` and `format` allow to define a property version (which is curre
{% top %}
### User-defined Functions
+The SQL Client allows users to create custom, user-defined functions to be used in SQL queries. Currently, these functions are restricted to be defined programmatically in Java/Scala classes or Python files.
-The SQL Client allows users to create custom, user-defined functions to be used in SQL queries. Currently, these functions are restricted to be defined programmatically in Java/Scala classes.
+In order to provide a Java/Scala user-defined function, you need to first implement and compile a function class that extends `ScalarFunction`, `AggregateFunction` or `TableFunction` (see [User-defined Functions]({{ site.baseurl }}/dev/table/functions/udfs.html)). One or more functions can then be packaged into a dependency JAR for the SQL Client.
-In order to provide a user-defined function, you need to first implement and compile a function class that extends `ScalarFunction`, `AggregateFunction` or `TableFunction` (see [User-defined Functions]({{ site.baseurl }}/dev/table/functions/udfs.html)). One or more functions can then be packaged into a dependency JAR for the SQL Client.
+In order to provide a Python user-defined function, you need to write a Python function and decorate it with the `pyflink.table.udf.udf` or `pyflink.table.udf.udtf` decorator (see [Python UDFs]({{ site.baseurl }}/dev/table/python/python_udfs.html)). One or more functions can then be placed into a Python file. The Python file and related dependencies need to be specified via the configuration (see [Python Configuration]({{ site.baseurl }}/dev/table/python/python_config.html)) in environment file or the command line options (see [Command Line Usage]({{ site.baseurl }}/ops/cli.html#usage)).
All functions must be declared in an environment file before being called. For each item in the list of `functions`, one must specify
- a `name` under which the function is registered,
-- the source of the function using `from` (restricted to be `class` for now),
+- the source of the function using `from` (restricted to be `class` (Java/Scala UDF) or `python` (Python UDF) for now),
+
+The Java/Scala UDF must specify:
+
- the `class` which indicates the fully qualified class name of the function and an optional list of `constructor` parameters for instantiation.
+The Python UDF must specify:
+
+- the `fully-qualified-name` which indicates the fully qualified name, i.e the "[module name].[object name]" of the function.
+
{% highlight yaml %}
functions:
- - name: ... # required: name of the function
- from: class # required: source of the function (can only be "class" for now)
- class: ... # required: fully qualified class name of the function
- constructor: # optional: constructor parameters of the function class
- - ... # optional: a literal parameter with implicit type
- - class: ... # optional: full class name of the parameter
- constructor: # optional: constructor parameters of the parameter's class
- - type: ... # optional: type of the literal parameter
- value: ... # optional: value of the literal parameter
+ - name: java_udf # required: name of the function
+ from: class # required: source of the function
+ class: ... # required: fully qualified class name of the function
+ constructor: # optional: constructor parameters of the function class
+ - ... # optional: a literal parameter with implicit type
+ - class: ... # optional: full class name of the parameter
+ constructor: # optional: constructor parameters of the parameter's class
+ - type: ... # optional: type of the literal parameter
+ value: ... # optional: value of the literal parameter
+ - name: python_udf # required: name of the function
+ from: python # required: source of the function
+ fully-qualified-name: ... # required: fully qualified class name of the function
{% endhighlight %}
-Make sure that the order and types of the specified parameters strictly match one of the constructors of your function class.
+For Java/Scala UDF, make sure that the order and types of the specified parameters strictly match one of the constructors of your function class.
#### Constructor Parameters
@@ -448,7 +457,6 @@ catalogs:
type: hive
property-version: 1
default-database: mydb2
- hive-version: 1.2.1
hive-conf-dir:
- name: catalog_2
type: hive
@@ -485,7 +493,6 @@ tables:
version: "0.11"
topic: OutputTopic
properties:
- zookeeper.connect: localhost:2181
bootstrap.servers: localhost:9092
group.id: testGroup
format:
diff --git a/docs/dev/table/streaming/dynamic_tables.zh.md b/docs/dev/table/streaming/dynamic_tables.zh.md
index a9d12d37ee8d61eea5e967dea1da90a38d4ac523..29065686619ae3ef9ae017055da53ce4c403ec26 100644
--- a/docs/dev/table/streaming/dynamic_tables.zh.md
+++ b/docs/dev/table/streaming/dynamic_tables.zh.md
@@ -22,127 +22,127 @@ specific language governing permissions and limitations
under the License.
-->
-SQL and the relational algebra have not been designed with streaming data in mind. As a consequence, there are few conceptual gaps between relational algebra (and SQL) and stream processing.
+SQL 和关系代数在设计时并未考虑流数据。因此,在关系代数(和 SQL)之间几乎没有概念上的差异。
-This page discusses these differences and explains how Flink can achieve the same semantics on unbounded data as a regular database engine on bounded data.
+本文会讨论这种差异,并介绍 Flink 如何在无界数据集上实现与数据库引擎在有界数据上的处理具有相同的语义。
* This will be replaced by the TOC
{:toc}
-Relational Queries on Data Streams
+DataStream 上的关系查询
----------------------------------
-The following table compares traditional relational algebra and stream processing with respect to input data, execution, and output results.
+下表比较了传统的关系代数和流处理与输入数据、执行和输出结果的关系。
-
Relational Algebra / SQL
-
Stream Processing
+
关系代数 / SQL
+
流处理
-
Relations (or tables) are bounded (multi-)sets of tuples.
-
A stream is an infinite sequences of tuples.
+
关系(或表)是有界(多)元组集合。
+
流是一个无限元组序列。
-
A query that is executed on batch data (e.g., a table in a relational database) has access to the complete input data.
-
A streaming query cannot access all data when it is started and has to "wait" for data to be streamed in.
+
对批数据(例如关系数据库中的表)执行的查询可以访问完整的输入数据。
+
流式查询在启动时不能访问所有数据,必须“等待”数据流入。
-
A batch query terminates after it produced a fixed sized result.
-
A streaming query continuously updates its result based on the received records and never completes.
+
批处理查询在产生固定大小的结果后终止。
+
流查询不断地根据接收到的记录更新其结果,并且始终不会结束。
-Despite these differences, processing streams with relational queries and SQL is not impossible. Advanced relational database systems offer a feature called *Materialized Views*. A materialized view is defined as a SQL query, just like a regular virtual view. In contrast to a virtual view, a materialized view caches the result of the query such that the query does not need to be evaluated when the view is accessed. A common challenge for caching is to prevent a cache from serving outdated results. A materialized view becomes outdated when the base tables of its definition query are modified. *Eager View Maintenance* is a technique to update a materialized view as soon as its base tables are updated.
+尽管存在这些差异,但是使用关系查询和 SQL 处理流并不是不可能的。高级关系数据库系统提供了一个称为 *物化视图(Materialized Views)* 的特性。物化视图被定义为一条 SQL 查询,就像常规的虚拟视图一样。与虚拟视图相反,物化视图缓存查询的结果,因此在访问视图时不需要对查询进行计算。缓存的一个常见难题是防止缓存为过期的结果提供服务。当其定义查询的基表被修改时,物化视图将过期。 *即时视图维护(Eager View Maintenance)* 是一种一旦更新了物化视图的基表就立即更新视图的技术。
-The connection between eager view maintenance and SQL queries on streams becomes obvious if we consider the following:
+如果我们考虑以下问题,那么即时视图维护和流上的SQL查询之间的联系就会变得显而易见:
-- A database table is the result of a *stream* of `INSERT`, `UPDATE`, and `DELETE` DML statements, often called *changelog stream*.
-- A materialized view is defined as a SQL query. In order to update the view, the query continuously processes the changelog streams of the view's base relations.
-- The materialized view is the result of the streaming SQL query.
+- 数据库表是 `INSERT`、`UPDATE` 和 `DELETE` DML 语句的 *stream* 的结果,通常称为 *changelog stream* 。
+- 物化视图被定义为一条 SQL 查询。为了更新视图,查询不断地处理视图的基本关系的changelog 流。
+- 物化视图是流式 SQL 查询的结果。
-With these points in mind, we introduce following concept of *Dynamic tables* in the next section.
+了解了这些要点之后,我们将在下一节中介绍 *动态表(Dynamic tables)* 的概念。
-Dynamic Tables & Continuous Queries
+动态表 & 连续查询(Continuous Query)
---------------------------------------
-*Dynamic tables* are the core concept of Flink's Table API and SQL support for streaming data. In contrast to the static tables that represent batch data, dynamic tables are changing over time. They can be queried like static batch tables. Querying dynamic tables yields a *Continuous Query*. A continuous query never terminates and produces a dynamic table as result. The query continuously updates its (dynamic) result table to reflect the changes on its (dynamic) input tables. Essentially, a continuous query on a dynamic table is very similar to a query that defines a materialized view.
+*动态表* 是 Flink 的支持流数据的 Table API 和 SQL 的核心概念。与表示批处理数据的静态表不同,动态表是随时间变化的。可以像查询静态批处理表一样查询它们。查询动态表将生成一个 *连续查询* 。一个连续查询永远不会终止,结果会生成一个动态表。查询不断更新其(动态)结果表,以反映其(动态)输入表上的更改。本质上,动态表上的连续查询非常类似于定义物化视图的查询。
-It is important to note that the result of a continuous query is always semantically equivalent to the result of the same query being executed in batch mode on a snapshot of the input tables.
+需要注意的是,连续查询的结果在语义上总是等价于以批处理模式在输入表快照上执行的相同查询的结果。
-The following figure visualizes the relationship of streams, dynamic tables, and continuous queries:
+下图显示了流、动态表和连续查询之间的关系:
-1. A stream is converted into a dynamic table.
-1. A continuous query is evaluated on the dynamic table yielding a new dynamic table.
-1. The resulting dynamic table is converted back into a stream.
+1. 将流转换为动态表。
+2. 在动态表上计算一个连续查询,生成一个新的动态表。
+3. 生成的动态表被转换回流。
-**Note:** Dynamic tables are foremost a logical concept. Dynamic tables are not necessarily (fully) materialized during query execution.
+**注意:** 动态表首先是一个逻辑概念。在查询执行期间不一定(完全)物化动态表。
-In the following, we will explain the concepts of dynamic tables and continuous queries with a stream of click events that have the following schema:
+在下面,我们将解释动态表和连续查询的概念,并使用具有以下模式的单击事件流:
{% highlight plain %}
[
- user: VARCHAR, // the name of the user
- cTime: TIMESTAMP, // the time when the URL was accessed
- url: VARCHAR // the URL that was accessed by the user
+ user: VARCHAR, // 用户名
+ cTime: TIMESTAMP, // 访问 URL 的时间
+ url: VARCHAR // 用户访问的 URL
]
{% endhighlight %}
-Defining a Table on a Stream
+在流上定义表
----------------------------
-In order to process a stream with a relational query, it has to be converted into a `Table`. Conceptually, each record of the stream is interpreted as an `INSERT` modification on the resulting table. Essentially, we are building a table from an `INSERT`-only changelog stream.
+为了使用关系查询处理流,必须将其转换成 `Table`。从概念上讲,流的每条记录都被解释为对结果表的 `INSERT` 操作。本质上我们正在从一个 `INSERT`-only 的 changelog 流构建表。
-The following figure visualizes how the stream of click event (left-hand side) is converted into a table (right-hand side). The resulting table is continuously growing as more records of the click stream are inserted.
+下图显示了单击事件流(左侧)如何转换为表(右侧)。当插入更多的单击流记录时,结果表将不断增长。
-**Note:** A table which is defined on a stream is internally not materialized.
+**注意:** 在流上定义的表在内部没有物化。
-### Continuous Queries
+### 连续查询
----------------------
-A continuous query is evaluated on a dynamic table and produces a new dynamic table as result. In contrast to a batch query, a continuous query never terminates and updates its result table according to the updates on its input tables. At any point in time, the result of a continuous query is semantically equivalent to the result of the same query being executed in batch mode on a snapshot of the input tables.
+在动态表上计算一个连续查询,并生成一个新的动态表。与批处理查询不同,连续查询从不终止,并根据其输入表上的更新更新其结果表。在任何时候,连续查询的结果在语义上与以批处理模式在输入表快照上执行的相同查询的结果相同。
-In the following we show two example queries on a `clicks` table that is defined on the stream of click events.
+在接下来的代码中,我们将展示 `clicks` 表上的两个示例查询,这个表是在点击事件流上定义的。
-The first query is a simple `GROUP-BY COUNT` aggregation query. It groups the `clicks` table on the `user` field and counts the number of visited URLs. The following figure shows how the query is evaluated over time as the `clicks` table is updated with additional rows.
+第一个查询是一个简单的 `GROUP-BY COUNT` 聚合查询。它基于 `user` 字段对 `clicks` 表进行分组,并统计访问的 URL 的数量。下面的图显示了当 `clicks` 表被附加的行更新时,查询是如何被评估的。
-When the query is started, the `clicks` table (left-hand side) is empty. The query starts to compute the result table, when the first row is inserted into the `clicks` table. After the first row `[Mary, ./home]` was inserted, the result table (right-hand side, top) consists of a single row `[Mary, 1]`. When the second row `[Bob, ./cart]` is inserted into the `clicks` table, the query updates the result table and inserts a new row `[Bob, 1]`. The third row `[Mary, ./prod?id=1]` yields an update of an already computed result row such that `[Mary, 1]` is updated to `[Mary, 2]`. Finally, the query inserts a third row `[Liz, 1]` into the result table, when the fourth row is appended to the `clicks` table.
+当查询开始,`clicks` 表(左侧)是空的。当第一行数据被插入到 `clicks` 表时,查询开始计算结果表。第一行数据 `[Mary,./home]` 插入后,结果表(右侧,上部)由一行 `[Mary, 1]` 组成。当第二行 `[Bob, ./cart]` 插入到 `clicks` 表时,查询会更新结果表并插入了一行新数据 `[Bob, 1]`。第三行 `[Mary, ./prod?id=1]` 将产生已计算的结果行的更新,`[Mary, 1]` 更新成 `[Mary, 2]`。最后,当第四行数据加入 `clicks` 表时,查询将第三行 `[Liz, 1]` 插入到结果表中。
-The second query is similar to the first one but groups the `clicks` table in addition to the `user` attribute also on an [hourly tumbling window]({{ site.baseurl }}/dev/table/sql/index.html#group-windows) before it counts the number of URLs (time-based computations such as windows are based on special [time attributes](time_attributes.html), which are discussed later.). Again, the figure shows the input and output at different points in time to visualize the changing nature of dynamic tables.
+第二条查询与第一条类似,但是除了用户属性之外,还将 `clicks` 分组至[每小时滚动窗口]({{ site.baseurl }}/zh/dev/table/sql/index.html#group-windows)中,然后计算 url 数量(基于时间的计算,例如基于特定[时间属性](time_attributes.html)的窗口,后面会讨论)。同样,该图显示了不同时间点的输入和输出,以可视化动态表的变化特性。
-As before, the input table `clicks` is shown on the left. The query continuously computes results every hour and updates the result table. The clicks table contains four rows with timestamps (`cTime`) between `12:00:00` and `12:59:59`. The query computes two results rows from this input (one for each `user`) and appends them to the result table. For the next window between `13:00:00` and `13:59:59`, the `clicks` table contains three rows, which results in another two rows being appended to the result table. The result table is updated, as more rows are appended to `clicks` over time.
+与前面一样,左边显示了输入表 `clicks`。查询每小时持续计算结果并更新结果表。clicks表包含四行带有时间戳(`cTime`)的数据,时间戳在 `12:00:00` 和 `12:59:59` 之间。查询从这个输入计算出两个结果行(每个 `user` 一个),并将它们附加到结果表中。对于 `13:00:00` 和 `13:59:59` 之间的下一个窗口,`clicks` 表包含三行,这将导致另外两行被追加到结果表。随着时间的推移,更多的行被添加到 `click` 中,结果表将被更新。
-### Update and Append Queries
+### 更新和追加查询
-Although the two example queries appear to be quite similar (both compute a grouped count aggregate), they differ in one important aspect:
-- The first query updates previously emitted results, i.e., the changelog stream that defines the result table contains `INSERT` and `UPDATE` changes.
-- The second query only appends to the result table, i.e., the changelog stream of the result table only consists of `INSERT` changes.
+虽然这两个示例查询看起来非常相似(都计算分组计数聚合),但它们在一个重要方面不同:
+- 第一个查询更新先前输出的结果,即定义结果表的 changelog 流包含 `INSERT` 和 `UPDATE` 操作。
+- 第二个查询只附加到结果表,即结果表的 changelog 流只包含 `INSERT` 操作。
-Whether a query produces an append-only table or an updated table has some implications:
-- Queries that produce update changes usually have to maintain more state (see the following section).
-- The conversion of an append-only table into a stream is different from the conversion of an updated table (see the [Table to Stream Conversion](#table-to-stream-conversion) section).
+一个查询是产生一个只追加的表还是一个更新的表有一些含义:
+- 产生更新更改的查询通常必须维护更多的状态(请参阅以下部分)。
+- 将 append-only 的表转换为流与将已更新的表转换为流是不同的(参阅[表到流的转换](#table-to-stream-conversion)章节)。
-### Query Restrictions
+### 查询限制
-Many, but not all, semantically valid queries can be evaluated as continuous queries on streams. Some queries are too expensive to compute, either due to the size of state that they need to maintain or because computing updates is too expensive.
+许多(但不是全部)语义上有效的查询可以作为流上的连续查询进行评估。有些查询代价太高而无法计算,这可能是由于它们需要维护的状态大小,也可能是由于计算更新代价太高。
-- **State Size:** Continuous queries are evaluated on unbounded streams and are often supposed to run for weeks or months. Hence, the total amount of data that a continuous query processes can be very large. Queries that have to update previously emitted results need to maintain all emitted rows in order to be able to update them. For instance, the first example query needs to store the URL count for each user to be able to increase the count and sent out a new result when the input table receives a new row. If only registered users are tracked, the number of counts to maintain might not be too high. However, if non-registered users get a unique user name assigned, the number of counts to maintain would grow over time and might eventually cause the query to fail.
+- **状态大小:** 连续查询在无界流上计算,通常应该运行数周或数月。因此,连续查询处理的数据总量可能非常大。必须更新先前输出的结果的查询需要维护所有输出的行,以便能够更新它们。例如,第一个查询示例需要存储每个用户的 URL 计数,以便能够增加该计数并在输入表接收新行时发送新结果。如果只跟踪注册用户,则要维护的计数数量可能不会太高。但是,如果未注册的用户分配了一个惟一的用户名,那么要维护的计数数量将随着时间增长,并可能最终导致查询失败。
{% highlight sql %}
SELECT user, COUNT(url)
@@ -150,7 +150,7 @@ FROM clicks
GROUP BY user;
{% endhighlight %}
-- **Computing Updates:** Some queries require to recompute and update a large fraction of the emitted result rows even if only a single input record is added or updated. Clearly, such queries are not well suited to be executed as continuous queries. An example is the following query which computes for each user a `RANK` based on the time of the last click. As soon as the `clicks` table receives a new row, the `lastAction` of the user is updated and a new rank must be computed. However since two rows cannot have the same rank, all lower ranked rows need to be updated as well.
+- **计算更新:** 有些查询需要重新计算和更新大量已输出的结果行,即使只添加或更新一条输入记录。显然,这样的查询不适合作为连续查询执行。下面的查询就是一个例子,它根据最后一次单击的时间为每个用户计算一个 `RANK`。一旦 `click` 表接收到一个新行,用户的 `lastAction` 就会更新,并必须计算一个新的排名。然而,由于两行不能具有相同的排名,所以所有较低排名的行也需要更新。
{% highlight sql %}
SELECT user, RANK() OVER (ORDER BY lastLogin)
@@ -159,31 +159,33 @@ FROM (
);
{% endhighlight %}
-The [Query Configuration](query_configuration.html) page discusses parameters to control the execution of continuous queries. Some parameters can be used to trade the size of maintained state for result accuracy.
+[查询配置](query_configuration.html)章节讨论了控制连续查询执行的参数。一些参数可以用来在维持状态的大小和获得结果的准确性之间做取舍。
-Table to Stream Conversion
+
+
+表到流的转换
--------------------------
-A dynamic table can be continuously modified by `INSERT`, `UPDATE`, and `DELETE` changes just like a regular database table. It might be a table with a single row, which is constantly updated, an insert-only table without `UPDATE` and `DELETE` modifications, or anything in between.
+动态表可以像普通数据库表一样通过 `INSERT`、`UPDATE` 和 `DELETE` 来不断修改。它可能是一个只有一行、不断更新的表,也可能是一个 insert-only 的表,没有 `UPDATE` 和 `DELETE` 修改,或者介于两者之间的其他表。
-When converting a dynamic table into a stream or writing it to an external system, these changes need to be encoded. Flink's Table API and SQL support three ways to encode the changes of a dynamic table:
+在将动态表转换为流或将其写入外部系统时,需要对这些更改进行编码。Flink的 Table API 和 SQL 支持三种方式来编码一个动态表的变化:
-* **Append-only stream:** A dynamic table that is only modified by `INSERT` changes can be converted into a stream by emitting the inserted rows.
+* **Append-only 流:** 仅通过 `INSERT` 操作修改的动态表可以通过输出插入的行转换为流。
-* **Retract stream:** A retract stream is a stream with two types of messages, *add messages* and *retract messages*. A dynamic table is converted into an retract stream by encoding an `INSERT` change as add message, a `DELETE` change as retract message, and an `UPDATE` change as a retract message for the updated (previous) row and an add message for the updating (new) row. The following figure visualizes the conversion of a dynamic table into a retract stream.
+* **Retract 流:** retract 流包含两种类型的 message: *add messages* 和 *retract messages* 。通过将`INSERT` 操作编码为 add message、将 `DELETE` 操作编码为 retract message、将 `UPDATE` 操作编码为更新(先前)行的 retract message 和更新(新)行的 add message,将动态表转换为 retract 流。下图显示了将动态表转换为 retract 流的过程。
-* **Upsert stream:** An upsert stream is a stream with two types of messages, *upsert messages* and *delete messages*. A dynamic table that is converted into an upsert stream requires a (possibly composite) unique key. A dynamic table with unique key is converted into a stream by encoding `INSERT` and `UPDATE` changes as upsert messages and `DELETE` changes as delete messages. The stream consuming operator needs to be aware of the unique key attribute in order to apply messages correctly. The main difference to a retract stream is that `UPDATE` changes are encoded with a single message and hence more efficient. The following figure visualizes the conversion of a dynamic table into an upsert stream.
+* **Upsert 流:** upsert 流包含两种类型的 message: *upsert messages* 和*delete messages*。转换为 upsert 流的动态表需要(可能是组合的)唯一键。通过将 `INSERT` 和 `UPDATE` 操作编码为 upsert message,将 `DELETE` 操作编码为 delete message ,将具有唯一键的动态表转换为流。消费流的算子需要知道唯一键的属性,以便正确地应用 message。与 retract 流的主要区别在于 `UPDATE` 操作是用单个 message 编码的,因此效率更高。下图显示了将动态表转换为 upsert 流的过程。
-The API to convert a dynamic table into a `DataStream` is discussed on the [Common Concepts]({{ site.baseurl }}/dev/table/common.html#convert-a-table-into-a-datastream) page. Please note that only append and retract streams are supported when converting a dynamic table into a `DataStream`. The `TableSink` interface to emit a dynamic table to an external system are discussed on the [TableSources and TableSinks](../sourceSinks.html#define-a-tablesink) page.
+在[通用概念]({{ site.baseurl }}/zh/dev/table/common.html#convert-a-table-into-a-datastream)中讨论了将动态表转换为 `DataStream` 的 API。请注意,在将动态表转换为 `DataStream` 时,只支持 append 流和 retract 流。在 [TableSources 和 TableSinks](../sourceSinks.html#define-a-tablesink) 章节讨论向外部系统输出动态表的 `TableSink` 接口。
{% top %}
diff --git a/docs/dev/table/streaming/joins.md b/docs/dev/table/streaming/joins.md
index deb31995de9ce8de2b2c47fed95e6fa12d38b0cf..e621a2057dec05d820344c37f3cb5e8c358fc56a 100644
--- a/docs/dev/table/streaming/joins.md
+++ b/docs/dev/table/streaming/joins.md
@@ -63,7 +63,7 @@ WHERE o.id = s.orderId AND
o.ordertime BETWEEN s.shiptime - INTERVAL '4' HOUR AND s.shiptime
{% endhighlight %}
-Compared to a regular join operation, this kind of join only supports append-only tables with time attributes. Since time attributes are quasi-monontic increasing, Flink can remove old values from its state without affecting the correctness of the result.
+Compared to a regular join operation, this kind of join only supports append-only tables with time attributes. Since time attributes are quasi-monotonic increasing, Flink can remove old values from its state without affecting the correctness of the result.
Join with a Temporal Table Function
--------------------------
diff --git a/docs/dev/table/streaming/joins.zh.md b/docs/dev/table/streaming/joins.zh.md
index deb31995de9ce8de2b2c47fed95e6fa12d38b0cf..e621a2057dec05d820344c37f3cb5e8c358fc56a 100644
--- a/docs/dev/table/streaming/joins.zh.md
+++ b/docs/dev/table/streaming/joins.zh.md
@@ -63,7 +63,7 @@ WHERE o.id = s.orderId AND
o.ordertime BETWEEN s.shiptime - INTERVAL '4' HOUR AND s.shiptime
{% endhighlight %}
-Compared to a regular join operation, this kind of join only supports append-only tables with time attributes. Since time attributes are quasi-monontic increasing, Flink can remove old values from its state without affecting the correctness of the result.
+Compared to a regular join operation, this kind of join only supports append-only tables with time attributes. Since time attributes are quasi-monotonic increasing, Flink can remove old values from its state without affecting the correctness of the result.
Join with a Temporal Table Function
--------------------------
diff --git a/docs/dev/table/streaming/query_configuration.md b/docs/dev/table/streaming/query_configuration.md
index 2a2f50cead1912fe9246ec4909fab3b60835573b..3bf0c45d9d10cd0efeafbd416a1ff97a90673fda 100644
--- a/docs/dev/table/streaming/query_configuration.md
+++ b/docs/dev/table/streaming/query_configuration.md
@@ -24,7 +24,7 @@ under the License.
Table API and SQL queries have the same semantics regardless whether their input is bounded batch input or unbounded stream input. In many cases, continuous queries on streaming input are capable of computing accurate results that are identical to offline computed results. However, this is not possible in general case because continuous queries have to restrict the size of the state they are maintaining in order to avoid to run out of storage and to be able to process unbounded streaming data over a long period of time. As a result, a continuous query might only be able to provide approximated results depending on the characteristics of the input data and the query itself.
-Flink's Table API and SQL interface provide parameters to tune the accuracy and resource consumption of continuous queries. The parameters are specified via a `QueryConfig` object. The `QueryConfig` can be obtained from the `TableEnvironment` and is passed back when a `Table` is translated, i.e., when it is [transformed into a DataStream]({{ site.baseurl }}/dev/table/common.html#convert-a-table-into-a-datastream-or-dataset) or [emitted via a TableSink](../common.html#emit-a-table).
+Flink's Table API and SQL interface provide parameters to tune the accuracy and resource consumption of continuous queries. The parameters are specified via a `TableConfig` object. The `TableConfig` can be obtained from the `TableEnvironment` and is passed back when a `Table` is translated, i.e., when it is [transformed into a DataStream]({{ site.baseurl }}/dev/table/common.html#convert-a-table-into-a-datastream-or-dataset) or [emitted via a TableSink](../common.html#emit-a-table).
@@ -33,9 +33,9 @@ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironm
StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
// obtain query configuration from TableEnvironment
-StreamQueryConfig qConfig = tableEnv.queryConfig();
+TableConfig tConfig = tableEnv.getConfig();
// set query parameters
-qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24));
+tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24));
// define query
Table result = ...
@@ -51,10 +51,10 @@ tableEnv.registerTableSink(
sink); // table sink
// emit result Table via a TableSink
-result.insertInto("outputTable", qConfig);
+result.insertInto("outputTable");
// convert result Table into a DataStream
-DataStream stream = tableEnv.toAppendStream(result, Row.class, qConfig);
+DataStream stream = tableEnv.toAppendStream(result, Row.class);
{% endhighlight %}
@@ -64,9 +64,9 @@ val env = StreamExecutionEnvironment.getExecutionEnvironment
val tableEnv = StreamTableEnvironment.create(env)
// obtain query configuration from TableEnvironment
-val qConfig: StreamQueryConfig = tableEnv.queryConfig
+val tConfig: TableConfig = tableEnv.getConfig
// set query parameters
-qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24))
+tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24))
// define query
val result: Table = ???
@@ -82,16 +82,16 @@ tableEnv.registerTableSink(
sink) // table sink
// emit result Table via a TableSink
-result.insertInto("outputTable", qConfig)
+result.insertInto("outputTable")
// convert result Table into a DataStream[Row]
-val stream: DataStream[Row] = result.toAppendStream[Row](qConfig)
+val stream: DataStream[Row] = result.toAppendStream[Row]
{% endhighlight %}
{% highlight python %}
-# use TableConfig instead of QueryConfig in python API
+# use TableConfig in python API
t_config = TableConfig()
# set query parameters
t_config.set_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24))
@@ -116,7 +116,7 @@ result.insert_into("outputTable")
-In the following we describe the parameters of the `QueryConfig` and how they affect the accuracy and resource consumption of a query.
+In the following we describe the parameters of the `TableConfig` and how they affect the accuracy and resource consumption of a query.
Idle State Retention Time
-------------------------
@@ -145,30 +145,30 @@ The parameters are specified as follows:
{% highlight java %}
-StreamQueryConfig qConfig = ...
+TableConfig tConfig = ...
// set idle state retention time: min = 12 hours, max = 24 hours
-qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24));
+tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24));
{% endhighlight %}
{% highlight scala %}
-val qConfig: StreamQueryConfig = ???
+val tConfig: TableConfig = ???
// set idle state retention time: min = 12 hours, max = 24 hours
-qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24))
+tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24))
{% endhighlight %}
{% highlight python %}
-q_config = ... # type: StreamQueryConfig
+t_config = ... # type: TableConfig
# set idle state retention time: min = 12 hours, max = 24 hours
-q_config.with_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24))
+t_config.set_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24))
{% endhighlight %}
diff --git a/docs/dev/table/streaming/query_configuration.zh.md b/docs/dev/table/streaming/query_configuration.zh.md
index 2a2f50cead1912fe9246ec4909fab3b60835573b..3bf0c45d9d10cd0efeafbd416a1ff97a90673fda 100644
--- a/docs/dev/table/streaming/query_configuration.zh.md
+++ b/docs/dev/table/streaming/query_configuration.zh.md
@@ -24,7 +24,7 @@ under the License.
Table API and SQL queries have the same semantics regardless whether their input is bounded batch input or unbounded stream input. In many cases, continuous queries on streaming input are capable of computing accurate results that are identical to offline computed results. However, this is not possible in general case because continuous queries have to restrict the size of the state they are maintaining in order to avoid to run out of storage and to be able to process unbounded streaming data over a long period of time. As a result, a continuous query might only be able to provide approximated results depending on the characteristics of the input data and the query itself.
-Flink's Table API and SQL interface provide parameters to tune the accuracy and resource consumption of continuous queries. The parameters are specified via a `QueryConfig` object. The `QueryConfig` can be obtained from the `TableEnvironment` and is passed back when a `Table` is translated, i.e., when it is [transformed into a DataStream]({{ site.baseurl }}/dev/table/common.html#convert-a-table-into-a-datastream-or-dataset) or [emitted via a TableSink](../common.html#emit-a-table).
+Flink's Table API and SQL interface provide parameters to tune the accuracy and resource consumption of continuous queries. The parameters are specified via a `TableConfig` object. The `TableConfig` can be obtained from the `TableEnvironment` and is passed back when a `Table` is translated, i.e., when it is [transformed into a DataStream]({{ site.baseurl }}/dev/table/common.html#convert-a-table-into-a-datastream-or-dataset) or [emitted via a TableSink](../common.html#emit-a-table).
@@ -33,9 +33,9 @@ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironm
StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
// obtain query configuration from TableEnvironment
-StreamQueryConfig qConfig = tableEnv.queryConfig();
+TableConfig tConfig = tableEnv.getConfig();
// set query parameters
-qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24));
+tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24));
// define query
Table result = ...
@@ -51,10 +51,10 @@ tableEnv.registerTableSink(
sink); // table sink
// emit result Table via a TableSink
-result.insertInto("outputTable", qConfig);
+result.insertInto("outputTable");
// convert result Table into a DataStream
-DataStream stream = tableEnv.toAppendStream(result, Row.class, qConfig);
+DataStream stream = tableEnv.toAppendStream(result, Row.class);
{% endhighlight %}
@@ -64,9 +64,9 @@ val env = StreamExecutionEnvironment.getExecutionEnvironment
val tableEnv = StreamTableEnvironment.create(env)
// obtain query configuration from TableEnvironment
-val qConfig: StreamQueryConfig = tableEnv.queryConfig
+val tConfig: TableConfig = tableEnv.getConfig
// set query parameters
-qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24))
+tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24))
// define query
val result: Table = ???
@@ -82,16 +82,16 @@ tableEnv.registerTableSink(
sink) // table sink
// emit result Table via a TableSink
-result.insertInto("outputTable", qConfig)
+result.insertInto("outputTable")
// convert result Table into a DataStream[Row]
-val stream: DataStream[Row] = result.toAppendStream[Row](qConfig)
+val stream: DataStream[Row] = result.toAppendStream[Row]
{% endhighlight %}
{% highlight python %}
-# use TableConfig instead of QueryConfig in python API
+# use TableConfig in python API
t_config = TableConfig()
# set query parameters
t_config.set_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24))
@@ -116,7 +116,7 @@ result.insert_into("outputTable")
-In the following we describe the parameters of the `QueryConfig` and how they affect the accuracy and resource consumption of a query.
+In the following we describe the parameters of the `TableConfig` and how they affect the accuracy and resource consumption of a query.
Idle State Retention Time
-------------------------
@@ -145,30 +145,30 @@ The parameters are specified as follows:
{% highlight java %}
-StreamQueryConfig qConfig = ...
+TableConfig tConfig = ...
// set idle state retention time: min = 12 hours, max = 24 hours
-qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24));
+tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24));
{% endhighlight %}
{% highlight scala %}
-val qConfig: StreamQueryConfig = ???
+val tConfig: TableConfig = ???
// set idle state retention time: min = 12 hours, max = 24 hours
-qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24))
+tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24))
{% endhighlight %}
{% highlight python %}
-q_config = ... # type: StreamQueryConfig
+t_config = ... # type: TableConfig
# set idle state retention time: min = 12 hours, max = 24 hours
-q_config.with_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24))
+t_config.set_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24))
{% endhighlight %}
diff --git a/docs/dev/table/streaming/time_attributes.zh.md b/docs/dev/table/streaming/time_attributes.zh.md
index 3b9c892eeb9369fabeb7c9794a5e92733726874f..5846028c16d34fe950dfc2b7ab5675710c863b91 100644
--- a/docs/dev/table/streaming/time_attributes.zh.md
+++ b/docs/dev/table/streaming/time_attributes.zh.md
@@ -1,5 +1,5 @@
---
-title: "Time Attributes"
+title: "时间属性"
nav-parent_id: streaming_tableapi
nav-pos: 2
---
@@ -22,29 +22,29 @@ specific language governing permissions and limitations
under the License.
-->
-Flink is able to process streaming data based on different notions of *time*.
+Flink 可以基于几种不同的 *时间* 概念来处理数据。
-- *Processing time* refers to the system time of the machine (also known as "wall-clock time") that is executing the respective operation.
-- *Event time* refers to the processing of streaming data based on timestamps which are attached to each row. The timestamps can encode when an event happened.
-- *Ingestion time* is the time that events enter Flink; internally, it is treated similarly to event time.
+- *处理时间* 指的是执行具体操作时的机器时间(也称作"挂钟时间")
+- *事件时间* 指的是数据本身携带的时间。这个时间是在事件产生时的时间。
+- *摄入时间* 指的是数据进入 Flink 的时间;在系统内部,会把它当做事件时间来处理。
-For more information about time handling in Flink, see the introduction about [Event Time and Watermarks]({{ site.baseurl }}/dev/event_time.html).
+对于时间相关的更多信息,可以参考 [事件时间和Watermark]({{ site.baseurl }}/zh/dev/event_time.html)。
-This page explains how time attributes can be defined for time-based operations in Flink's Table API & SQL.
+本页面说明了如何在 Flink Table API & SQL 里面定义时间以及相关的操作。
* This will be replaced by the TOC
{:toc}
-Introduction to Time Attributes
+时间属性介绍
-------------------------------
-Time-based operations such as windows in both the [Table API]({{ site.baseurl }}/dev/table/tableApi.html#group-windows) and [SQL]({{ site.baseurl }}/dev/table/sql/queries.html#group-windows) require information about the notion of time and its origin. Therefore, tables can offer *logical time attributes* for indicating time and accessing corresponding timestamps in table programs.
+像窗口(在 [Table API]({{ site.baseurl }}/zh/dev/table/tableApi.html#group-windows) 和 [SQL]({{ site.baseurl }}/zh/dev/table/sql/queries.html#group-windows) )这种基于时间的操作,需要有时间信息。因此,Table API 中的表就需要提供*逻辑时间属性*来表示时间,以及支持时间相关的操作。
-Time attributes can be part of every table schema. They are defined when creating a table from a CREATE TABLE DDL or a `DataStream` or are pre-defined when using a `TableSource`. Once a time attribute has been defined at the beginning, it can be referenced as a field and can be used in time-based operations.
+每种类型的表都可以有时间属性,可以在用CREATE TABLE DDL创建表的时候指定、也可以在 `DataStream` 中指定、也可以在定义 `TableSource` 时指定。一旦时间属性定义好,它就可以像普通列一样使用,也可以在时间相关的操作中使用。
-As long as a time attribute is not modified and is simply forwarded from one part of the query to another, it remains a valid time attribute. Time attributes behave like regular timestamps and can be accessed for calculations. If a time attribute is used in a calculation, it will be materialized and becomes a regular timestamp. Regular timestamps do not cooperate with Flink's time and watermarking system and thus can not be used for time-based operations anymore.
+只要时间属性没有被修改,而是简单地从一个表传递到另一个表,它就仍然是一个有效的时间属性。时间属性可以像普通的时间戳的列一样被使用和计算。一旦时间属性被用在了计算中,它就会被物化,进而变成一个普通的时间戳。普通的时间戳是无法跟 Flink 的时间以及watermark等一起使用的,所以普通的时间戳就无法用在时间相关的操作中。
-Table programs require that the corresponding time characteristic has been specified for the streaming environment:
+Table API 程序需要在 streaming environment 中指定时间属性:
-Processing time
+处理时间
---------------
-Processing time allows a table program to produce results based on the time of the local machine. It is the simplest notion of time but does not provide determinism. It neither requires timestamp extraction nor watermark generation.
+处理时间是基于机器的本地时间来处理数据,它是最简单的一种时间概念,但是它不能提供确定性。它既不需要从数据里获取时间,也不需要生成 watermark。
-There are three ways to define a processing time attribute.
+共有三种方法可以定义处理时间。
-### Defining in create table DDL
+### 在创建表的 DDL 中定义
-The processing time attribute is defined as a computed column in create table DDL using the system `PROCTIME()` function. Please see [CREATE TABLE DDL]({{ site.baseurl }}/dev/table/sql/create.html#create-table) for more information about computed column.
+处理时间属性可以在创建表的 DDL 中用计算列的方式定义,用 `PROCTIME()` 就可以定义处理时间。关于计算列,更多信息可以参考:[CREATE TABLE DDL]({{ site.baseurl }}/zh/dev/table/sql/create.html#create-table)
{% highlight sql %}
CREATE TABLE user_actions (
user_name STRING,
data STRING,
- user_action_time AS PROCTIME() -- declare an additional field as a processing time attribute
+ user_action_time AS PROCTIME() -- 声明一个额外的列作为处理时间属性
) WITH (
...
);
@@ -110,16 +110,16 @@ GROUP BY TUMBLE(user_action_time, INTERVAL '10' MINUTE);
{% endhighlight %}
-### During DataStream-to-Table Conversion
+### 在 DataStream 到 Table 转换时定义
-The processing time attribute is defined with the `.proctime` property during schema definition. The time attribute must only extend the physical schema by an additional logical field. Thus, it can only be defined at the end of the schema definition.
+处理时间属性可以在 schema 定义的时候用 `.proctime` 后缀来定义。时间属性一定不能定义在一个已有字段上,所以它只能定义在 schem 定义的最后。
{% highlight java %}
DataStream> stream = ...;
-// declare an additional logical field as a processing time attribute
+// 声明一个额外的字段作为时间属性字段
Table table = tEnv.fromDataStream(stream, "user_name, data, user_action_time.proctime");
WindowedTable windowedTable = table.window(Tumble.over("10.minutes").on("user_action_time").as("userActionWindow"));
@@ -129,7 +129,7 @@ WindowedTable windowedTable = table.window(Tumble.over("10.minutes").on("user_ac
{% highlight scala %}
val stream: DataStream[(String, String)] = ...
-// declare an additional logical field as a processing time attribute
+// 声明一个额外的字段作为时间属性字段
val table = tEnv.fromDataStream(stream, 'UserActionTimestamp, 'user_name, 'data, 'user_action_time.proctime)
val windowedTable = table.window(Tumble over 10.minutes on 'user_action_time as 'userActionWindow)
@@ -137,14 +137,14 @@ val windowedTable = table.window(Tumble over 10.minutes on 'user_action_time as
-### Using a TableSource
+### 使用 TableSource 定义
-The processing time attribute is defined by a `TableSource` that implements the `DefinedProctimeAttribute` interface. The logical time attribute is appended to the physical schema defined by the return type of the `TableSource`.
+处理时间属性可以在实现了 `DefinedProctimeAttribute` 的 `TableSource` 中定义。逻辑的时间属性会放在 `TableSource` 已有物理字段的最后
{% highlight java %}
-// define a table source with a processing attribute
+// 定义一个由处理时间属性的 table source
public class UserActionSource implements StreamTableSource, DefinedProctimeAttribute {
@Override
@@ -163,7 +163,7 @@ public class UserActionSource implements StreamTableSource, DefinedProctime
@Override
public String getProctimeAttribute() {
- // field with this name will be appended as a third field
+ // 这个名字的列会被追加到最后,作为第三列
return "user_action_time";
}
}
@@ -178,7 +178,7 @@ WindowedTable windowedTable = tEnv
{% highlight scala %}
-// define a table source with a processing attribute
+// 定义一个由处理时间属性的 table source
class UserActionSource extends StreamTableSource[Row] with DefinedProctimeAttribute {
override def getReturnType = {
@@ -194,7 +194,7 @@ class UserActionSource extends StreamTableSource[Row] with DefinedProctimeAttrib
}
override def getProctimeAttribute = {
- // field with this name will be appended as a third field
+ // 这个名字的列会被追加到最后,作为第三列
"user_action_time"
}
}
@@ -209,20 +209,20 @@ val windowedTable = tEnv
-Event time
+事件时间
----------
-Event time allows a table program to produce results based on the time that is contained in every record. This allows for consistent results even in case of out-of-order events or late events. It also ensures replayable results of the table program when reading records from persistent storage.
+事件时间允许程序按照数据中包含的时间来处理,这样可以在有乱序或者晚到的数据的情况下产生一致的处理结果。它可以保证从外部存储读取数据后产生可以复现(replayable)的结果。
-Additionally, event time allows for unified syntax for table programs in both batch and streaming environments. A time attribute in a streaming environment can be a regular field of a record in a batch environment.
+除此之外,事件时间可以让程序在流式和批式作业中使用同样的语法。在流式程序中的事件时间属性,在批式程序中就是一个正常的时间字段。
-In order to handle out-of-order events and distinguish between on-time and late events in streaming, Flink needs to extract timestamps from events and make some kind of progress in time (so-called [watermarks]({{ site.baseurl }}/dev/event_time.html)).
+为了能够处理乱序的事件,并且区分正常到达和晚到的事件,Flink 需要从事件中获取事件时间并且产生 watermark([watermarks]({{ site.baseurl }}/zh/dev/event_time.html))。
-An event time attribute can be defined either in create table DDL or during DataStream-to-Table conversion or by using a TableSource.
+事件时间属性也有类似于处理时间的三种定义方式:在DDL中定义、在 DataStream 到 Table 转换时定义、用 TableSource 定义。
-### Defining in create table DDL
+### 在 DDL 中定义
-The event time attribute is defined using WATERMARK statement in CREATE TABLE DDL. A watermark statement defines a watermark generation expression on an existing event time field, which marks the event time field as event time attribute. Please see [CREATE TABLE DDL]({{ site.baseurl }}/dev/table/sql/create.html#create-table) for more information about watermark statement and watermark strategies.
+事件时间属性可以用 WATERMARK 语句在 CREATE TABLE DDL 中进行定义。WATERMARK 语句在一个已有字段上定义一个 watermark 生成表达式,同时标记这个已有字段为时间属性字段。更多信息可以参考:[CREATE TABLE DDL]({{ site.baseurl }}/zh/dev/table/sql/create.html#create-table)
{% highlight sql %}
@@ -230,7 +230,7 @@ CREATE TABLE user_actions (
user_name STRING,
data STRING,
user_action_time TIMESTAMP(3),
- -- declare user_action_time as event time attribute and use 5 seconds delayed watermark strategy
+ -- 声明 user_action_time 是事件时间属性,并且用 延迟 5 秒的策略来生成 watermark
WATERMARK FOR user_action_time AS user_action_time - INTERVAL '5' SECOND
) WITH (
...
@@ -243,16 +243,16 @@ GROUP BY TUMBLE(user_action_time, INTERVAL '10' MINUTE);
{% endhighlight %}
-### During DataStream-to-Table Conversion
+### 在 DataStream 到 Table 转换时定义
-The event time attribute is defined with the `.rowtime` property during schema definition. [Timestamps and watermarks]({{ site.baseurl }}/dev/event_time.html) must have been assigned in the `DataStream` that is converted.
+事件时间属性可以用 `.rowtime` 后缀在定义 `DataStream` schema 的时候来定义。[时间戳和 watermark]({{ site.baseurl }}/zh/dev/event_time.html) 在这之前一定是在 `DataStream` 上已经定义好了。
-There are two ways of defining the time attribute when converting a `DataStream` into a `Table`. Depending on whether the specified `.rowtime` field name exists in the schema of the `DataStream` or not, the timestamp field is either
+在从 `DataStream` 到 `Table` 转换时定义事件时间属性有两种方式。取决于用 `.rowtime` 后缀修饰的字段名字是否是已有字段,事件时间字段可以是:
-- appended as a new field to the schema or
-- replaces an existing field.
+- 在 schema 的结尾追加一个新的字段
+- 替换一个已经存在的字段。
-In either case the event time timestamp field will hold the value of the `DataStream` event time timestamp.
+不管在哪种情况下,事件时间字段都表示 `DataStream` 中定义的事件的时间戳。
@@ -260,20 +260,19 @@ In either case the event time timestamp field will hold the value of the `DataSt
// Option 1:
-// extract timestamp and assign watermarks based on knowledge of the stream
+// 基于 stream 中的事件产生时间戳和 watermark
DataStream> stream = inputStream.assignTimestampsAndWatermarks(...);
-// declare an additional logical field as an event time attribute
+// 声明一个额外的逻辑字段作为事件时间属性
Table table = tEnv.fromDataStream(stream, "user_name, data, user_action_time.rowtime");
// Option 2:
-// extract timestamp from first field, and assign watermarks based on knowledge of the stream
+// 从第一个字段获取事件时间,并且产生 watermark
DataStream> stream = inputStream.assignTimestampsAndWatermarks(...);
-// the first field has been used for timestamp extraction, and is no longer necessary
-// replace first field with a logical event time attribute
+// 第一个字段已经用作事件时间抽取了,不用再用一个新字段来表示事件时间了
Table table = tEnv.fromDataStream(stream, "user_action_time.rowtime, user_name, data");
// Usage:
@@ -286,20 +285,19 @@ WindowedTable windowedTable = table.window(Tumble.over("10.minutes").on("user_ac
// Option 1:
-// extract timestamp and assign watermarks based on knowledge of the stream
+// 基于 stream 中的事件产生时间戳和 watermark
val stream: DataStream[(String, String)] = inputStream.assignTimestampsAndWatermarks(...)
-// declare an additional logical field as an event time attribute
+// 声明一个额外的逻辑字段作为事件时间属性
val table = tEnv.fromDataStream(stream, 'user_name, 'data, 'user_action_time.rowtime)
// Option 2:
-// extract timestamp from first field, and assign watermarks based on knowledge of the stream
+// 从第一个字段获取事件时间,并且产生 watermark
val stream: DataStream[(Long, String, String)] = inputStream.assignTimestampsAndWatermarks(...)
-// the first field has been used for timestamp extraction, and is no longer necessary
-// replace first field with a logical event time attribute
+// 第一个字段已经用作事件时间抽取了,不用再用一个新字段来表示事件时间了
val table = tEnv.fromDataStream(stream, 'user_action_time.rowtime, 'user_name, 'data)
// Usage:
@@ -309,19 +307,18 @@ val windowedTable = table.window(Tumble over 10.minutes on 'user_action_time as
-### Using a TableSource
+### 使用 TableSource 定义
-The event time attribute is defined by a `TableSource` that implements the `DefinedRowtimeAttributes` interface. The `getRowtimeAttributeDescriptors()` method returns a list of `RowtimeAttributeDescriptor` for describing the final name of a time attribute, a timestamp extractor to derive the values of the attribute, and the watermark strategy associated with the attribute.
+事件时间属性可以在实现了 `DefinedRowTimeAttributes` 的 `TableSource` 中定义。`getRowtimeAttributeDescriptors()` 方法返回 `RowtimeAttributeDescriptor` 的列表,包含了描述事件时间属性的字段名字、如何计算事件时间、以及 watermark 生成策略等信息。
-Please make sure that the `DataStream` returned by the `getDataStream()` method is aligned with the defined time attribute.
-The timestamps of the `DataStream` (the ones which are assigned by a `TimestampAssigner`) are only considered if a `StreamRecordTimestamp` timestamp extractor is defined.
-Watermarks of a `DataStream` are only preserved if a `PreserveWatermarks` watermark strategy is defined.
-Otherwise, only the values of the `TableSource`'s rowtime attribute are relevant.
+同时需要确保 `getDataStream` 返回的 `DataStream` 已经定义好了时间属性。
+只有在定义了 `StreamRecordTimestamp` 时间戳分配器的时候,才认为 `DataStream` 是有时间戳信息的。
+只有定义了 `PreserveWatermarks` watermark 生成策略的 `DataStream` 的 watermark 才会被保留。反之,则只有时间字段的值是生效的。
{% highlight java %}
-// define a table source with a rowtime attribute
+// 定义一个有事件时间属性的 table source
public class UserActionSource implements StreamTableSource, DefinedRowtimeAttributes {
@Override
@@ -334,17 +331,17 @@ public class UserActionSource implements StreamTableSource, DefinedRowtimeA
@Override
public DataStream getDataStream(StreamExecutionEnvironment execEnv) {
- // create stream
+ // 构造 DataStream
// ...
- // assign watermarks based on the "user_action_time" attribute
+ // 基于 "user_action_time" 定义 watermark
DataStream stream = inputStream.assignTimestampsAndWatermarks(...);
return stream;
}
@Override
public List getRowtimeAttributeDescriptors() {
- // Mark the "user_action_time" attribute as event-time attribute.
- // We create one attribute descriptor of "user_action_time".
+ // 标记 "user_action_time" 字段是事件时间字段
+ // 给 "user_action_time" 构造一个时间属性描述符
RowtimeAttributeDescriptor rowtimeAttrDescr = new RowtimeAttributeDescriptor(
"user_action_time",
new ExistingField("user_action_time"),
@@ -364,7 +361,7 @@ WindowedTable windowedTable = tEnv
{% highlight scala %}
-// define a table source with a rowtime attribute
+// 定义一个有事件时间属性的 table source
class UserActionSource extends StreamTableSource[Row] with DefinedRowtimeAttributes {
override def getReturnType = {
@@ -374,16 +371,16 @@ class UserActionSource extends StreamTableSource[Row] with DefinedRowtimeAttribu
}
override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = {
- // create stream
+ // 构造 DataStream
// ...
- // assign watermarks based on the "user_action_time" attribute
+ // 基于 "user_action_time" 定义 watermark
val stream = inputStream.assignTimestampsAndWatermarks(...)
stream
}
override def getRowtimeAttributeDescriptors: util.List[RowtimeAttributeDescriptor] = {
- // Mark the "user_action_time" attribute as event-time attribute.
- // We create one attribute descriptor of "user_action_time".
+ // 标记 "user_action_time" 字段是事件时间字段
+ // 给 "user_action_time" 构造一个时间属性描述符
val rowtimeAttrDescr = new RowtimeAttributeDescriptor(
"user_action_time",
new ExistingField("user_action_time"),
diff --git a/docs/dev/table/types.zh.md b/docs/dev/table/types.zh.md
index 4dc42af4270b90e5c5d887214fb3ddcfde5980a6..252ff021bf1f4c889ee50163a223986667a1b0ea 100644
--- a/docs/dev/table/types.zh.md
+++ b/docs/dev/table/types.zh.md
@@ -1,5 +1,5 @@
---
-title: "Data Types"
+title: "数据类型"
nav-parent_id: tableapi
nav-pos: 20
---
@@ -22,62 +22,47 @@ specific language governing permissions and limitations
under the License.
-->
-Due to historical reasons, before Flink 1.9, Flink's Table & SQL API data types were
-tightly coupled to Flink's `TypeInformation`. `TypeInformation` is used in the DataStream
-and DataSet API and is sufficient to describe all information needed to serialize and
-deserialize JVM-based objects in a distributed setting.
+由于历史原因,在 Flink 1.9 之前,Flink Table & SQL API 的数据类型与 Flink 的 `TypeInformation` 耦合紧密。`TypeInformation` 在 DataStream 和 DataSet API 中被使用,并且足以用来用于描述分布式环境中 JVM 对象的序列化和反序列化操作所需的全部信息。
-However, `TypeInformation` was not designed to represent logical types independent of
-an actual JVM class. In the past, it was difficult to map SQL standard types to this
-abstraction. Furthermore, some types were not SQL-compliant and introduced without a
-bigger picture in mind.
+然而,`TypeInformation` 并不是为独立于 JVM class 的逻辑类型而设计的。之前很难将 SQL 的标准类型映射到 `TypeInformation` 抽象。此外,有一些类型并不是兼容 SQL 的并且引入的时候没有长远规划过。
-Starting with Flink 1.9, the Table & SQL API will receive a new type system that serves as a long-term
-solution for API stability and standard compliance.
+从 Flink 1.9 开始,Table & SQL API 开始启用一种新的类型系统作为长期解决方案,用来保持 API 稳定性和 SQL 标准的兼容性。
-Reworking the type system is a major effort that touches almost all user-facing interfaces. Therefore, its
-introduction spans multiple releases, and the community aims to finish this effort by Flink 1.10.
+重新设计类型系统是一项涉及几乎所有的面向用户接口的重大工作。因此,它的引入跨越多个版本,社区的目标是在 Flink 1.10 完成这项工作。
-Due to the simultaneous addition of a new planner for table programs (see [FLINK-11439](https://issues.apache.org/jira/browse/FLINK-11439)),
-not every combination of planner and data type is supported. Furthermore, planners might not support every
-data type with the desired precision or parameter.
+同时由于为 Table 编程添加了新的 Planner 详见([FLINK-11439](https://issues.apache.org/jira/browse/FLINK-11439)), 并不是每种 Planner 都支持所有的数据类型。此外,Planner 对于数据类型的精度和参数化支持也可能是不完整的。
-Attention Please see the planner compatibility table and limitations
-section before using a data type.
+注意 在使用数据类型之前请参阅 Planner 的兼容性表和局限性章节。
* This will be replaced by the TOC
{:toc}
-Data Type
+数据类型
---------
-A *data type* describes the logical type of a value in the table ecosystem. It can be used to declare input and/or
-output types of operations.
+*数据类型* 描述 Table 编程环境中的值的逻辑类型。它可以被用来声明操作的输入输出类型。
-Flink's data types are similar to the SQL standard's *data type* terminology but also contain information
-about the nullability of a value for efficient handling of scalar expressions.
+Flink 的数据类型和 SQL 标准的 *数据类型* 术语类似,但也包含了可空属性,可以被用于标量表达式(scalar expression)的优化。
-Examples of data types are:
+数据类型的示例:
- `INT`
- `INT NOT NULL`
- `INTERVAL DAY TO SECOND(3)`
- `ROW, myOtherField TIMESTAMP(3)>`
-A list of all pre-defined data types can be found [below](#list-of-data-types).
+全部的预定义数据类型见[下面](#数据类型列表)列表。
-### Data Types in the Table API
+### Table API 的数据类型
-Users of the JVM-based API work with instances of `org.apache.flink.table.types.DataType` within the Table API or when
-defining connectors, catalogs, or user-defined functions.
+JVM API 的用户可以在 Table API 中使用 `org.apache.flink.table.types.DataType` 的实例,以及定义连接器(Connector)、Catalog 或者用户自定义函数(User-Defined Function)。
-A `DataType` instance has two responsibilities:
-- **Declaration of a logical type** which does not imply a concrete physical representation for transmission
-or storage but defines the boundaries between JVM-based languages and the table ecosystem.
-- *Optional:* **Giving hints about the physical representation of data to the planner** which is useful at the edges to other APIs .
+一个 `DataType` 实例有两个作用:
+- **逻辑类型的声明**,它不表达具体物理类型的存储和转换,但是定义了基于 JVM 的语言和 Table 编程环境之间的边界。
+- *可选的:* **向 Planner 提供有关数据的物理表示的提示**,这对于边界 API 很有用。
-For JVM-based languages, all pre-defined data types are available in `org.apache.flink.table.api.DataTypes`.
+对于基于 JVM 的语言,所有预定义的数据类型都在 `org.apache.flink.table.api.DataTypes` 里提供。
-It is recommended to add a star import to your table programs for having a fluent API:
+建议使用星号将全部的 API 导入到 Table 程序中以便于使用:
@@ -99,71 +84,61 @@ val t: DataType = INTERVAL(DAY(), SECOND(3));
-#### Physical Hints
+#### 物理提示
-Physical hints are required at the edges of the table ecosystem where the SQL-based type system ends and
-programming-specific data types are required. Hints indicate the data format that an implementation
-expects.
+在 Table 编程环境中,基于 SQL 的类型系统与程序指定的数据类型之间需要物理提示。该提示指出了实现预期的数据格式。
-For example, a data source could express that it produces values for logical `TIMESTAMP`s using a `java.sql.Timestamp` class
-instead of using `java.time.LocalDateTime` which would be the default. With this information, the runtime is able to convert
-the produced class into its internal data format. In return, a data sink can declare the data format it consumes from the runtime.
+例如,Data Source 能够使用类 `java.sql.Timestamp` 来表达逻辑上的 `TIMESTAMP` 产生的值,而不是使用缺省的 `java.time.LocalDateTime`。有了这些信息,运行时就能够将产生的类转换为其内部数据格式。反过来,Data Sink 可以声明它从运行时消费的数据格式。
-Here are some examples of how to declare a bridging conversion class:
+下面是一些如何声明桥接转换类的示例:
{% highlight java %}
-// tell the runtime to not produce or consume java.time.LocalDateTime instances
-// but java.sql.Timestamp
+// 告诉运行时不要产生或者消费 java.time.LocalDateTime 实例
+// 而是使用 java.sql.Timestamp
DataType t = DataTypes.TIMESTAMP(3).bridgedTo(java.sql.Timestamp.class);
-// tell the runtime to not produce or consume boxed integer arrays
-// but primitive int arrays
+// 告诉运行时不要产生或者消费装箱的整数数组
+// 而是使用基本数据类型的整数数组
DataType t = DataTypes.ARRAY(DataTypes.INT().notNull()).bridgedTo(int[].class);
{% endhighlight %}
{% highlight scala %}
-// tell the runtime to not produce or consume java.time.LocalDateTime instances
-// but java.sql.Timestamp
+// 告诉运行时不要产生或者消费 java.time.LocalDateTime 实例
+// 而是使用 java.sql.Timestamp
val t: DataType = DataTypes.TIMESTAMP(3).bridgedTo(classOf[java.sql.Timestamp]);
-// tell the runtime to not produce or consume boxed integer arrays
-// but primitive int arrays
+// 告诉运行时不要产生或者消费装箱的整数数组
+// 而是使用基本数据类型的整数数组
val t: DataType = DataTypes.ARRAY(DataTypes.INT().notNull()).bridgedTo(classOf[Array[Int]]);
{% endhighlight %}
-Attention Please note that physical hints are usually only required if the
-API is extended. Users of predefined sources/sinks/functions do not need to define such hints. Hints within
-a table program (e.g. `field.cast(TIMESTAMP(3).bridgedTo(Timestamp.class))`) are ignored.
+注意 请注意,通常只有在扩展 API 时才需要物理提示。
+预定义的 Source、Sink、Function 的用户不需要定义这样的提示。在 Table 编程中(例如 `field.cast(TIMESTAMP(3).bridgedTo(Timestamp.class))`)这些提示将被忽略。
-Planner Compatibility
+Planner 兼容性
---------------------
-As mentioned in the introduction, reworking the type system will span multiple releases, and the support of each data
-type depends on the used planner. This section aims to summarize the most significant differences.
+正如简介里提到的,重新开发类型系统将跨越多个版本,每个数据类型的支持取决于使用的 Planner。本节旨在总结最重要的差异。
-### Old Planner
+### 旧的 Planner
-Flink's old planner, introduced before Flink 1.9, primarily supports type information. It has only limited
-support for data types. It is possible to declare data types that can be translated into type information such that the
-old planner understands them.
+Flink 1.9 之前引入的旧的 Planner 主要支持类型信息(Type Information),它只对数据类型提供有限的支持,可以声明能够转换为类型信息的数据类型,以便旧的 Planner 能够理解它们。
-The following table summarizes the difference between data type and type information. Most simple types, as well as the
-row type remain the same. Time types, array types, and the decimal type need special attention. Other hints as the ones
-mentioned are not allowed.
+下表总结了数据类型和类型信息之间的区别。大多数简单类型以及 Row 类型保持不变。Time 类型、 Array 类型和 Decimal 类型需要特别注意。不允许使用其他的类型提示。
-For the *Type Information* column the table omits the prefix `org.apache.flink.table.api.Types`.
+对于 *类型信息* 列,该表省略了前缀 `org.apache.flink.table.api.Types`。
-For the *Data Type Representation* column the table omits the prefix `org.apache.flink.table.api.DataTypes`.
+对于 *数据类型表示* 列,该表省略了前缀 `org.apache.flink.table.api.DataTypes`。
-| Type Information | Java Expression String | Data Type Representation | Remarks for Data Type |
+| 类型信息 | Java 表达式字符串 | 数据类型表示 | 数据类型备注 |
|:-----------------|:-----------------------|:-------------------------|:----------------------|
| `STRING()` | `STRING` | `STRING()` | |
| `BOOLEAN()` | `BOOLEAN` | `BOOLEAN()` | |
@@ -174,35 +149,33 @@ For the *Data Type Representation* column the table omits the prefix `org.apache
| `FLOAT()` | `FLOAT` | `FLOAT()` | |
| `DOUBLE()` | `DOUBLE` | `DOUBLE()` | |
| `ROW(...)` | `ROW<...>` | `ROW(...)` | |
-| `BIG_DEC()` | `DECIMAL` | [`DECIMAL()`] | Not a 1:1 mapping as precision and scale are ignored and Java's variable precision and scale are used. |
+| `BIG_DEC()` | `DECIMAL` | [`DECIMAL()`] | 不是 1:1 的映射,因为精度和小数位被忽略,Java 的可变精度和小数位被使用。 |
| `SQL_DATE()` | `SQL_DATE` | `DATE()` `.bridgedTo(java.sql.Date.class)` | |
| `SQL_TIME()` | `SQL_TIME` | `TIME(0)` `.bridgedTo(java.sql.Time.class)` | |
| `SQL_TIMESTAMP()` | `SQL_TIMESTAMP` | `TIMESTAMP(3)` `.bridgedTo(java.sql.Timestamp.class)` | |
| `INTERVAL_MONTHS()` | `INTERVAL_MONTHS` | `INTERVAL(MONTH())` `.bridgedTo(Integer.class)` | |
| `INTERVAL_MILLIS()` | `INTERVAL_MILLIS` | `INTERVAL(DataTypes.SECOND(3))` `.bridgedTo(Long.class)` | |
-| `PRIMITIVE_ARRAY(...)` | `PRIMITIVE_ARRAY<...>` | `ARRAY(DATATYPE.notNull()` `.bridgedTo(PRIMITIVE.class))` | Applies to all JVM primitive types except for `byte`. |
+| `PRIMITIVE_ARRAY(...)` | `PRIMITIVE_ARRAY<...>` | `ARRAY(DATATYPE.notNull()` `.bridgedTo(PRIMITIVE.class))` | 应用于除 `byte` 外的全部 JVM 基本数据类型。 |
| `PRIMITIVE_ARRAY(BYTE())` | `PRIMITIVE_ARRAY` | `BYTES()` | |
| `OBJECT_ARRAY(...)` | `OBJECT_ARRAY<...>` | `ARRAY(` `DATATYPE.bridgedTo(OBJECT.class))` | |
| `MULTISET(...)` | | `MULTISET(...)` | |
| `MAP(..., ...)` | `MAP<...,...>` | `MAP(...)` | |
-| other generic types | | `RAW(...)` | |
+| 其他通用类型 | | `RAW(...)` | |
-Attention If there is a problem with the new type system. Users
-can fallback to type information defined in `org.apache.flink.table.api.Types` at any time.
+注意 如果对于新的类型系统有任何疑问,用户可以随时切换到 `org.apache.flink.table.api.Types` 中定义的 type information。
-### New Blink Planner
+### 新的 Blink Planner
-The new Blink planner supports all of types of the old planner. This includes in particular
-the listed Java expression strings and type information.
+新的 Blink Planner 支持旧的 Planner 的全部类型,尤其包括列出的 Java 表达式字符串和类型信息。
-The following data types are supported:
+支持以下数据类型:
-| Data Type | Remarks for Data Type |
+| 数据类型 | 数据类型的备注 |
|:----------|:----------------------|
-| `STRING` | `CHAR` and `VARCHAR` are not supported yet. |
+| `STRING` | `CHAR` 和 `VARCHAR` 暂不支持。 |
| `BOOLEAN` | |
-| `BYTES` | `BINARY` and `VARBINARY` are not supported yet. |
-| `DECIMAL` | Supports fixed precision and scale. |
+| `BYTES` | `BINARY` 和 `VARBINARY` 暂不支持。 |
+| `DECIMAL` | 支持固定精度和小数位数。 |
| `TINYINT` | |
| `SMALLINT` | |
| `INTEGER` | |
@@ -210,40 +183,37 @@ The following data types are supported:
| `FLOAT` | |
| `DOUBLE` | |
| `DATE` | |
-| `TIME` | Supports only a precision of `0`. |
-| `TIMESTAMP` | Supports only a precision of `3`. |
-| `TIMESTAMP WITH LOCAL TIME ZONE` | Supports only a precision of `3`. |
-| `INTERVAL` | Supports only interval of `MONTH` and `SECOND(3)`. |
+| `TIME` | 支持的精度仅为 `0`。 |
+| `TIMESTAMP` | 支持的精度仅为 `3`。 |
+| `TIMESTAMP WITH LOCAL TIME ZONE` | 支持的精度仅为 `3`。 |
+| `INTERVAL` | 仅支持 `MONTH` 和 `SECOND(3)` 区间。 |
| `ARRAY` | |
| `MULTISET` | |
| `MAP` | |
| `ROW` | |
| `RAW` | |
-Limitations
+局限性
-----------
-**Java Expression String**: Java expression strings in the Table API such as `table.select("field.cast(STRING)")`
-have not been updated to the new type system yet. Use the string representations declared in
-the [old planner section](#old-planner).
+**Java 表达式字符串**:Table API 中的 Java 表达式字符串,例如 `table.select("field.cast(STRING)")`,尚未被更新到新的类型系统中,使用[旧的 Planner 章节](#旧的-planner)中声明的字符串来表示。
-**Connector Descriptors and SQL Client**: Descriptor string representations have not been updated to the new
-type system yet. Use the string representation declared in the [Connect to External Systems section](./connect.html#type-strings)
+**连接器描述符和 SQL 客户端**:描述符字符串的表示形式尚未更新到新的类型系统。使用在[连接到外部系统章节](./connect.html#type-strings)中声明的字符串表示。
-**User-defined Functions**: User-defined functions cannot declare a data type yet.
+**用户自定义函数**:用户自定义函数尚不能声明数据类型。
-List of Data Types
+数据类型列表
------------------
-This section lists all pre-defined data types. For the JVM-based Table API those types are also available in `org.apache.flink.table.api.DataTypes`.
+本节列出了所有预定义的数据类型。对于基于 JVM 的 Table API,这些类型也可以从 `org.apache.flink.table.api.DataTypes` 中找到。
-### Character Strings
+### 字符串
#### `CHAR`
-Data type of a fixed-length character string.
+固定长度字符串的数据类型。
-**Declaration**
+**声明**
@@ -262,21 +232,20 @@ DataTypes.CHAR(n)
-The type can be declared using `CHAR(n)` where `n` is the number of code points. `n` must have a value between `1`
-and `2,147,483,647` (both inclusive). If no length is specified, `n` is equal to `1`.
+此类型用 `CHAR(n)` 声明,其中 `n` 表示字符数量。`n` 的值必须在 `1` 和 `2,147,483,647` 之间(含边界值)。如果未指定长度,`n` 等于 `1`。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:------------------------|
-|`java.lang.String` | X | X | *Default* |
-|`byte[]` | X | X | Assumes UTF-8 encoding. |
+|`java.lang.String` | X | X | *缺省* |
+|`byte[]` | X | X | 假设使用 UTF-8 编码。 |
#### `VARCHAR` / `STRING`
-Data type of a variable-length character string.
+可变长度字符串的数据类型。
-**Declaration**
+**声明**
@@ -299,25 +268,24 @@ DataTypes.STRING()
-The type can be declared using `VARCHAR(n)` where `n` is the maximum number of code points. `n` must have a value
-between `1` and `2,147,483,647` (both inclusive). If no length is specified, `n` is equal to `1`.
+此类型用 `VARCHAR(n)` 声明,其中 `n` 表示最大的字符数量。`n` 的值必须在 `1` 和 `2,147,483,647` 之间(含边界值)。如果未指定长度,`n` 等于 `1`。
-`STRING` is a synonym for `VARCHAR(2147483647)`.
+`STRING` 等价于 `VARCHAR(2147483647)`.
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:------------------------|
-|`java.lang.String` | X | X | *Default* |
-|`byte[]` | X | X | Assumes UTF-8 encoding. |
+|`java.lang.String` | X | X | *缺省* |
+|`byte[]` | X | X | 假设使用 UTF-8 编码。 |
-### Binary Strings
+### 二进制字符串
#### `BINARY`
-Data type of a fixed-length binary string (=a sequence of bytes).
+固定长度二进制字符串的数据类型(=字节序列)。
-**Declaration**
+**声明**
@@ -336,20 +304,19 @@ DataTypes.BINARY(n)
-The type can be declared using `BINARY(n)` where `n` is the number of bytes. `n` must have a value
-between `1` and `2,147,483,647` (both inclusive). If no length is specified, `n` is equal to `1`.
+此类型用 `BINARY(n)` 声明,其中 `n` 是字节数量。`n` 的值必须在 `1` 和 `2,147,483,647` 之间(含边界值)。如果未指定长度,`n` 等于 `1`。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:------------------------|
-|`byte[]` | X | X | *Default* |
+|`byte[]` | X | X | *缺省* |
#### `VARBINARY` / `BYTES`
-Data type of a variable-length binary string (=a sequence of bytes).
+可变长度二进制字符串的数据类型(=字节序列)。
-**Declaration**
+**声明**
@@ -372,25 +339,23 @@ DataTypes.BYTES()
-The type can be declared using `VARBINARY(n)` where `n` is the maximum number of bytes. `n` must
-have a value between `1` and `2,147,483,647` (both inclusive). If no length is specified, `n` is
-equal to `1`.
+此类型用 `VARBINARY(n)` 声明,其中 `n` 是最大的字节数量。`n` 的值必须在 `1` 和 `2,147,483,647` 之间(含边界值)。如果未指定长度,`n` 等于 `1`。
-`BYTES` is a synonym for `VARBINARY(2147483647)`.
+`BYTES` 等价于 `VARBINARY(2147483647)`。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:------------------------|
-|`byte[]` | X | X | *Default* |
+|`byte[]` | X | X | *缺省* |
-### Exact Numerics
+### 精确数值
#### `DECIMAL`
-Data type of a decimal number with fixed precision and scale.
+精度和小数位数固定的十进制数字的数据类型。
-**Declaration**
+**声明**
@@ -418,25 +383,21 @@ DataTypes.DECIMAL(p, s)
-The type can be declared using `DECIMAL(p, s)` where `p` is the number of digits in a
-number (*precision*) and `s` is the number of digits to the right of the decimal point
-in a number (*scale*). `p` must have a value between `1` and `38` (both inclusive). `s`
-must have a value between `0` and `p` (both inclusive). The default value for `p` is 10.
-The default value for `s` is `0`.
+此类型用 `DECIMAL(p, s)` 声明,其中 `p` 是数字的位数(*精度*),`s` 是数字中小数点右边的位数(*尾数*)。`p` 的值必须介于 `1` 和 `38` 之间(含边界值)。`s` 的值必须介于 `0` 和 `p` 之间(含边界值)。其中 `p` 的缺省值是 `10`,`s` 的缺省值是 `0`。
-`NUMERIC(p, s)` and `DEC(p, s)` are synonyms for this type.
+`NUMERIC(p, s)` 和 `DEC(p, s)` 都等价于这个类型。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:----------------------|:-----:|:------:|:------------------------|
-|`java.math.BigDecimal` | X | X | *Default* |
+|`java.math.BigDecimal` | X | X | *缺省* |
#### `TINYINT`
-Data type of a 1-byte signed integer with values from `-128` to `127`.
+1 字节有符号整数的数据类型,其值从 `-128` to `127`。
-**Declaration**
+**声明**
@@ -454,18 +415,18 @@ DataTypes.TINYINT()
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:---------------------------------------------|
-|`java.lang.Byte` | X | X | *Default* |
-|`byte` | X | (X) | Output only if type is not nullable. |
+|`java.lang.Byte` | X | X | *缺省* |
+|`byte` | X | (X) | 仅当类型不可为空时才输出。 |
#### `SMALLINT`
-Data type of a 2-byte signed integer with values from `-32,768` to `32,767`.
+2 字节有符号整数的数据类型,其值从 `-32,768` 到 `32,767`。
-**Declaration**
+**声明**
@@ -483,18 +444,18 @@ DataTypes.SMALLINT()
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:---------------------------------------------|
-|`java.lang.Short` | X | X | *Default* |
-|`short` | X | (X) | Output only if type is not nullable. |
+|`java.lang.Short` | X | X | *缺省* |
+|`short` | X | (X) | 仅当类型不可为空时才输出。 |
#### `INT`
-Data type of a 4-byte signed integer with values from `-2,147,483,648` to `2,147,483,647`.
+4 字节有符号整数的数据类型,其值从 `-2,147,483,648` 到 `2,147,483,647`。
-**Declaration**
+**声明**
@@ -514,21 +475,20 @@ DataTypes.INT()
-`INTEGER` is a synonym for this type.
+`INTEGER` 等价于此类型。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:---------------------------------------------|
-|`java.lang.Integer` | X | X | *Default* |
-|`int` | X | (X) | Output only if type is not nullable. |
+|`java.lang.Integer` | X | X | *缺省* |
+|`int` | X | (X) | 仅当类型不可为空时才输出。 |
#### `BIGINT`
-Data type of an 8-byte signed integer with values from `-9,223,372,036,854,775,808` to
-`9,223,372,036,854,775,807`.
+8 字节有符号整数的数据类型,其值从 `-9,223,372,036,854,775,808` 到 `9,223,372,036,854,775,807`。
-**Declaration**
+**声明**
@@ -546,22 +506,22 @@ DataTypes.BIGINT()
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:---------------------------------------------|
-|`java.lang.Long` | X | X | *Default* |
-|`long` | X | (X) | Output only if type is not nullable. |
+|`java.lang.Long` | X | X | *缺省* |
+|`long` | X | (X) | 仅当类型不可为空时才输出。 |
-### Approximate Numerics
+### 近似数值
#### `FLOAT`
-Data type of a 4-byte single precision floating point number.
+4 字节单精度浮点数的数据类型。
-Compared to the SQL standard, the type does not take parameters.
+与 SQL 标准相比,该类型不带参数。
-**Declaration**
+**声明**
@@ -579,18 +539,18 @@ DataTypes.FLOAT()
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:---------------------------------------------|
-|`java.lang.Float` | X | X | *Default* |
-|`float` | X | (X) | Output only if type is not nullable. |
+|`java.lang.Float` | X | X | *缺省* |
+|`float` | X | (X) | 仅当类型不可为空时才输出。 |
#### `DOUBLE`
-Data type of an 8-byte double precision floating point number.
+8 字节双精度浮点数的数据类型。
-**Declaration**
+**声明**
@@ -610,25 +570,24 @@ DataTypes.DOUBLE()
-`DOUBLE PRECISION` is a synonym for this type.
+`DOUBLE PRECISION` 等价于此类型。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:---------------------------------------------|
-|`java.lang.Double` | X | X | *Default* |
-|`double` | X | (X) | Output only if type is not nullable. |
+|`java.lang.Double` | X | X | *缺省* |
+|`double` | X | (X) | 仅当类型不可为空时才输出。 |
-### Date and Time
+### 日期和时间
#### `DATE`
-Data type of a date consisting of `year-month-day` with values ranging from `0000-01-01`
-to `9999-12-31`.
+日期的数据类型由 `year-month-day` 组成,范围从 `0000-01-01` 到 `9999-12-31`。
-Compared to the SQL standard, the range starts at year `0000`.
+与 SQL 标准相比,年的范围从 `0000` 开始。
-**Declaration**
+**声明**
@@ -646,25 +605,22 @@ DataTypes.DATE()
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:---------------------|:-----:|:------:|:---------------------------------------------|
-|`java.time.LocalDate` | X | X | *Default* |
+|`java.time.LocalDate` | X | X | *缺省* |
|`java.sql.Date` | X | X | |
-|`java.lang.Integer` | X | X | Describes the number of days since epoch. |
-|`int` | X | (X) | Describes the number of days since epoch. Output only if type is not nullable. |
+|`java.lang.Integer` | X | X | 描述从 Epoch 算起的天数。 |
+|`int` | X | (X) | 描述从 Epoch 算起的天数。 仅当类型不可为空时才输出。 |
#### `TIME`
-Data type of a time *without* time zone consisting of `hour:minute:second[.fractional]` with
-up to nanosecond precision and values ranging from `00:00:00.000000000` to
-`23:59:59.999999999`.
+*不带*时区的时间数据类型,由 `hour:minute:second[.fractional]` 组成,精度达到纳秒,范围从 `00:00:00.000000000` 到 `23:59:59.999999999`。
-Compared to the SQL standard, leap seconds (`23:59:60` and `23:59:61`) are not supported as
-the semantics are closer to `java.time.LocalTime`. A time *with* time zone is not provided.
+与 SQL 标准相比,不支持闰秒(`23:59:60` 和 `23:59:61`),语义上更接近于 `java.time.LocalTime`。没有提供*带有*时区的时间。
-**Declaration**
+**声明**
@@ -683,35 +639,28 @@ DataTypes.TIME(p)
-The type can be declared using `TIME(p)` where `p` is the number of digits of fractional
-seconds (*precision*). `p` must have a value between `0` and `9` (both inclusive). If no
-precision is specified, `p` is equal to `0`.
+此类型用 `TIME(p)` 声明,其中 `p` 是秒的小数部分的位数(*精度*)。`p` 的值必须介于 `0` 和 `9` 之间(含边界值)。如果未指定精度,则 `p` 等于 `0`。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 |输入 |输出 |备注 |
|:---------------------|:-----:|:------:|:----------------------------------------------------|
-|`java.time.LocalTime` | X | X | *Default* |
+|`java.time.LocalTime` | X | X | *缺省* |
|`java.sql.Time` | X | X | |
-|`java.lang.Integer` | X | X | Describes the number of milliseconds of the day. |
-|`int` | X | (X) | Describes the number of milliseconds of the day. Output only if type is not nullable. |
-|`java.lang.Long` | X | X | Describes the number of nanoseconds of the day. |
-|`long` | X | (X) | Describes the number of nanoseconds of the day. Output only if type is not nullable. |
+|`java.lang.Integer` | X | X | 描述自当天以来的毫秒数。 |
+|`int` | X | (X) | 描述自当天以来的毫秒数。 仅当类型不可为空时才输出。 |
+|`java.lang.Long` | X | X | 描述自当天以来的纳秒数。 |
+|`long` | X | (X) | 描述自当天以来的纳秒数。 仅当类型不可为空时才输出。 |
#### `TIMESTAMP`
-Data type of a timestamp *without* time zone consisting of `year-month-day hour:minute:second[.fractional]`
-with up to nanosecond precision and values ranging from `0000-01-01 00:00:00.000000000` to
-`9999-12-31 23:59:59.999999999`.
+*不带*时区的时间戳数据类型,由 `year-month-day hour:minute:second[.fractional]` 组成,精度达到纳秒,范围从 `0000-01-01 00:00:00.000000000` 到 `9999-12-31 23:59:59.999999999`。
-Compared to the SQL standard, leap seconds (`23:59:60` and `23:59:61`) are not supported as
-the semantics are closer to `java.time.LocalDateTime`.
+与 SQL 标准相比,不支持闰秒(`23:59:60` 和 `23:59:61`),语义上更接近于 `java.time.LocalDateTime`。
-A conversion from and to `BIGINT` (a JVM `long` type) is not supported as this would imply a time
-zone. However, this type is time zone free. For more `java.time.Instant`-like semantics use
-`TIMESTAMP WITH LOCAL TIME ZONE`.
+不支持和 `BIGINT`(JVM `long` 类型)互相转换,因为这意味着有时区,然而此类型是无时区的。对于语义上更接近于 `java.time.Instant` 的需求请使用 `TIMESTAMP WITH LOCAL TIME ZONE`。
-**Declaration**
+**声明**
@@ -733,33 +682,27 @@ DataTypes.TIMESTAMP(p)
-The type can be declared using `TIMESTAMP(p)` where `p` is the number of digits of fractional
-seconds (*precision*). `p` must have a value between `0` and `9` (both inclusive). If no precision
-is specified, `p` is equal to `6`.
+此类型用 `TIMESTAMP(p)` 声明,其中 `p` 是秒的小数部分的位数(*精度*)。`p` 的值必须介于 `0` 和 `9` 之间(含边界值)。如果未指定精度,则 `p` 等于 `6`。
-`TIMESTAMP(p) WITHOUT TIME ZONE` is a synonym for this type.
+`TIMESTAMP(p) WITHOUT TIME ZONE` 等价于此类型。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------------|:-----:|:------:|:----------------------------------------------------|
-|`java.time.LocalDateTime` | X | X | *Default* |
+|`java.time.LocalDateTime` | X | X | *缺省* |
|`java.sql.Timestamp` | X | X | |
#### `TIMESTAMP WITH TIME ZONE`
-Data type of a timestamp *with* time zone consisting of `year-month-day hour:minute:second[.fractional] zone`
-with up to nanosecond precision and values ranging from `0000-01-01 00:00:00.000000000 +14:59` to
-`9999-12-31 23:59:59.999999999 -14:59`.
+*带有*时区的时间戳数据类型,由 `year-month-day hour:minute:second[.fractional] zone` 组成,精度达到纳秒,范围从 `0000-01-01 00:00:00.000000000 +14:59` 到
+`9999-12-31 23:59:59.999999999 -14:59`。
-Compared to the SQL standard, leap seconds (`23:59:60` and `23:59:61`) are not supported as the semantics
-are closer to `java.time.OffsetDateTime`.
+与 SQL 标准相比,不支持闰秒(`23:59:60` 和 `23:59:61`),语义上更接近于 `java.time.OffsetDateTime`。
-Compared to `TIMESTAMP WITH LOCAL TIME ZONE`, the time zone offset information is physically
-stored in every datum. It is used individually for every computation, visualization, or communication
-to external systems.
+与 `TIMESTAMP WITH LOCAL TIME ZONE` 相比,时区偏移信息物理存储在每个数据中。它单独用于每次计算、可视化或者与外部系统的通信。
-**Declaration**
+**声明**
-The type can be declared using `TIMESTAMP(p) WITH TIME ZONE` where `p` is the number of digits of
-fractional seconds (*precision*). `p` must have a value between `0` and `9` (both inclusive). If no
-precision is specified, `p` is equal to `6`.
+此类型用 `TIMESTAMP(p) WITH TIME ZONE` 声明,其中 `p` 是秒的小数部分的位数(*精度*)。`p` 的值必须介于 `0` 和 `9` 之间(含边界值)。如果未指定精度,则 `p` 等于 `6`。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:--------------------------|:-----:|:------:|:---------------------|
-|`java.time.OffsetDateTime` | X | X | *Default* |
-|`java.time.ZonedDateTime` | X | | Ignores the zone ID. |
+|`java.time.OffsetDateTime` | X | X | *缺省* |
+|`java.time.ZonedDateTime` | X | | 忽略时区 ID。 |
#### `TIMESTAMP WITH LOCAL TIME ZONE`
-Data type of a timestamp *with local* time zone consisting of `year-month-day hour:minute:second[.fractional] zone`
-with up to nanosecond precision and values ranging from `0000-01-01 00:00:00.000000000 +14:59` to
-`9999-12-31 23:59:59.999999999 -14:59`.
+*带有本地*时区的时间戳数据类型,由 `year-month-day hour:minute:second[.fractional] zone` 组成,精度达到纳秒,范围从 `0000-01-01 00:00:00.000000000 +14:59` 到
+`9999-12-31 23:59:59.999999999 -14:59`。
-Leap seconds (`23:59:60` and `23:59:61`) are not supported as the semantics are closer to `java.time.OffsetDateTime`.
+不支持闰秒(`23:59:60` 和 `23:59:61`),语义上更接近于 `java.time.OffsetDateTime`。
-Compared to `TIMESTAMP WITH TIME ZONE`, the time zone offset information is not stored physically
-in every datum. Instead, the type assumes `java.time.Instant` semantics in UTC time zone at
-the edges of the table ecosystem. Every datum is interpreted in the local time zone configured in
-the current session for computation and visualization.
+与 `TIMESTAMP WITH TIME ZONE` 相比,时区偏移信息并非物理存储在每个数据中。相反,此类型在 Table 编程环境的 UTC 时区中采用 `java.time.Instant` 语义。每个数据都在当前会话中配置的本地时区中进行解释,以便用于计算和可视化。
-This type fills the gap between time zone free and time zone mandatory timestamp types by allowing
-the interpretation of UTC timestamps according to the configured session time zone.
+此类型允许根据配置的会话时区来解释 UTC 时间戳,从而填补了时区无关和时区相关的时间戳类型之间的鸿沟。
-**Declaration**
+**声明**
-The type can be declared using `TIMESTAMP(p) WITH LOCAL TIME ZONE` where `p` is the number
-of digits of fractional seconds (*precision*). `p` must have a value between `0` and `9`
-(both inclusive). If no precision is specified, `p` is equal to `6`.
+此类型用 `TIMESTAMP(p) WITH LOCAL TIME ZONE` 声明,其中 `p` 是秒的小数部分的位数(*精度*)。`p` 的值必须介于 `0` 和 `9` 之间(含边界值)。如果未指定精度,则 `p` 等于 `6`。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 |输入 |输出 |备注 |
|:-------------------|:-----:|:------:|:--------------------------------------------------|
-|`java.time.Instant` | X | X | *Default* |
-|`java.lang.Integer` | X | X | Describes the number of seconds since epoch. |
-|`int` | X | (X) | Describes the number of seconds since epoch. Output only if type is not nullable. |
-|`java.lang.Long` | X | X | Describes the number of milliseconds since epoch. |
-|`long` | X | (X) | Describes the number of milliseconds since epoch. Output only if type is not nullable. |
+|`java.time.Instant` | X | X | *缺省* |
+|`java.lang.Integer` | X | X | 描述从 Epoch 算起的秒数。 |
+|`int` | X | (X) | 描述从 Epoch 算起的秒数。 仅当类型不可为空时才输出。 |
+|`java.lang.Long` | X | X | 描述从 Epoch 算起的毫秒数。 |
+|`long` | X | (X) | 描述从 Epoch 算起的毫秒数。 仅当类型不可为空时才输出。 |
#### `INTERVAL YEAR TO MONTH`
-Data type for a group of year-month interval types.
+一组 Year-Month Interval 数据类型。
-The type must be parameterized to one of the following resolutions:
-- interval of years,
-- interval of years to months,
-- or interval of months.
+此类型必被参数化为以下情况中的一种:
+- Year 时间间隔、
+- Year-Month 时间间隔、
+- Month 时间间隔。
-An interval of year-month consists of `+years-months` with values ranging from `-9999-11` to
-`+9999-11`.
+Year-Month Interval 由 `+years-months` 组成,其范围从 `-9999-11` 到 `+9999-11`。
-The value representation is the same for all types of resolutions. For example, an interval
-of months of 50 is always represented in an interval-of-years-to-months format (with default
-year precision): `+04-02`.
+所有类型的表达能力均相同。例如,Month 时间间隔下的 `50` 等价于 Year-Month 时间间隔(缺省年份精度)下的 `+04-02`。
-**Declaration**
+**声明**
-The type can be declared using the above combinations where `p` is the number of digits of years
-(*year precision*). `p` must have a value between `1` and `4` (both inclusive). If no year precision
-is specified, `p` is equal to `2`.
+可以使用以上组合来声明类型,其中 `p` 是年数(*年精度*)的位数。`p` 的值必须介于 `1` 和 `4` 之间(含边界值)。如果未指定年精度,`p` 则等于 `2`。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:-----------------------------------|
-|`java.time.Period` | X | X | Ignores the `days` part. *Default* |
-|`java.lang.Integer` | X | X | Describes the number of months. |
-|`int` | X | (X) | Describes the number of months. Output only if type is not nullable. |
+|`java.time.Period` | X | X | 忽略 `days` 部分。 *缺省* |
+|`java.lang.Integer` | X | X | 描述月的数量。 |
+|`int` | X | (X) | 描述月的数量。 仅当类型不可为空时才输出。 |
#### `INTERVAL DAY TO MONTH`
-Data type for a group of day-time interval types.
+一组 Day-Time Interval 数据类型。
-The type must be parameterized to one of the following resolutions with up to nanosecond precision:
-- interval of days,
-- interval of days to hours,
-- interval of days to minutes,
-- interval of days to seconds,
-- interval of hours,
-- interval of hours to minutes,
-- interval of hours to seconds,
-- interval of minutes,
-- interval of minutes to seconds,
-- or interval of seconds.
+此类型达到纳秒精度,必被参数化为以下情况中的一种:
+- Day 时间间隔、
+- Day-Hour 时间间隔、
+- Day-Minute 时间间隔、
+- Day-Second 时间间隔、
+- Hour 时间间隔、
+- Hour-Minute 时间间隔、
+- Hour-Second 时间间隔、
+- Minute 时间间隔、
+- Minute-Second 时间间隔、
+- Second 时间间隔。
-An interval of day-time consists of `+days hours:months:seconds.fractional` with values ranging from
-`-999999 23:59:59.999999999` to `+999999 23:59:59.999999999`. The value representation is the same
-for all types of resolutions. For example, an interval of seconds of 70 is always represented in
-an interval-of-days-to-seconds format (with default precisions): `+00 00:01:10.000000`.
+Day-Time 时间间隔由 `+days hours:months:seconds.fractional` 组成,其范围从 `-999999 23:59:59.999999999` 到 `+999999 23:59:59.999999999`。
-**Declaration**
+所有类型的表达能力均相同。例如,Second 时间间隔下的 `70` 等价于 Day-Second 时间间隔(缺省精度)下的 `+00 00:01:10.000000`。
+
+**声明**
-The type can be declared using the above combinations where `p1` is the number of digits of days
-(*day precision*) and `p2` is the number of digits of fractional seconds (*fractional precision*).
-`p1` must have a value between `1` and `6` (both inclusive). `p2` must have a value between `0`
-and `9` (both inclusive). If no `p1` is specified, it is equal to `2` by default. If no `p2` is
-specified, it is equal to `6` by default.
+可以使用以上组合来声明类型,其中 `p1` 是天数(*天精度*)的位数,`p2` 是秒的小数部分的位数(*小数精度*)。`p1` 的值必须介于 `1` 和之间 `6`(含边界值),`p2` 的值必须介于 `0` 和之间 `9`(含边界值)。如果 `p1` 未指定值,则缺省等于 `2`,如果 `p2` 未指定值,则缺省等于 `6`。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:--------------------|:-----:|:------:|:--------------------------------------|
-|`java.time.Duration` | X | X | *Default* |
-|`java.lang.Long` | X | X | Describes the number of milliseconds. |
-|`long` | X | (X) | Describes the number of milliseconds. Output only if type is not nullable. |
+|`java.time.Duration` | X | X | *缺省* |
+|`java.lang.Long` | X | X | 描述毫秒数。 |
+|`long` | X | (X) | 描述毫秒数。 仅当类型不可为空时才输出。 |
-### Constructured Data Types
+### 结构化的数据类型
#### `ARRAY`
-Data type of an array of elements with same subtype.
+具有相同子类型元素的数组的数据类型。
-Compared to the SQL standard, the maximum cardinality of an array cannot be specified but is
-fixed at `2,147,483,647`. Also, any valid type is supported as a subtype.
+与 SQL 标准相比,无法指定数组的最大长度,而是被固定为 `2,147,483,647`。另外,任何有效类型都可以作为子类型。
-**Declaration**
+**声明**
@@ -993,28 +916,25 @@ DataTypes.ARRAY(t)
-The type can be declared using `ARRAY` where `t` is the data type of the contained
-elements.
+此类型用 `ARRAY` 声明,其中 `t` 是所包含元素的数据类型。
-`t ARRAY` is a synonym for being closer to the SQL standard. For example, `INT ARRAY` is
-equivalent to `ARRAY`.
+`t ARRAY` 接近等价于 SQL 标准。例如,`INT ARRAY` 等价于 `ARRAY`。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:----------|:-----:|:------:|:----------------------------------|
-|*t*`[]` | (X) | (X) | Depends on the subtype. *Default* |
+|*t*`[]` | (X) | (X) | 依赖于子类型。 *缺省* |
#### `MAP`
-Data type of an associative array that maps keys (including `NULL`) to values (including `NULL`). A map
-cannot contain duplicate keys; each key can map to at most one value.
+将键(包括 `NULL`)映射到值(包括 `NULL`)的关联数组的数据类型。映射不能包含重复的键;每个键最多可以映射到一个值。
-There is no restriction of element types; it is the responsibility of the user to ensure uniqueness.
+元素类型没有限制;确保唯一性是用户的责任。
-The map type is an extension to the SQL standard.
+Map 类型是 SQL 标准的扩展。
-**Declaration**
+**声明**
@@ -1032,24 +952,22 @@ DataTypes.MAP(kt, vt)
-The type can be declared using `MAP` where `kt` is the data type of the key elements
-and `vt` is the data type of the value elements.
+此类型用 `MAP` 声明,其中 `kt` 是键的数据类型,`vt` 是值的数据类型。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:--------------------------------------|:-----:|:------:|:----------|
-| `java.util.Map` | X | X | *Default* |
-| *subclass* of `java.util.Map` | X | | |
+| `java.util.Map` | X | X | *缺省* |
+| `java.util.Map` 的*子类型* | X | | |
#### `MULTISET`
-Data type of a multiset (=bag). Unlike a set, it allows for multiple instances for each of its
-elements with a common subtype. Each unique value (including `NULL`) is mapped to some multiplicity.
+多重集合的数据类型(=bag)。与集合不同的是,它允许每个具有公共子类型的元素有多个实例。每个唯一值(包括 `NULL`)都映射到某种多重性。
-There is no restriction of element types; it is the responsibility of the user to ensure uniqueness.
+元素类型没有限制;确保唯一性是用户的责任。
-**Declaration**
+**声明**
@@ -1068,33 +986,28 @@ DataTypes.MULTISET(t)
-The type can be declared using `MULTISET` where `t` is the data type
-of the contained elements.
+此类型用 `MULTISET` 声明,其中 `t` 是所包含元素的数据类型。
-`t MULTISET` is a synonym for being closer to the SQL standard. For example, `INT MULTISET` is
-equivalent to `MULTISET`.
+`t MULTISET` 接近等价于 SQL 标准。例如,`INT MULTISET` 等价于 `MULTISET`。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------------------------|:-----:|:------:|:---------------------------------------------------------|
-|`java.util.Map` | X | X | Assigns each value to an integer multiplicity. *Default* |
-| *subclass* of `java.util.Map` | X | | Assigns each value to an integer multiplicity. |
+|`java.util.Map` | X | X | 将每个值可多重地分配给一个整数 *缺省* |
+|`java.util.Map` 的*子类型*| X | | 将每个值可多重地分配给一个整数 |
#### `ROW`
-Data type of a sequence of fields.
+字段序列的数据类型。
-A field consists of a field name, field type, and an optional description. The most specific type
-of a row of a table is a row type. In this case, each column of the row corresponds to the field
-of the row type that has the same ordinal position as the column.
+字段由字段名称、字段类型和可选的描述组成。表中的行的是最特殊的类型是 Row 类型。在这种情况下,行中的每一列对应于相同位置的列的 Row 类型的字段。
-Compared to the SQL standard, an optional field description simplifies the handling with complex
-structures.
+与 SQL 标准相比,可选的字段描述简化了复杂结构的处理。
-A row type is similar to the `STRUCT` type known from other non-standard-compliant frameworks.
+Row 类型类似于其他非标准兼容框架中的 `STRUCT` 类型。
-**Declaration**
+**声明**
-The type can be declared using `ROW` where `n` is the unique name of
-a field, `t` is the logical type of a field, `d` is the description of a field.
+此类型用 `ROW` 声明,其中 `n` 是唯一的字段名称,`t` 是字段的逻辑类型,`d` 是字段的描述。
-`ROW(...)` is a synonym for being closer to the SQL standard. For example, `ROW(myField INT, myOtherField BOOLEAN)` is
-equivalent to `ROW`.
+`ROW(...)` 接近等价于 SQL 标准。例如,`ROW(myField INT, myOtherField BOOLEAN)` 等价于 `ROW`。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:----------------------------|:-----:|:------:|:------------------------|
-|`org.apache.flink.types.Row` | X | X | *Default* |
+|`org.apache.flink.types.Row` | X | X | *缺省* |
-### Other Data Types
+### 其他数据类型
#### `BOOLEAN`
-Data type of a boolean with a (possibly) three-valued logic of `TRUE`, `FALSE`, and `UNKNOWN`.
+(可能)具有 `TRUE`、`FALSE` 和 `UNKNOWN` 三值逻辑的布尔数据类型。
-**Declaration**
+**声明**
@@ -1153,26 +1064,24 @@ DataTypes.BOOLEAN()
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:-------------------|:-----:|:------:|:-------------------------------------|
-|`java.lang.Boolean` | X | X | *Default* |
-|`boolean` | X | (X) | Output only if type is not nullable. |
+|`java.lang.Boolean` | X | X | *缺省* |
+|`boolean` | X | (X) | 仅当类型不可为空时才输出。 |
#### `NULL`
-Data type for representing untyped `NULL` values.
+表示空类型 `NULL` 值的数据类型。
-The null type is an extension to the SQL standard. A null type has no other value
-except `NULL`, thus, it can be cast to any nullable type similar to JVM semantics.
+NULL 类型是 SQL 标准的扩展。NULL 类型除 `NULL` 值以外没有其他值,因此可以将其强制转换为 JVM 里的任何可空类型。
-This type helps in representing unknown types in API calls that use a `NULL` literal
-as well as bridging to formats such as JSON or Avro that define such a type as well.
+此类型有助于使用 `NULL` 字面量表示 `API` 调用中的未知类型,以及桥接到定义该类型的 JSON 或 Avro 等格式。
-This type is not very useful in practice and is just mentioned here for completeness.
+这种类型在实践中不是很有用,为完整起见仅在此提及。
-**Declaration**
+**声明**
@@ -1190,21 +1099,20 @@ DataTypes.NULL()
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:------------------|:-----:|:------:|:-------------------------------------|
-|`java.lang.Object` | X | X | *Default* |
-|*any class* | | (X) | Any non-primitive type. |
+|`java.lang.Object` | X | X | *缺省* |
+|*任何类型* | | (X) | 任何非基本数据类型 |
#### `RAW`
-Data type of an arbitrary serialized type. This type is a black box within the table ecosystem
-and is only deserialized at the edges.
+任意序列化类型的数据类型。此类型是 Table 编程环境中的黑箱,仅在边缘反序列化。
-The raw type is an extension to the SQL standard.
+Raw 类型是 SQL 标准的扩展。
-**Declaration**
+**声明**
@@ -1224,18 +1132,15 @@ DataTypes.RAW(typeInfo)
-The type can be declared using `RAW('class', 'snapshot')` where `class` is the originating class and
-`snapshot` is the serialized `TypeSerializerSnapshot` in Base64 encoding. Usually, the type string is not
-declared directly but is generated while persisting the type.
+此类型用 `RAW('class', 'snapshot')` 声明,其中 `class` 是原始类,`snapshot` 是 Base64 编码的序列化的 `TypeSerializerSnapshot`。通常,类型字符串不是直接声明的,而是在保留类型时生成的。
-In the API, the `RAW` type can be declared either by directly supplying a `Class` + `TypeSerializer` or
-by passing `TypeInformation` and let the framework extract `Class` + `TypeSerializer` from there.
+在 API 中,可以通过直接提供 `Class` + `TypeSerializer` 或通过传递 `TypeInformation` 并让框架从那里提取 `Class` + `TypeSerializer` 来声明 `RAW` 类型。
-**Bridging to JVM Types**
+**JVM 类型**
-| Java Type | Input | Output | Remarks |
+| Java 类型 | 输入 | 输出 | 备注 |
|:------------------|:-----:|:------:|:-------------------------------------------|
-|*class* | X | X | Originating class or subclasses (for input) or superclasses (for output). *Default* |
+|*类型* | X | X | 原始类或子类(用于输入)或超类(用于输出)。 *缺省* |
|`byte[]` | | X | |
{% top %}
diff --git a/docs/dev/types_serialization.md b/docs/dev/types_serialization.md
index fdd7af7448c6b691934e5900e49c968914ba4057..24c725644545bc23500e0efb6de245880f6c5d66 100644
--- a/docs/dev/types_serialization.md
+++ b/docs/dev/types_serialization.md
@@ -30,6 +30,207 @@ generic type extraction, and type serialization framework. This document describ
* This will be replaced by the TOC
{:toc}
+## Supported Data Types
+
+Flink places some restrictions on the type of elements that can be in a DataSet or DataStream.
+The reason for this is that the system analyzes the types to determine
+efficient execution strategies.
+
+There are seven different categories of data types:
+
+1. **Java Tuples** and **Scala Case Classes**
+2. **Java POJOs**
+3. **Primitive Types**
+4. **Regular Classes**
+5. **Values**
+6. **Hadoop Writables**
+7. **Special Types**
+
+#### Tuples and Case Classes
+
+
+
+
+Tuples are composite types that contain a fixed number of fields with various types.
+The Java API provides classes from `Tuple1` up to `Tuple25`. Every field of a tuple
+can be an arbitrary Flink type including further tuples, resulting in nested tuples. Fields of a
+tuple can be accessed directly using the field's name as `tuple.f4`, or using the generic getter method
+`tuple.getField(int position)`. The field indices start at 0. Note that this stands in contrast
+to the Scala tuples, but it is more consistent with Java's general indexing.
+
+{% highlight java %}
+DataStream> wordCounts = env.fromElements(
+ new Tuple2("hello", 1),
+ new Tuple2("world", 2));
+
+wordCounts.map(new MapFunction, Integer>() {
+ @Override
+ public Integer map(Tuple2 value) throws Exception {
+ return value.f1;
+ }
+});
+
+wordCounts.keyBy(0); // also valid .keyBy("f0")
+
+
+{% endhighlight %}
+
+
+
+
+Scala case classes (and Scala tuples which are a special case of case classes), are composite types that contain a fixed number of fields with various types. Tuple fields are addressed by their 1-offset names such as `_1` for the first field. Case class fields are accessed by their name.
+
+{% highlight scala %}
+case class WordCount(word: String, count: Int)
+val input = env.fromElements(
+ WordCount("hello", 1),
+ WordCount("world", 2)) // Case Class Data Set
+
+input.keyBy("word")// key by field expression "word"
+
+val input2 = env.fromElements(("hello", 1), ("world", 2)) // Tuple2 Data Set
+
+input2.keyBy(0, 1) // key by field positions 0 and 1
+{% endhighlight %}
+
+
+
+
+#### POJOs
+
+Java and Scala classes are treated by Flink as a special POJO data type if they fulfill the following requirements:
+
+- The class must be public.
+
+- It must have a public constructor without arguments (default constructor).
+
+- All fields are either public or must be accessible through getter and setter functions. For a field called `foo` the getter and setter methods must be named `getFoo()` and `setFoo()`.
+
+- The type of a field must be supported by a registered serializer.
+
+POJOs are generally represented with a `PojoTypeInfo` and serialized with the `PojoSerializer` (using [Kryo](https://github.com/EsotericSoftware/kryo) as configurable fallback).
+The exception is when the POJOs are actually Avro types (Avro Specific Records) or produced as "Avro Reflect Types".
+In that case the POJO's are represented by an `AvroTypeInfo` and serialized with the `AvroSerializer`.
+You can also register your own custom serializer if required; see [Serialization](https://ci.apache.org/projects/flink/flink-docs-stable/dev/types_serialization.html#serialization-of-pojo-types) for further information.
+
+Flink analyzes the structure of POJO types, i.e., it learns about the fields of a POJO. As a result POJO types are easier to use than general types. Moreover, Flink can process POJOs more efficiently than general types.
+
+The following example shows a simple POJO with two public fields.
+
+
+
+{% highlight java %}
+public class WordWithCount {
+
+ public String word;
+ public int count;
+
+ public WordWithCount() {}
+
+ public WordWithCount(String word, int count) {
+ this.word = word;
+ this.count = count;
+ }
+}
+
+DataStream wordCounts = env.fromElements(
+ new WordWithCount("hello", 1),
+ new WordWithCount("world", 2));
+
+wordCounts.keyBy("word"); // key by field expression "word"
+
+{% endhighlight %}
+
+
+{% highlight scala %}
+class WordWithCount(var word: String, var count: Int) {
+ def this() {
+ this(null, -1)
+ }
+}
+
+val input = env.fromElements(
+ new WordWithCount("hello", 1),
+ new WordWithCount("world", 2)) // Case Class Data Set
+
+input.keyBy("word")// key by field expression "word"
+
+{% endhighlight %}
+
+
+
+#### Primitive Types
+
+Flink supports all Java and Scala primitive types such as `Integer`, `String`, and `Double`.
+
+#### General Class Types
+
+Flink supports most Java and Scala classes (API and custom).
+Restrictions apply to classes containing fields that cannot be serialized, like file pointers, I/O streams, or other native
+resources. Classes that follow the Java Beans conventions work well in general.
+
+All classes that are not identified as POJO types (see POJO requirements above) are handled by Flink as general class types.
+Flink treats these data types as black boxes and is not able to access their content (e.g., for efficient sorting). General types are de/serialized using the serialization framework [Kryo](https://github.com/EsotericSoftware/kryo).
+
+#### Values
+
+*Value* types describe their serialization and deserialization manually. Instead of going through a
+general purpose serialization framework, they provide custom code for those operations by means of
+implementing the `org.apache.flinktypes.Value` interface with the methods `read` and `write`. Using
+a Value type is reasonable when general purpose serialization would be highly inefficient. An
+example would be a data type that implements a sparse vector of elements as an array. Knowing that
+the array is mostly zero, one can use a special encoding for the non-zero elements, while the
+general purpose serialization would simply write all array elements.
+
+The `org.apache.flinktypes.CopyableValue` interface supports manual internal cloning logic in a
+similar way.
+
+Flink comes with pre-defined Value types that correspond to basic data types. (`ByteValue`,
+`ShortValue`, `IntValue`, `LongValue`, `FloatValue`, `DoubleValue`, `StringValue`, `CharValue`,
+`BooleanValue`). These Value types act as mutable variants of the basic data types: Their value can
+be altered, allowing programmers to reuse objects and take pressure off the garbage collector.
+
+
+#### Hadoop Writables
+
+You can use types that implement the `org.apache.hadoop.Writable` interface. The serialization logic
+defined in the `write()`and `readFields()` methods will be used for serialization.
+
+#### Special Types
+
+You can use special types, including Scala's `Either`, `Option`, and `Try`.
+The Java API has its own custom implementation of `Either`.
+Similarly to Scala's `Either`, it represents a value of two possible types, *Left* or *Right*.
+`Either` can be useful for error handling or operators that need to output two different types of records.
+
+#### Type Erasure & Type Inference
+
+*Note: This Section is only relevant for Java.*
+
+The Java compiler throws away much of the generic type information after compilation. This is
+known as *type erasure* in Java. It means that at runtime, an instance of an object does not know
+its generic type any more. For example, instances of `DataStream` and `DataStream` look the
+same to the JVM.
+
+Flink requires type information at the time when it prepares the program for execution (when the
+main method of the program is called). The Flink Java API tries to reconstruct the type information
+that was thrown away in various ways and store it explicitly in the data sets and operators. You can
+retrieve the type via `DataStream.getType()`. The method returns an instance of `TypeInformation`,
+which is Flink's internal way of representing types.
+
+The type inference has its limits and needs the "cooperation" of the programmer in some cases.
+Examples for that are methods that create data sets from collections, such as
+`ExecutionEnvironment.fromCollection(),` where you can pass an argument that describes the type. But
+also generic functions like `MapFunction` may need extra type information.
+
+The
+{% gh_link /flink-core/src/main/java/org/apache/flink/api/java/typeutils/ResultTypeQueryable.java "ResultTypeQueryable" %}
+interface can be implemented by input formats and functions to tell the API
+explicitly about their return type. The *input types* that the functions are invoked with can
+usually be inferred by the result types of the previous operations.
+
+{% top %}
+
## Type handling in Flink
diff --git a/docs/dev/user_defined_functions.md b/docs/dev/user_defined_functions.md
new file mode 100644
index 0000000000000000000000000000000000000000..bdfbe54cee3c9321ca2867356b46ec8f5313c178
--- /dev/null
+++ b/docs/dev/user_defined_functions.md
@@ -0,0 +1,241 @@
+---
+title: 'User-Defined Functions'
+nav-id: user_defined_function
+nav-parent_id: streaming
+nav-pos: 4
+---
+
+
+Most operations require a user-defined function. This section lists different
+ways of how they can be specified. We also cover `Accumulators`, which can be
+used to gain insights into your Flink application.
+
+
+
+
+## Implementing an interface
+
+The most basic way is to implement one of the provided interfaces:
+
+{% highlight java %}
+class MyMapFunction implements MapFunction {
+ public Integer map(String value) { return Integer.parseInt(value); }
+};
+data.map(new MyMapFunction());
+{% endhighlight %}
+
+## Anonymous classes
+
+You can pass a function as an anonymous class:
+{% highlight java %}
+data.map(new MapFunction () {
+ public Integer map(String value) { return Integer.parseInt(value); }
+});
+{% endhighlight %}
+
+## Java 8 Lambdas
+
+Flink also supports Java 8 Lambdas in the Java API.
+
+{% highlight java %}
+data.filter(s -> s.startsWith("http://"));
+{% endhighlight %}
+
+{% highlight java %}
+data.reduce((i1,i2) -> i1 + i2);
+{% endhighlight %}
+
+## Rich functions
+
+All transformations that require a user-defined function can
+instead take as argument a *rich* function. For example, instead of
+
+{% highlight java %}
+class MyMapFunction implements MapFunction {
+ public Integer map(String value) { return Integer.parseInt(value); }
+};
+{% endhighlight %}
+
+you can write
+
+{% highlight java %}
+class MyMapFunction extends RichMapFunction {
+ public Integer map(String value) { return Integer.parseInt(value); }
+};
+{% endhighlight %}
+
+and pass the function as usual to a `map` transformation:
+
+{% highlight java %}
+data.map(new MyMapFunction());
+{% endhighlight %}
+
+Rich functions can also be defined as an anonymous class:
+{% highlight java %}
+data.map (new RichMapFunction() {
+ public Integer map(String value) { return Integer.parseInt(value); }
+});
+{% endhighlight %}
+
+
+
+
+
+## Lambda Functions
+
+As already seen in previous examples all operations accept lambda functions for describing
+the operation:
+{% highlight scala %}
+val data: DataSet[String] = // [...]
+data.filter { _.startsWith("http://") }
+{% endhighlight %}
+
+{% highlight scala %}
+val data: DataSet[Int] = // [...]
+data.reduce { (i1,i2) => i1 + i2 }
+// or
+data.reduce { _ + _ }
+{% endhighlight %}
+
+## Rich functions
+
+All transformations that take as argument a lambda function can
+instead take as argument a *rich* function. For example, instead of
+
+{% highlight scala %}
+data.map { x => x.toInt }
+{% endhighlight %}
+
+you can write
+
+{% highlight scala %}
+class MyMapFunction extends RichMapFunction[String, Int] {
+ def map(in: String):Int = { in.toInt }
+};
+{% endhighlight %}
+
+and pass the function to a `map` transformation:
+
+{% highlight scala %}
+data.map(new MyMapFunction())
+{% endhighlight %}
+
+Rich functions can also be defined as an anonymous class:
+{% highlight scala %}
+data.map (new RichMapFunction[String, Int] {
+ def map(in: String):Int = { in.toInt }
+})
+{% endhighlight %}
+
+
+
+
+Rich functions provide, in addition to the user-defined function (map,
+reduce, etc), four methods: `open`, `close`, `getRuntimeContext`, and
+`setRuntimeContext`. These are useful for parameterizing the function
+(see [Passing Parameters to Functions]({{ site.baseurl }}/dev/batch/index.html#passing-parameters-to-functions)),
+creating and finalizing local state, accessing broadcast variables (see
+[Broadcast Variables]({{ site.baseurl }}/dev/batch/index.html#broadcast-variables)), and for accessing runtime
+information such as accumulators and counters (see
+[Accumulators and Counters](#accumulators--counters)), and information
+on iterations (see [Iterations]({{ site.baseurl }}/dev/batch/iterations.html)).
+
+{% top %}
+
+## Accumulators & Counters
+
+Accumulators are simple constructs with an **add operation** and a **final accumulated result**,
+which is available after the job ended.
+
+The most straightforward accumulator is a **counter**: You can increment it using the
+```Accumulator.add(V value)``` method. At the end of the job Flink will sum up (merge) all partial
+results and send the result to the client. Accumulators are useful during debugging or if you
+quickly want to find out more about your data.
+
+Flink currently has the following **built-in accumulators**. Each of them implements the
+{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java "Accumulator" %}
+interface.
+
+- {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/IntCounter.java "__IntCounter__" %},
+ {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongCounter.java "__LongCounter__" %}
+ and {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/DoubleCounter.java "__DoubleCounter__" %}:
+ See below for an example using a counter.
+- {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java "__Histogram__" %}:
+ A histogram implementation for a discrete number of bins. Internally it is just a map from Integer
+ to Integer. You can use this to compute distributions of values, e.g. the distribution of
+ words-per-line for a word count program.
+
+__How to use accumulators:__
+
+First you have to create an accumulator object (here a counter) in the user-defined transformation
+function where you want to use it.
+
+{% highlight java %}
+private IntCounter numLines = new IntCounter();
+{% endhighlight %}
+
+Second you have to register the accumulator object, typically in the ```open()``` method of the
+*rich* function. Here you also define the name.
+
+{% highlight java %}
+getRuntimeContext().addAccumulator("num-lines", this.numLines);
+{% endhighlight %}
+
+You can now use the accumulator anywhere in the operator function, including in the ```open()``` and
+```close()``` methods.
+
+{% highlight java %}
+this.numLines.add(1);
+{% endhighlight %}
+
+The overall result will be stored in the ```JobExecutionResult``` object which is
+returned from the `execute()` method of the execution environment
+(currently this only works if the execution waits for the
+completion of the job).
+
+{% highlight java %}
+myJobExecutionResult.getAccumulatorResult("num-lines")
+{% endhighlight %}
+
+All accumulators share a single namespace per job. Thus you can use the same accumulator in
+different operator functions of your job. Flink will internally merge all accumulators with the same
+name.
+
+A note on accumulators and iterations: Currently the result of accumulators is only available after
+the overall job has ended. We plan to also make the result of the previous iteration available in the
+next iteration. You can use
+{% gh_link /flink-java/src/main/java/org/apache/flink/api/java/operators/IterativeDataSet.java#L98 "Aggregators" %}
+to compute per-iteration statistics and base the termination of iterations on such statistics.
+
+__Custom accumulators:__
+
+To implement your own accumulator you simply have to write your implementation of the Accumulator
+interface. Feel free to create a pull request if you think your custom accumulator should be shipped
+with Flink.
+
+You have the choice to implement either
+{% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Accumulator.java "Accumulator" %}
+or {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/SimpleAccumulator.java "SimpleAccumulator" %}.
+
+```Accumulator``` is most flexible: It defines a type ```V``` for the value to add, and a
+result type ```R``` for the final result. E.g. for a histogram, ```V``` is a number and ```R``` is
+ a histogram. ```SimpleAccumulator``` is for the cases where both types are the same, e.g. for counters.
+
+{% top %}
diff --git a/docs/fig/bounded-unbounded.png b/docs/fig/bounded-unbounded.png
new file mode 100644
index 0000000000000000000000000000000000000000..29dfe8a884bc82caddadf5ee5c41c8d149d17c84
Binary files /dev/null and b/docs/fig/bounded-unbounded.png differ
diff --git a/docs/fig/connected-streams.svg b/docs/fig/connected-streams.svg
new file mode 100644
index 0000000000000000000000000000000000000000..ad973d6eca9207935577372550cb2c9367a00e00
--- /dev/null
+++ b/docs/fig/connected-streams.svg
@@ -0,0 +1,66 @@
+
+
+
+
+
+
diff --git a/docs/fig/distributed-runtime.svg b/docs/fig/distributed-runtime.svg
new file mode 100644
index 0000000000000000000000000000000000000000..7833846a50e3d0e5eb65c4f19c148a1d55b1cb82
--- /dev/null
+++ b/docs/fig/distributed-runtime.svg
@@ -0,0 +1,475 @@
+
+
+
+
+
+
diff --git a/docs/fig/flink-application-sources-sinks.png b/docs/fig/flink-application-sources-sinks.png
new file mode 100644
index 0000000000000000000000000000000000000000..70ce858a89ba4da9a6aa4c3ca33c8bce15e88cc7
Binary files /dev/null and b/docs/fig/flink-application-sources-sinks.png differ
diff --git a/docs/fig/keyBy.png b/docs/fig/keyBy.png
new file mode 100644
index 0000000000000000000000000000000000000000..4e094c4fd330ce35914cc7039510c88f6c11b996
Binary files /dev/null and b/docs/fig/keyBy.png differ
diff --git a/docs/fig/local-state.png b/docs/fig/local-state.png
new file mode 100644
index 0000000000000000000000000000000000000000..a8884845e26a8a31a97f5536c2ee95ea249e4a53
Binary files /dev/null and b/docs/fig/local-state.png differ
diff --git a/docs/fig/parallel-job.png b/docs/fig/parallel-job.png
new file mode 100644
index 0000000000000000000000000000000000000000..2fb2bc9895beae5fa7155859c54766c09ab835c3
Binary files /dev/null and b/docs/fig/parallel-job.png differ
diff --git a/docs/fig/program_dataflow.svg b/docs/fig/program_dataflow.svg
index 67ce15732ea7edccd0b6a50feb2dfb78befb366e..f2aa78359fa26abcae097c978a758cf97be297ed 100644
--- a/docs/fig/program_dataflow.svg
+++ b/docs/fig/program_dataflow.svg
@@ -25,10 +25,35 @@ under the License.
xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
xmlns:svg="http://www.w3.org/2000/svg"
xmlns="http://www.w3.org/2000/svg"
- version="1.1"
- width="632.86151"
+ xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+ xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+ inkscape:version="1.0beta1 (d565813, 2019-09-28)"
+ sodipodi:docname="program_dataflow.svg"
+ id="svg2"
height="495.70895"
- id="svg2">
+ width="632.86151"
+ version="1.1">
+ image/svg+xml
-
+
-
- Source
- DataStream
- <
- String
- >
- lines =
- env.
- addSource
- (
- new
- FlinkKafkaConsumer
- <>(…));
- DataStream
- <
- Event
- > events =
- lines.
- map
- ((line)
- -
- >
- parse
- (line));
- DataStream
- <
- Statistics
- > stats = events
- .
- keyBy
- (
- "id"
- )
- .
- timeWindow
- (
- Time.seconds
- (10))
- .
- apply
- (
- new
- MyWindowAggregationFunction
- ());
- stats.
- addSink
- (
- new
- BucketingSink
- (path));
-
- Source
-
- map()
-
-
-
-
-
- Transformation
- Transformation
- Source
- Operator
-
-
- keyBy
- ()/
- window()/
- apply()
-
-
- Sink
-
- Transformation
- Operators
- Sink
- Operator
-
-
-
-
-
- Stream
-
-
- Sink
-
- Streaming Dataflow
-
+ Source
+ DataStream
+ <
+ String
+ >
+ lines =
+ env.
+ addSource
+ (
+ new
+ FlinkKafkaConsumer
+ <>(…));
+ DataStream
+ <
+ Event
+ > events =
+ lines.
+ map
+ ((line)
+ -
+ >
+ parse
+ (line));
+ DataStream
+ <
+ Statistics
+ > stats = events
+ .
+ keyBy
+ (
+ event-> event.id
+ )
+ .
+ timeWindow
+ (
+ Time.seconds
+ (10))
+ .
+ apply
+ (
+ new
+ MyWindowAggregationFunction
+ ());
+ stats.
+ addSink
+ (
+ new
+ BucketingSink
+ (path));
+
+ Source
+
+ map()
+
+
+
+
+
+ Transformation
+ Transformation
+ Source
+ Operator
+
+
+ keyBy
+ ()/
+ window()/
+ apply()
+
+
+ Sink
+
+ Transformation
+ Operators
+ Sink
+ Operator
+
+
+
+
+
+ Stream
+
+
+ Sink
+
+ Streaming Dataflow
+
-
+ id="tspan3259" />
diff --git a/docs/fig/transformation.svg b/docs/fig/transformation.svg
new file mode 100644
index 0000000000000000000000000000000000000000..ff843f0493a2409a70f86c6326f0f1b9e4078290
--- /dev/null
+++ b/docs/fig/transformation.svg
@@ -0,0 +1,58 @@
+
+
+
+
+
+
diff --git a/docs/fig/window-assigners.svg b/docs/fig/window-assigners.svg
new file mode 100644
index 0000000000000000000000000000000000000000..a36ff89ed7095704879fbf932ec031fa2c30a976
--- /dev/null
+++ b/docs/fig/window-assigners.svg
@@ -0,0 +1,328 @@
+
+
+
+
+
+
diff --git a/docs/flinkDev/building.md b/docs/flinkDev/building.md
index c024e6386145582068bf03f700b9a7da47b85160..04b0f3128ee20f16612dcf05e1421ee12b143f96 100644
--- a/docs/flinkDev/building.md
+++ b/docs/flinkDev/building.md
@@ -65,7 +65,7 @@ Then go to the root directory of flink source code and run this command to build
cd flink-python; python setup.py sdist bdist_wheel
{% endhighlight %}
-Note Python 3.5 or higher is required to build PyFlink.
+Note Python version (3.5, 3.6 or 3.7) is required to build PyFlink.
The sdist and wheel package will be found under `./flink-python/dist/`. Either of them could be used for pip installation, such as:
diff --git a/docs/flinkDev/building.zh.md b/docs/flinkDev/building.zh.md
index 756d484c4d60848a2d719c42a1fb19de9f3ae21f..e18f539f386be37d8a1c342f66f67b9596ec7fd3 100644
--- a/docs/flinkDev/building.zh.md
+++ b/docs/flinkDev/building.zh.md
@@ -65,7 +65,7 @@ mvn clean install -DskipTests -Dfast
cd flink-python; python setup.py sdist bdist_wheel
{% endhighlight %}
-注意事项 构建PyFlink需要Python3.5及以上的版本.
+注意事项 构建PyFlink需要Python的版本为3.5, 3.6 或者 3.7.
构建好的源码发布包和wheel包位于`./flink-python/dist/`目录下。它们均可使用pip安装,比如:
diff --git a/docs/flinkDev/index.md b/docs/flinkDev/index.md
index 462feae47620b961cf903cf8543bd9aeaa963a2d..49308754ca14d1d8c5fbc0e629fb6a34d8c5cb4f 100644
--- a/docs/flinkDev/index.md
+++ b/docs/flinkDev/index.md
@@ -4,7 +4,7 @@ nav-title: ' Flink De
title: "Flink Development"
nav-id: "flinkdev"
nav-parent_id: root
-nav-pos: 8
+nav-pos: 20
---
+
+When deciding how and where to run Flink, there's a wide range of options available.
+
+* This will be replaced by the TOC
+{:toc}
+
+## Deployment Targets
+
+Apache Flink ships with first class support for a number of common deployment targets.
+
+
+
+
+
+ Local
+
+
+ Run Flink locally for basic testing and experimentation
+ Learn more
+
+
+
+
+
+
+ Standalone
+
+
+ A simple solution for running Flink on bare metal or VM's
+ Learn more
+
+ A generic resource manager for running distriubted systems
+ Learn more
+
+
+
+
+
+
+ Docker
+
+
+ A popular solution for running Flink within a containerized environment
+ Learn more
+
+
+
+
+
+
+ Kubernetes
+
+
+ An automated system for deploying containerized applications
+ Learn more
+
+
+
+
+
+## Vendor Solutions
+
+A number of vendors offer managed or fully hosted Flink solutions.
+None of these vendors are officially supported or endorsed by the Apache Flink PMC.
+Please refer to vendor maintained documentation on how to use these products.
+
+
+
+#### AliCloud Realtime Compute
+
+[Website](https://www.alibabacloud.com/products/realtime-compute)
+
+Supported Environments:
+AliCloud
+
+#### Amazon EMR
+
+[Website](https://aws.amazon.com/emr/)
+
+Supported Environments:
+AWS
+
+#### Amazon Kinesis Data Analytics For Java
+
+[Website](https://docs.aws.amazon.com/kinesisanalytics/latest/java/what-is.html)
+
+Supported Environments:
+AWS
+
+#### Cloudera
+
+[Website](https://www.cloudera.com/)
+
+Supported Environment:
+AWS
+Azure
+Google Cloud
+On-Premise
+
+#### Eventador
+
+[Website](https://eventador.io)
+
+Supported Environment:
+AWS
+
+#### Huawei Cloud Stream Service
+
+[Website](https://www.huaweicloud.com/en-us/product/cs.html)
+
+Supported Environment:
+Huawei Cloud
+
+#### Ververica Platform
+
+[Website](https://www.ververica.com/platform-overview)
+
+Supported Environments:
+AliCloud
+AWS
+Azure
+Google Cloud
+On-Premise
diff --git a/docs/ops/deployment/index.zh.md b/docs/ops/deployment/index.zh.md
index 196e3808cb0e4b041e519eb490109ffade302802..4499abb5c086c8147fd673662a7b08c31cadd24e 100644
--- a/docs/ops/deployment/index.zh.md
+++ b/docs/ops/deployment/index.zh.md
@@ -3,6 +3,7 @@ title: "集群与部署"
nav-id: deployment
nav-parent_id: ops
nav-pos: 1
+nav-show_overview: true
---
+
+When deciding how and where to run Flink, there's a wide range of options available.
+
+* This will be replaced by the TOC
+{:toc}
+
+## Deployment Targets
+
+Apache Flink ships with first class support for a number of common deployment targets.
+
+
+
+
+
+ Local
+
+
+ Run Flink locally for basic testing and experimentation
+ Learn more
+
+
+
+
+
+
+ Standalone
+
+
+ A simple solution for running Flink on bare metal or VM's
+ Learn more
+
+ A generic resource manager for running distriubted systems
+ Learn more
+
+
+
+
+
+
+ Docker
+
+
+ A popular solution for running Flink within a containerized environment
+ Learn more
+
+
+
+
+
+
+ Kubernetes
+
+
+ An automated system for deploying containerized applications
+ Learn more
+
+
+
+
+
+## Vendor Solutions
+
+A number of vendors offer managed or fully hosted Flink solutions.
+None of these vendors are officially supported or endorsed by the Apache Flink PMC.
+Please refer to vendor maintained documentation on how to use these products.
+
+
+
+#### AliCloud Realtime Compute
+
+[Website](https://www.alibabacloud.com/products/realtime-compute)
+
+Supported Environments:
+AliCloud
+
+#### Amazon EMR
+
+[Website](https://aws.amazon.com/emr/)
+
+Supported Environments:
+AWS
+
+#### Amazon Kinesis Data Analytics For Java
+
+[Website](https://docs.aws.amazon.com/kinesisanalytics/latest/java/what-is.html)
+
+Supported Environments:
+AWS
+
+#### Cloudera
+
+[Website](https://www.cloudera.com/)
+
+Supported Environment:
+AWS
+Azure
+Google Cloud
+On-Premise
+
+#### Eventador
+
+[Website](https://eventador.io)
+
+Supported Environment:
+AWS
+
+#### Huawei Cloud Stream Service
+
+[Website](https://www.huaweicloud.com/en-us/product/cs.html)
+
+Supported Environment:
+Huawei Cloud
+
+#### Ververica Platform
+
+[Website](https://www.ververica.com/platform-overview)
+
+Supported Environments:
+AliCloud
+AWS
+Azure
+Google Cloud
+On-Premise
diff --git a/docs/ops/deployment/kubernetes.md b/docs/ops/deployment/kubernetes.md
index f30f21a22843b498681dbd80494eb987b86d763d..adce518d1a64b74f2d7608b81d7a644e8668d7d5 100644
--- a/docs/ops/deployment/kubernetes.md
+++ b/docs/ops/deployment/kubernetes.md
@@ -36,13 +36,13 @@ Please follow [Kubernetes' setup guide](https://kubernetes.io/docs/setup/) in or
If you want to run Kubernetes locally, we recommend using [MiniKube](https://kubernetes.io/docs/setup/minikube/).
- Note: If using MiniKube please make sure to execute `minikube ssh 'sudo ip link set docker0 promisc on'` before deploying a Flink cluster.
- Otherwise Flink components are not able to self reference themselves through a Kubernetes service.
+ Note: If using MiniKube please make sure to execute `minikube ssh 'sudo ip link set docker0 promisc on'` before deploying a Flink cluster.
+ Otherwise Flink components are not able to self reference themselves through a Kubernetes service.
## Flink session cluster on Kubernetes
-A Flink session cluster is executed as a long-running Kubernetes Deployment.
+A Flink session cluster is executed as a long-running Kubernetes Deployment.
Note that you can run multiple Flink jobs on a session cluster.
Each job needs to be submitted to the cluster after the cluster has been deployed.
@@ -91,15 +91,28 @@ In order to terminate the Flink session cluster, use `kubectl`:
## Flink job cluster on Kubernetes
-A Flink job cluster is a dedicated cluster which runs a single job.
-The job is part of the image and, thus, there is no extra job submission needed.
+A Flink job cluster is a dedicated cluster which runs a single job.
+The job is part of the image and, thus, there is no extra job submission needed.
### Creating the job-specific image
The Flink job cluster image needs to contain the user code jars of the job for which the cluster is started.
Therefore, one needs to build a dedicated container image for every job.
Please follow these [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/docker/README.md) to build the Docker image.
-
+
+### Using plugins
+
+As described in the [plugins]({{ site.baseurl }}/ops/plugins.html) documentation page: in order to use plugins they must be
+copied to the correct location in the flink installation for them to work.
+
+The simplest way to enable plugins for use on Kubernetes is to modify the provided Flink docker image by adding
+an additional layer. This does however assume you have a docker registry available where you can push images to and
+that is accessible by your Kubernetes cluster.
+
+How this can be done is described on the [Docker Setup]({{ site.baseurl }}/zh/ops/deployment/docker.html#using-plugins) page.
+
+With such an image created you can now start your Kubernetes based Flink cluster which can use the enabled plugins.
+
### Deploy Flink job cluster on Kubernetes
In order to deploy the a job cluster on Kubernetes please follow these [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/kubernetes/README.md#deploy-flink-job-cluster).
@@ -130,7 +143,7 @@ data:
blob.server.port: 6124
jobmanager.rpc.port: 6123
taskmanager.rpc.port: 6122
- jobmanager.heap.size: 1024m
+ jobmanager.memory.process.size: 1472m
taskmanager.memory.process.size: 1024m
log4j.properties: |+
rootLogger.level = INFO
diff --git a/docs/ops/deployment/kubernetes.zh.md b/docs/ops/deployment/kubernetes.zh.md
index 014c9372a8d74b77c21dbf8bcc7e23747c1a9dba..d72407567a63f4c4ace0939352066d7ba6f71b0d 100644
--- a/docs/ops/deployment/kubernetes.zh.md
+++ b/docs/ops/deployment/kubernetes.zh.md
@@ -100,6 +100,19 @@ The Flink job cluster image needs to contain the user code jars of the job for w
Therefore, one needs to build a dedicated container image for every job.
Please follow these [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/docker/README.md) to build the Docker image.
+### Using plugins
+
+As described in the [plugins]({{ site.baseurl }}/zh/ops/plugins.html) documentation page: in order to use plugins they must be
+copied to the correct location in the flink installation for them to work.
+
+The simplest way to enable plugins for use on Kubernetes is to modify the provided Flink docker image by adding
+an additional layer. This does however assume you have a docker registry available where you can push images to and
+that is accessible by your Kubernetes cluster.
+
+How this can be done is described on the [Docker Setup]({{ site.baseurl }}/zh/ops/deployment/docker.html#using-plugins) page.
+
+With such an image created you can now start your Kubernetes based Flink cluster which can use the enabled plugins.
+
### Deploy Flink job cluster on Kubernetes
In order to deploy the a job cluster on Kubernetes please follow these [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/kubernetes/README.md#deploy-flink-job-cluster).
@@ -130,7 +143,7 @@ data:
blob.server.port: 6124
jobmanager.rpc.port: 6123
taskmanager.rpc.port: 6122
- jobmanager.heap.size: 1024m
+ jobmanager.memory.process.size: 1472m
taskmanager.memory.process.size: 1024m
log4j.properties: |+
rootLogger.level = INFO
diff --git a/docs/ops/deployment/local.md b/docs/ops/deployment/local.md
index 55ff4b30978d24e65630a62ae831cdf4c2b0748d..367cf8d665c27e320720ffa3444aad4b11491825 100644
--- a/docs/ops/deployment/local.md
+++ b/docs/ops/deployment/local.md
@@ -30,7 +30,8 @@ Get a local Flink cluster up and running in a few simple steps.
## Setup: Download and Start Flink
-Flink runs on __Linux, Mac OS X, and Windows__.
+Flink runs on __Linux and Mac OS X__.
+Note: Windows users can run Flink in Cygwin or WSL.
To be able to run Flink, the only requirement is to have a working __Java 8 or 11__ installation.
You can check the correct installation of Java by issuing the following command:
@@ -97,26 +98,9 @@ $ cd build-target
### Start a Local Flink Cluster
-
-Note: The ``bin`` folder of your Java Runtime Environment must be included in Window's ``%PATH%`` variable.
-Follow this [guide](http://www.java.com/en/download/help/path.xml) to add Java to the ``%PATH%`` variable.
-
-{% highlight bash %}
-$ cd flink
-$ cd bin
-$ start-cluster.bat
-{% endhighlight %}
-
-After that, you need to open a second terminal to run jobs using `flink.bat`.
-
-
-
Check the __Dispatcher's web frontend__ at [http://localhost:8081](http://localhost:8081) and make sure everything is up and running. The web frontend should report a single available TaskManager instance.
diff --git a/docs/ops/deployment/local.zh.md b/docs/ops/deployment/local.zh.md
index bcbbfdabefc4a3cc05e6a6bb5dc469b2b7b4c394..b63d16d5c6fb2d55bb81a5d46c72a3f00d34b11f 100644
--- a/docs/ops/deployment/local.zh.md
+++ b/docs/ops/deployment/local.zh.md
@@ -30,7 +30,8 @@ Get a local Flink cluster up and running in a few simple steps.
## Setup: Download and Start Flink
-Flink runs on __Linux, Mac OS X, and Windows__.
+Flink runs on __Linux and Mac OS X__.
+Note: Windows users can run Flink in Cygwin or WSL.
To be able to run Flink, the only requirement is to have a working __Java 8 or 11__ installation.
You can check the correct installation of Java by issuing the following command:
@@ -97,26 +98,9 @@ $ cd build-target
### Start a Local Flink Cluster
-
-Note: The ``bin`` folder of your Java Runtime Environment must be included in Window's ``%PATH%`` variable.
-Follow this [guide](http://www.java.com/en/download/help/path.xml) to add Java to the ``%PATH%`` variable.
-
-{% highlight bash %}
-$ cd flink
-$ cd bin
-$ start-cluster.bat
-{% endhighlight %}
-
-After that, you need to open a second terminal to run jobs using `flink.bat`.
-
-
-
Check the __Dispatcher's web frontend__ at [http://localhost:8081](http://localhost:8081) and make sure everything is up and running. The web frontend should report a single available TaskManager instance.
diff --git a/docs/ops/deployment/mesos.md b/docs/ops/deployment/mesos.md
index 1df18384fc0358670228a3f60c14570eec863dbc..a273171660fda0bbcf39fbbb83feabc88b61e057 100644
--- a/docs/ops/deployment/mesos.md
+++ b/docs/ops/deployment/mesos.md
@@ -215,7 +215,7 @@ For example:
bin/mesos-appmaster.sh \
-Dmesos.master=master.foobar.org:5050 \
- -Djobmanager.heap.size=1024m \
+ -Djobmanager.memory.process.size=1472m \
-Djobmanager.rpc.port=6123 \
-Drest.port=8081 \
-Dtaskmanager.memory.process.size=3500m \
@@ -236,7 +236,7 @@ Here is an example configuration for Marathon:
{
"id": "flink",
- "cmd": "$FLINK_HOME/bin/mesos-appmaster.sh -Djobmanager.heap.size=1024m -Djobmanager.rpc.port=6123 -Drest.port=8081 -Dtaskmanager.memory.process.size=1024m -Dtaskmanager.numberOfTaskSlots=2 -Dparallelism.default=2 -Dmesos.resourcemanager.tasks.cpus=1",
+ "cmd": "$FLINK_HOME/bin/mesos-appmaster.sh -Djobmanager.memory.process.size=1472m -Djobmanager.rpc.port=6123 -Drest.port=8081 -Dtaskmanager.memory.process.size=1024m -Dtaskmanager.numberOfTaskSlots=2 -Dparallelism.default=2 -Dmesos.resourcemanager.tasks.cpus=1",
"cpus": 1.0,
"mem": 1024
}
diff --git a/docs/ops/deployment/mesos.zh.md b/docs/ops/deployment/mesos.zh.md
index 83fcd9a7bcada955b7081bf1db6bcf990cada776..2d082357634df29294f0a8ded966b03dbb5faec9 100644
--- a/docs/ops/deployment/mesos.zh.md
+++ b/docs/ops/deployment/mesos.zh.md
@@ -215,7 +215,7 @@ For example:
bin/mesos-appmaster.sh \
-Dmesos.master=master.foobar.org:5050 \
- -Djobmanager.heap.size=1024m \
+ -Djobmanager.memory.process.size=1472m \
-Djobmanager.rpc.port=6123 \
-Drest.port=8081 \
-Dtaskmanager.memory.process.size=3500m \
@@ -236,7 +236,7 @@ Here is an example configuration for Marathon:
{
"id": "flink",
- "cmd": "$FLINK_HOME/bin/mesos-appmaster.sh -Djobmanager.heap.size=1024m -Djobmanager.rpc.port=6123 -Drest.port=8081 -Dtaskmanager.memory.process.size=1024m -Dtaskmanager.numberOfTaskSlots=2 -Dparallelism.default=2 -Dmesos.resourcemanager.tasks.cpus=1",
+ "cmd": "$FLINK_HOME/bin/mesos-appmaster.sh -Djobmanager.memory.process.size=1472m -Djobmanager.rpc.port=6123 -Drest.port=8081 -Dtaskmanager.memory.process.size=1024m -Dtaskmanager.numberOfTaskSlots=2 -Dparallelism.default=2 -Dmesos.resourcemanager.tasks.cpus=1",
"cpus": 1.0,
"mem": 1024
}
diff --git a/docs/ops/deployment/native_kubernetes.md b/docs/ops/deployment/native_kubernetes.md
index 908b8c62c17ec1284aebb447d43acd27b0396b70..b3806113ec42cb8166ba8d03d0a4d40586330dd3 100644
--- a/docs/ops/deployment/native_kubernetes.md
+++ b/docs/ops/deployment/native_kubernetes.md
@@ -126,12 +126,12 @@ $ echo 'stop' | ./bin/kubernetes-session.sh -Dkubernetes.cluster-id=
#### Manual Resource Cleanup
-Flink uses [Kubernetes ownerReference's](https://kubernetes.io/docs/concepts/workloads/controllers/garbage-collection/) to cleanup all cluster components.
-All the Flink created resources, including `ConfigMap`, `Service`, `Deployment`, `Pod`, have been set the ownerReference to `service/`.
-When the service is deleted, all other resource will be deleted automatically.
+Flink uses [Kubernetes OwnerReference's](https://kubernetes.io/docs/concepts/workloads/controllers/garbage-collection/) to cleanup all cluster components.
+All the Flink created resources, including `ConfigMap`, `Service`, `Pod`, have been set the OwnerReference to `deployment/`.
+When the deployment is deleted, all other resources will be deleted automatically.
{% highlight bash %}
-$ kubectl delete service/
+$ kubectl delete deployment/
{% endhighlight %}
## Log Files
@@ -154,6 +154,32 @@ appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m
If the pod is running, you can use `kubectl exec -it bash` to tunnel in and view the logs or debug the process.
+## Using plugins
+
+As described in the [plugins]({{ site.baseurl }}/ops/plugins.html) documentation page: in order to use plugins they must be
+copied to the correct location in the flink installation for them to work.
+
+The simplest way to enable plugins for use on Kubernetes is to modify the provided official Flink docker images by adding
+an additional layer. This does however assume you have a docker registry available where you can push images to and
+that is accessible by your Kubernetes cluster.
+
+How this can be done is described on the [Docker Setup]({{ site.baseurl }}/ops/deployment/docker.html#using-plugins) page.
+
+With such an image created you can now start your Kubernetes based Flink session cluster with the additional parameter
+`kubernetes.container.image` which must specify the image that was created: `docker.example.nl/flink:{{ site.version }}-2.12-s3`
+
+Extending the above example command to start the session cluster makes it this:
+
+{% highlight bash %}
+./bin/kubernetes-session.sh \
+ -Dkubernetes.cluster-id= \
+ -Dtaskmanager.memory.process.size=4096m \
+ -Dkubernetes.taskmanager.cpu=2 \
+ -Dtaskmanager.numberOfTaskSlots=4 \
+ -Dresourcemanager.taskmanager-timeout=3600000 \
+ -Dkubernetes.container.image=docker.example.nl/flink:{{ site.version }}-2.12-s3
+{% endhighlight %}
+
## Kubernetes concepts
### Namespaces
diff --git a/docs/ops/deployment/native_kubernetes.zh.md b/docs/ops/deployment/native_kubernetes.zh.md
index 10d47e02a7b84abfb84367013c789521649938cf..b9c328d84d391677a7fac932506aad2f2a319a04 100644
--- a/docs/ops/deployment/native_kubernetes.zh.md
+++ b/docs/ops/deployment/native_kubernetes.zh.md
@@ -126,12 +126,12 @@ $ echo 'stop' | ./bin/kubernetes-session.sh -Dkubernetes.cluster-id=
#### Manual Resource Cleanup
-Flink uses [Kubernetes ownerReference's](https://kubernetes.io/docs/concepts/workloads/controllers/garbage-collection/) to cleanup all cluster components.
-All the Flink created resources, including `ConfigMap`, `Service`, `Deployment`, `Pod`, have been set the ownerReference to `service/`.
-When the service is deleted, all other resource will be deleted automatically.
+Flink uses [Kubernetes OwnerReference's](https://kubernetes.io/docs/concepts/workloads/controllers/garbage-collection/) to cleanup all cluster components.
+All the Flink created resources, including `ConfigMap`, `Service`, `Pod`, have been set the OwnerReference to `deployment/`.
+When the deployment is deleted, all other resources will be deleted automatically.
{% highlight bash %}
-$ kubectl delete service/
+$ kubectl delete deployment/
{% endhighlight %}
## Log Files
@@ -154,6 +154,32 @@ appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m
If the pod is running, you can use `kubectl exec -it bash` to tunnel in and view the logs or debug the process.
+## Using plugins
+
+As described in the [plugins]({{ site.baseurl }}/zh/ops/plugins.html) documentation page: in order to use plugins they must be
+copied to the correct location in the flink installation for them to work.
+
+The simplest way to enable plugins for use on Kubernetes is to modify the provided official Flink docker images by adding
+an additional layer. This does however assume you have a docker registry available where you can push images to and
+that is accessible by your Kubernetes cluster.
+
+How this can be done is described on the [Docker Setup]({{ site.baseurl }}/zh/ops/deployment/docker.html#using-plugins) page.
+
+With such an image created you can now start your Kubernetes based Flink session cluster with the additional parameter
+`kubernetes.container.image` which must specify the image that was created: `docker.example.nl/flink:{{ site.version }}-2.12-s3`
+
+Extending the above example command to start the session cluster makes it this:
+
+{% highlight bash %}
+./bin/kubernetes-session.sh \
+ -Dkubernetes.cluster-id= \
+ -Dtaskmanager.memory.process.size=4096m \
+ -Dkubernetes.taskmanager.cpu=2 \
+ -Dtaskmanager.numberOfTaskSlots=4 \
+ -Dresourcemanager.taskmanager-timeout=3600000 \
+ -Dkubernetes.container.image=docker.example.nl/flink:{{ site.version }}-2.12-s3
+{% endhighlight %}
+
## Kubernetes concepts
### Namespaces
diff --git a/docs/ops/filesystems/azure.md b/docs/ops/filesystems/azure.md
index d63c9d63a6871a41df8bfb6aff89311c0d228b93..e24b1125b31ce3dc8a140f8f6241402ea474f59d 100644
--- a/docs/ops/filesystems/azure.md
+++ b/docs/ops/filesystems/azure.md
@@ -53,7 +53,7 @@ env.setStateBackend(new FsStateBackend("wasb://@$.blob.core.windows.net:
{% endhighlight %}
-Alternatively, the the filesystem can be configured to read the Azure Blob Storage key from an
+Alternatively, the filesystem can be configured to read the Azure Blob Storage key from an
environment variable `AZURE_STORAGE_KEY` by setting the following configuration keys in
`flink-conf.yaml`.
diff --git a/docs/ops/filesystems/azure.zh.md b/docs/ops/filesystems/azure.zh.md
index 4d81b5596b682b52bf1ae9dbdb79b7eb6d703a62..0257bd8769bcfa9ff1a7f9ea0b1ed2f4c72f3257 100644
--- a/docs/ops/filesystems/azure.zh.md
+++ b/docs/ops/filesystems/azure.zh.md
@@ -1,6 +1,6 @@
---
-title: "Azure Blob Storage"
-nav-title: Azure Blob Storage
+title: "Azure Blob 存储"
+nav-title: Azure Blob 存储
nav-parent_id: filesystems
nav-pos: 3
---
@@ -23,60 +23,54 @@ specific language governing permissions and limitations
under the License.
-->
-[Azure Blob Storage](https://docs.microsoft.com/en-us/azure/storage/) is a Microsoft-managed service providing cloud storage for a variety of use cases.
-You can use Azure Blob Storage with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl }}/ops/state/state_backends.html)
+[Azure Blob 存储](https://docs.microsoft.com/en-us/azure/storage/) 是一项由 Microsoft 管理的服务,能提供多种应用场景下的云存储。
+Azure Blob 存储可与 Flink 一起使用以**读取**和**写入数据**,以及与[流 State Backend]({{ site.baseurl }}/zh/ops/state/state_backends.html) 结合使用。
* This will be replaced by the TOC
{:toc}
-You can use Azure Blob Storage objects like regular files by specifying paths in the following format:
+通过以下格式指定路径,Azure Blob 存储对象可类似于普通文件使用:
{% highlight plain %}
wasb://@$.blob.core.windows.net/
-// SSL encrypted access
+// SSL 加密访问
wasbs://@$.blob.core.windows.net/
{% endhighlight %}
-See below for how to use Azure Blob Storage in a Flink job:
+参见以下代码了解如何在 Flink 作业中使用 Azure Blob 存储:
{% highlight java %}
-// Read from Azure Blob storage
+// 读取 Azure Blob 存储
env.readTextFile("wasb://@$.blob.core.windows.net/");
-// Write to Azure Blob storage
+// 写入 Azure Blob 存储
stream.writeAsText("wasb://@$.blob.core.windows.net/")
-// Use Azure Blob Storage as FsStatebackend
+// 将 Azure Blob 存储用作 FsStatebackend
env.setStateBackend(new FsStateBackend("wasb://@$.blob.core.windows.net/"));
{% endhighlight %}
-### Shaded Hadoop Azure Blob Storage file system
+### Shaded Hadoop Azure Blob 存储文件系统
-To use `flink-azure-fs-hadoop,` copy the respective JAR file from the `opt` directory to the `plugins` directory of your Flink distribution before starting Flink, e.g.
+为使用 flink-azure-fs-hadoop,在启动 Flink 之前,将对应的 JAR 文件从 opt 目录复制到 Flink 发行版中的 plugin 目录下的一个文件夹中,例如:
{% highlight bash %}
mkdir ./plugins/azure-fs-hadoop
cp ./opt/flink-azure-fs-hadoop-{{ site.version }}.jar ./plugins/azure-fs-hadoop/
{% endhighlight %}
-`flink-azure-fs-hadoop` registers default FileSystem wrappers for URIs with the *wasb://* and *wasbs://* (SSL encrypted access) scheme.
+`flink-azure-fs-hadoop` 为使用 *wasb://* 和 *wasbs://* (SSL 加密访问) 的 URI 注册了默认的文件系统包装器。
-### Credentials Configuration
-
-Hadoop's Azure Filesystem supports configuration of credentials via the Hadoop configuration as
-outlined in the [Hadoop Azure Blob Storage documentation](https://hadoop.apache.org/docs/current/hadoop-azure/index.html#Configuring_Credentials).
-For convenience Flink forwards all Flink configurations with a key prefix of `fs.azure` to the
-Hadoop configuration of the filesystem. Consequentially, the azure blob storage key can be configured
-in `flink-conf.yaml` via:
+### 凭据配置
+Hadoop 的 Azure 文件系统支持通过 Hadoop 配置来配置凭据,如 [Hadoop Azure Blob Storage 文档](https://hadoop.apache.org/docs/current/hadoop-azure/index.html#Configuring_Credentials) 所述。
+为方便起见,Flink 将所有的 Flink 配置添加 `fs.azure` 键前缀后转发至文件系统的 Hadoop 配置中。因此,可通过以下方法在 `flink-conf.yaml` 中配置 Azure Blob 存储密钥:
{% highlight yaml %}
fs.azure.account.key..blob.core.windows.net:
{% endhighlight %}
-Alternatively, the the filesystem can be configured to read the Azure Blob Storage key from an
-environment variable `AZURE_STORAGE_KEY` by setting the following configuration keys in
-`flink-conf.yaml`.
+或者通过在 `flink-conf.yaml` 中设置以下配置键,将文件系统配置为从环境变量 `AZURE_STORAGE_KEY` 读取 Azure Blob 存储密钥:
{% highlight yaml %}
fs.azure.account.keyprovider..blob.core.windows.net: org.apache.flink.fs.azurefs.EnvironmentVariableKeyProvider
diff --git a/docs/ops/filesystems/common.md b/docs/ops/filesystems/common.md
index a8a371ed6a8f07739a27e0c8839fbf1644050f95..468e62502ea5d6c38528a1f424646cd232b605cb 100644
--- a/docs/ops/filesystems/common.md
+++ b/docs/ops/filesystems/common.md
@@ -62,7 +62,7 @@ If the opening of the stream takes longer than `fs..limit.timeout`, the
To prevent inactive streams from taking up the full pool (preventing new connections to be opened), you can add an inactivity timeout which forcibly closes them if they do not read/write any bytes for at least that amount of time: `fs..limit.stream-timeout`.
-Limit enforcment on a per TaskManager/file system basis.
+Limit enforcement on a per TaskManager/file system basis.
Because file systems creation occurs per scheme and authority, different
authorities have independent connection pools. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools.
diff --git a/docs/ops/filesystems/common.zh.md b/docs/ops/filesystems/common.zh.md
index a8a371ed6a8f07739a27e0c8839fbf1644050f95..2c5b1675b542de480800a6b37ed088031509264b 100644
--- a/docs/ops/filesystems/common.zh.md
+++ b/docs/ops/filesystems/common.zh.md
@@ -1,5 +1,5 @@
---
-title: "Common Configurations"
+title: "通用配置"
nav-parent_id: filesystems
nav-pos: 0
---
@@ -22,48 +22,41 @@ specific language governing permissions and limitations
under the License.
-->
-Apache Flink provides several standard configuration settings that work across all file system implementations.
+Apache Flink 提供了一些对所有文件系统均适用的基本配置。
* This will be replaced by the TOC
{:toc}
-## Default File System
+## 默认文件系统
-A default scheme (and authority) is used if paths to files do not explicitly specify a file system scheme (and authority).
+如果文件路径未明确指定文件系统的 scheme(和 authority),将会使用默认的 scheme(和 authority):
{% highlight yaml %}
fs.default-scheme:
{% endhighlight %}
-For example, if the default file system configured as `fs.default-scheme: hdfs://localhost:9000/`, then a file path of
-`/user/hugo/in.txt` is interpreted as `hdfs://localhost:9000/user/hugo/in.txt`.
+例如默认的文件系统配置为 `fs.default-scheme: hdfs://localhost:9000/`,则文件路径 `/user/hugo/in.txt` 将被处理为 `hdfs://localhost:9000/user/hugo/in.txt`。
-## Connection limiting
+## 连接限制
-You can limit the total number of connections that a file system can concurrently open which is useful when the file system cannot handle a large number
-of concurrent reads/writes or open connections at the same time.
+如果文件系统不能处理大量并发读/写操作或连接,可以为文件系统同时打开的总连接数设置上限。
-For example, small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint.
+例如在一个大型 Flink 任务建立 checkpoint 时,具有少量 RPC handler 的小型 HDFS 集群可能会由于建立了过多的连接而过载。
-To limit a specific file system's connections, add the following entries to the Flink configuration. The file system to be limited is identified by
-its scheme.
+要限制文件系统的连接数,可将下列配置添加至 Flink 配置中。设置限制的文件系统由其 scheme 指定:
{% highlight yaml %}
-fs..limit.total: (number, 0/-1 mean no limit)
-fs..limit.input: (number, 0/-1 mean no limit)
-fs..limit.output: (number, 0/-1 mean no limit)
-fs..limit.timeout: (milliseconds, 0 means infinite)
-fs..limit.stream-timeout: (milliseconds, 0 means infinite)
+fs..limit.total: (数量,0/-1 表示无限制)
+fs..limit.input: (数量,0/-1 表示无限制)
+fs..limit.output: (数量,0/-1 表示无限制)
+fs..limit.timeout: (毫秒,0 表示无穷)
+fs..limit.stream-timeout: (毫秒,0 表示无穷)
{% endhighlight %}
-You can limit the number of input/output connections (streams) separately (`fs..limit.input` and `fs..limit.output`), as well as impose a limit on
-the total number of concurrent streams (`fs..limit.total`). If the file system tries to open more streams, the operation blocks until some streams close.
-If the opening of the stream takes longer than `fs..limit.timeout`, the stream opening fails.
+输入和输出连接(流)的数量可以分别进行限制(`fs..limit.input` 和 `fs..limit.output`),也可以限制并发流的总数(`fs..limit.total`)。如果文件系统尝试打开更多的流,操作将被阻塞直至某些流关闭。如果打开流的时间超过 `fs..limit.timeout`,则流打开失败。
-To prevent inactive streams from taking up the full pool (preventing new connections to be opened), you can add an inactivity timeout which forcibly closes them if they do not read/write any bytes for at least that amount of time: `fs..limit.stream-timeout`.
+为避免不活动的流占满整个连接池(阻止新连接的建立),可以在配置中添加无活动超时时间,如果连接至少在 `fs..limit.stream-timeout` 时间内没有读/写操作,则连接会被强制关闭。
-Limit enforcment on a per TaskManager/file system basis.
-Because file systems creation occurs per scheme and authority, different
-authorities have independent connection pools. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools.
+连接数是按每个 TaskManager/文件系统来进行限制的。因为文件系统的创建是按照 scheme 和 authority 进行的,所以不同的 authority 具有独立的连接池,例如 `hdfs://myhdfs:50010/` 和 `hdfs://anotherhdfs:4399/` 会有单独的连接池。
{% top %}
\ No newline at end of file
diff --git a/docs/ops/filesystems/index.zh.md b/docs/ops/filesystems/index.zh.md
index a27ec72dec2cb2f7cc7860d24b8036cc654a78e2..5494f850ad3d775dffb8033cbb35d6e637f54e47 100644
--- a/docs/ops/filesystems/index.zh.md
+++ b/docs/ops/filesystems/index.zh.md
@@ -1,5 +1,5 @@
---
-title: "File Systems"
+title: "文件系统"
nav-id: filesystems
nav-parent_id: ops
nav-show_overview: true
@@ -24,104 +24,82 @@ specific language governing permissions and limitations
under the License.
-->
-Apache Flink uses file systems to consume and persistently store data, both for the results of applications and for fault tolerance and recovery.
-These are some of most of the popular file systems, including *local*, *hadoop-compatible*, *Amazon S3*, *MapR FS*, *OpenStack Swift FS*, *Aliyun OSS* and *Azure Blob Storage*.
+Apache Flink 使用文件系统来消费和持久化地存储数据,以处理应用结果以及容错与恢复。以下是一些最常用的文件系统:*本地存储*,*hadoop-compatible*,*Amazon S3*,*MapR FS*,*OpenStack Swift FS*,*阿里云 OSS* 和 *Azure Blob Storage*。
-The file system used for a particular file is determined by its URI scheme.
-For example, `file:///home/user/text.txt` refers to a file in the local file system, while `hdfs://namenode:50010/data/user/text.txt` is a file in a specific HDFS cluster.
+文件使用的文件系统通过其 URI Scheme 指定。例如 `file:///home/user/text.txt` 表示一个在本地文件系统中的文件,`hdfs://namenode:50010/data/user/text.txt` 表示一个在指定 HDFS 集群中的文件。
-File system instances are instantiated once per process and then cached/pooled, to avoid configuration overhead per stream creation and to enforce certain constraints, such as connection/stream limits.
+文件系统在每个进程实例化一次,然后进行缓存/池化,从而避免每次创建流时的配置开销,并强制执行特定的约束,如连接/流的限制。
* This will be replaced by the TOC
{:toc}
-## Local File System
+## 本地文件系统
-Flink has built-in support for the file system of the local machine, including any NFS or SAN drives mounted into that local file system.
-It can be used by default without additional configuration. Local files are referenced with the *file://* URI scheme.
+Flink 原生支持本地机器上的文件系统,包括任何挂载到本地文件系统的 NFS 或 SAN 驱动器,默认即可使用,无需额外配置。本地文件可通过 *file://* URI Scheme 引用。
-## Pluggable File Systems
+## 外部文件系统
-The Apache Flink project supports the following file systems:
+Apache Flink 支持下列文件系统:
+ - [**Amazon S3**](./s3.html) 对象存储由 `flink-s3-fs-presto` 和 `flink-s3-fs-hadoop` 两种替代实现提供支持。这两种实现都是独立的,没有依赖项。
- - [**Amazon S3**](./s3.html) object storage is supported by two alternative implementations: `flink-s3-fs-presto` and `flink-s3-fs-hadoop`.
- Both implementations are self-contained with no dependency footprint.
+ - **MapR FS** 文件系统适配器已在 Flink 的主发行版中通过 *maprfs://* URI Scheme 支持。MapR 库需要在 classpath 中指定(例如在 `lib` 目录中)。
- - **MapR FS** file system adapter is already supported in the main Flink distribution under the *maprfs://* URI scheme.
- You must provide the MapR libraries in the classpath (for example in `lib` directory).
+ - **OpenStack Swift FS** 由 `flink-swift-fs-hadoop` 支持,并通过 *swift://* URI scheme 使用。该实现基于 [Hadoop Project](https://hadoop.apache.org/),但其是独立的,没有依赖项。
+ 将 Flink 作为库使用时,使用该文件系统需要添加相应的 Maven 依赖项(`org.apache.flink:flink-swift-fs-hadoop:{{ site.version }}`)。
- - **OpenStack Swift FS** is supported by `flink-swift-fs-hadoop` and registered under the *swift://* URI scheme.
- The implementation is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint.
- To use it when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-swift-fs-hadoop:{{ site.version }}`).
-
- - **[Aliyun Object Storage Service](./oss.html)** is supported by `flink-oss-fs-hadoop` and registered under the *oss://* URI scheme.
- The implementation is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint.
+ - **[阿里云对象存储](./oss.html)**由 `flink-oss-fs-hadoop` 支持,并通过 *oss://* URI scheme 使用。该实现基于 [Hadoop Project](https://hadoop.apache.org/),但其是独立的,没有依赖项。
- - **[Azure Blob Storage](./azure.html)** is supported by `flink-azure-fs-hadoop` and registered under the *wasb(s)://* URI schemes.
- The implementation is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint.
+ - **[Azure Blob Storage](./azure.html)** 由`flink-azure-fs-hadoop` 支持,并通过 *wasb(s)://* URI scheme 使用。该实现基于 [Hadoop Project](https://hadoop.apache.org/),但其是独立的,没有依赖项。
-Except **MapR FS**, you can and should use any of them as [plugins](../plugins.html).
+除 **MapR FS** 之外,上述文件系统可以并且需要作为[插件](../plugins.html)使用。
-To use a pluggable file systems, copy the corresponding JAR file from the `opt` directory to a directory under `plugins` directory
-of your Flink distribution before starting Flink, e.g.
+使用外部文件系统时,在启动 Flink 之前需将对应的 JAR 文件从 `opt` 目录复制到 Flink 发行版 `plugin` 目录下的某一文件夹中,例如:
{% highlight bash %}
mkdir ./plugins/s3-fs-hadoop
cp ./opt/flink-s3-fs-hadoop-{{ site.version }}.jar ./plugins/s3-fs-hadoop/
{% endhighlight %}
-Attention The [plugin](../plugins.html) mechanism for file systems was introduced in Flink version `1.9` to
-support dedicated Java class loaders per plugin and to move away from the class shading mechanism.
-You can still use the provided file systems (or your own implementations) via the old mechanism by copying the corresponding
-JAR file into `lib` directory. However, **since 1.10, s3 plugins must be loaded through the plugin mechanism**; the old
-way no longer works as these plugins are not shaded anymore (or more specifically the classes are not relocated since 1.10).
+注意 文件系统的[插件](../plugins.html)机制在 Flink 版本 1.9 中引入,以支持每个插件专有 Java 类加载器,并避免类隐藏机制。您仍然可以通过旧机制使用文件系统,即将对应的 JAR 文件复制到 `lib` 目录中,或使用您自己的实现方式,但是从版本 1.10 开始,**S3 插件必须通过插件机制加载**,因为这些插件不再被隐藏(版本 1.10 之后类不再被重定位),旧机制不再可用。
-It's encouraged to use the [plugins](../plugins.html)-based loading mechanism for file systems that support it. Loading file systems components from the `lib`
-directory will not supported in future Flink versions.
+尽可能通过基于[插件](../plugins.html)的加载机制使用支持的文件系统。未来的 Flink 版本将不再支持通过 `lib` 目录加载文件系统组件。
-## Adding a new pluggable File System implementation
+## 添加新的外部文件系统实现
-File systems are represented via the `org.apache.flink.core.fs.FileSystem` class, which captures the ways to access and modify files and objects in that file system.
+文件系统由类 `org.apache.flink.core.fs.FileSystem` 表示,该类定义了访问与修改文件系统中文件与对象的方法。
-To add a new file system:
+要添加一个新的文件系统:
- - Add the File System implementation, which is a subclass of `org.apache.flink.core.fs.FileSystem`.
- - Add a factory that instantiates that file system and declares the scheme under which the FileSystem is registered. This must be a subclass of `org.apache.flink.core.fs.FileSystemFactory`.
- - Add a service entry. Create a file `META-INF/services/org.apache.flink.core.fs.FileSystemFactory` which contains the class name of your file system factory class
- (see the [Java Service Loader docs](https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html) for more details).
+ - 添加文件系统实现,它应是 `org.apache.flink.core.fs.FileSystem` 的子类。
+ - 添加 Factory 类,以实例化该文件系统并声明文件系统所注册的 scheme, 它应是 `org.apache.flink.core.fs.FileSystemFactory` 的子类。
+ - 添加 Service Entry。创建文件 `META-INF/services/org.apache.flink.core.fs.FileSystemFactory`,文件中包含文件系统 Factory 类的类名。
+ (更多细节请查看 [Java Service Loader docs](https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html))
-During plugins discovery, the file system factory class will be loaded by a dedicated Java class loader to avoid class conflicts with other plugins and Flink components.
-The same class loader should be used during file system instantiation and the file system operation calls.
+在插件检索时,文件系统 Factory 类会由一个专用的 Java 类加载器加载,从而避免与其他类或 Flink 组件冲突。在文件系统实例化和文件系统调用时,应使用该类加载器。
-Warning In practice, it means you should avoid using `Thread.currentThread().getContextClassLoader()` class loader
-in your implementation.
+警告 实际上这表示您的实现应避免使用 `Thread.currentThread().getContextClassLoader()` 类加载器。
-## Hadoop File System (HDFS) and its other implementations
+## Hadoop 文件系统 (HDFS) 及其其他实现
-For all schemes where Flink cannot find a directly supported file system, it falls back to Hadoop.
-All Hadoop file systems are automatically available when `flink-runtime` and the Hadoop libraries are on the classpath.
-See also **[Hadoop Integration]({{ site.baseurl }}/ops/deployment/hadoop.html)**.
+所有 Flink 无法找到直接支持的文件系统均将回退为 Hadoop。
+当 `flink-runtime` 和 Hadoop 类包含在 classpath 中时,所有的 Hadoop 文件系统将自动可用。参见 **[Hadoop 集成]({{ site.baseurl }}/zh/ops/deployment/hadoop.html)**。
-This way, Flink seamlessly supports all of Hadoop file systems implementing the `org.apache.hadoop.fs.FileSystem` interface,
-and all Hadoop-compatible file systems (HCFS).
-
- - HDFS (tested)
- - [Google Cloud Storage Connector for Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector) (tested)
- - [Alluxio](http://alluxio.org/) (tested, see configuration specifics below)
- - [XtreemFS](http://www.xtreemfs.org/) (tested)
- - FTP via [Hftp](http://hadoop.apache.org/docs/r1.2.1/hftp.html) (not tested)
- - HAR (not tested)
+因此,Flink 无缝支持所有实现 `org.apache.hadoop.fs.FileSystem` 接口的 Hadoop 文件系统和所有兼容 Hadoop 的文件系统 (Hadoop-compatible file system, HCFS):
+ - HDFS (已测试)
+ - [Google Cloud Storage Connector for Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector)(已测试)
+ - [Alluxio](http://alluxio.org/)(已测试,参见下文的配置详细信息)
+ - [XtreemFS](http://www.xtreemfs.org/)(已测试)
+ - FTP via [Hftp](http://hadoop.apache.org/docs/r1.2.1/hftp.html)(未测试)
+ - HAR(未测试)
- ...
-The Hadoop configuration has to have an entry for the required file system implementation in the `core-site.xml` file.
-See example for **[Alluxio]({{ site.baseurl }}/ops/filesystems/#alluxio)**.
+Hadoop 配置须在 `core-site.xml` 文件中包含所需文件系统的实现。可查看 **[Alluxio 的示例]({{ site.baseurl }}/zh/ops/filesystems/#alluxio)**。
-We recommend using Flink's built-in file systems unless required otherwise. Using a Hadoop File System directly may be required,
-for example, when using that file system for YARN's resource storage, via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`.
+除非有其他的需要,建议使用 Flink 内置的文件系统。在某些情况下,如通过配置 Hadoop `core-site.xml` 中的 `fs.defaultFS` 属性将文件系统作为 YARN 的资源存储时,可能需要直接使用 Hadoop 文件系统。
### Alluxio
-For Alluxio support add the following entry into the `core-site.xml` file:
+在 `core-site.xml` 文件中添加以下条目以支持 Alluxio:
{% highlight xml %}
diff --git a/docs/ops/filesystems/oss.md b/docs/ops/filesystems/oss.md
index 6b2be11cd5bb0b34581e4f2dc6e18c2aedc2b0c3..c34bd19a413a2be96cb9335b1c2af264a8e6ba36 100644
--- a/docs/ops/filesystems/oss.md
+++ b/docs/ops/filesystems/oss.md
@@ -52,7 +52,7 @@ env.setStateBackend(new FsStateBackend("oss:///"));
### Shaded Hadoop OSS file system
-To use `flink-oss-fs-hadoop,` copy the respective JAR file from the `opt` directory to a directory in `plugins` directory of your Flink distribution before starting Flink, e.g.
+To use `flink-oss-fs-hadoop`, copy the respective JAR file from the `opt` directory to a directory in `plugins` directory of your Flink distribution before starting Flink, e.g.
{% highlight bash %}
mkdir ./plugins/oss-fs-hadoop
@@ -82,7 +82,7 @@ An alternative `CredentialsProvider` can also be configured in the `flink-conf.y
# Read Credentials from OSS_ACCESS_KEY_ID and OSS_ACCESS_KEY_SECRET
fs.oss.credentials.provider: com.aliyun.oss.common.auth.EnvironmentVariableCredentialsProvider
{% endhighlight %}
-Other credential providers can be found under https://github.com/aliyun/aliyun-oss-java-sdk/tree/master/src/main/java/com/aliyun/oss/common/auth.
+Other credential providers can be found under [here](https://github.com/aliyun/aliyun-oss-java-sdk/tree/master/src/main/java/com/aliyun/oss/common/auth).
diff --git a/docs/ops/filesystems/oss.zh.md b/docs/ops/filesystems/oss.zh.md
index a93754662b0b36086dd83e89eef6e684659a744f..91bb1739aa23cf564f472d955d61e6e19ff5f103 100644
--- a/docs/ops/filesystems/oss.zh.md
+++ b/docs/ops/filesystems/oss.zh.md
@@ -1,6 +1,6 @@
---
-title: "Aliyun 对象存储服务 (OSS)"
-nav-title: Aliyun OSS
+title: "阿里云对象存储服务 (OSS)"
+nav-title: 阿里云 OSS
nav-parent_id: filesystems
nav-pos: 2
---
@@ -23,66 +23,66 @@ specific language governing permissions and limitations
under the License.
-->
-## OSS: Object Storage Service
+## OSS:对象存储服务
-[Aliyun Object Storage Service](https://www.aliyun.com/product/oss) (Aliyun OSS) is widely used, particularly popular among China’s cloud users, and it provides cloud object storage for a variety of use cases.
-You can use OSS with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl }}/ops/state/state_backends.html)
+[阿里云对象存储服务](https://www.aliyun.com/product/oss) (Aliyun OSS) 使用广泛,尤其在中国云用户中十分流行,能提供多种应用场景下的云对象存储。OSS 可与 Flink 一起使用以读取与存储数据,以及与[流 State Backend]({{ site.baseurl }}/zh/ops/state/state_backends.html) 结合使用。
* This will be replaced by the TOC
{:toc}
-You can use OSS objects like regular files by specifying paths in the following format:
+通过以下格式指定路径,OSS 对象可类似于普通文件使用:
{% highlight plain %}
oss:///
{% endhighlight %}
-Below shows how to use OSS in a Flink job:
+以下代码展示了如何在 Flink 作业中使用 OSS:
{% highlight java %}
-// Read from OSS bucket
+// 读取 OSS bucket
env.readTextFile("oss:///");
-// Write to OSS bucket
+// 写入 OSS bucket
stream.writeAsText("oss:///")
-// Use OSS as FsStatebackend
+// 将 OSS 用作 FsStatebackend
env.setStateBackend(new FsStateBackend("oss:///"));
{% endhighlight %}
-### Shaded Hadoop OSS file system
+### Shaded Hadoop OSS 文件系统
-To use `flink-oss-fs-hadoop,` copy the respective JAR file from the `opt` directory to a directory in `plugins` directory of your Flink distribution before starting Flink, e.g.
+为使用 `flink-oss-fs-hadoop`,在启动 Flink 之前,将对应的 JAR 文件从 `opt` 目录复制到 Flink 发行版中的 `plugin` 目录下的一个文件夹中,例如:
{% highlight bash %}
mkdir ./plugins/oss-fs-hadoop
cp ./opt/flink-oss-fs-hadoop-{{ site.version }}.jar ./plugins/oss-fs-hadoop/
{% endhighlight %}
-`flink-oss-fs-hadoop` registers default FileSystem wrappers for URIs with the *oss://* scheme.
+`flink-oss-fs-hadoop` 为使用 *oss://* scheme 的 URI 注册了默认的文件系统包装器。
-#### Configurations setup
+#### 配置设置
-After setting up the OSS FileSystem wrapper, you need to add some configurations to make sure that Flink is allowed to access your OSS buckets.
+在设置好 OSS 文件系统包装器之后,需要添加一些配置以保证 Flink 有权限访问 OSS buckets。
-To allow for easy adoption, you can use the same configuration keys in `flink-conf.yaml` as in Hadoop's `core-site.xml`
+为了简单使用,可直接在 `flink-conf.yaml` 中使用与 Hadoop `core-site.xml` 相同的配置关键字。
-You can see the configuration keys in the [Hadoop OSS documentation](http://hadoop.apache.org/docs/current/hadoop-aliyun/tools/hadoop-aliyun/index.html).
+可在 [Hadoop OSS 文档](http://hadoop.apache.org/docs/current/hadoop-aliyun/tools/hadoop-aliyun/index.html) 中查看配置关键字。
-There are some required configurations that must be added to `flink-conf.yaml` (**Other configurations defined in Hadoop OSS documentation are advanced configurations which used by performance tuning**):
+一些配置必须添加至 `flink-conf.yaml` (**在 Hadoop OSS 文档中定义的其它配置为用作性能调优的高级配置**):
{% highlight yaml %}
-fs.oss.endpoint: Aliyun OSS endpoint to connect to
+fs.oss.endpoint: 连接的 Aliyun OSS endpoint
fs.oss.accessKeyId: Aliyun access key ID
fs.oss.accessKeySecret: Aliyun access key secret
{% endhighlight %}
-An alternative `CredentialsProvider` can also be configured in the `flink-conf.yaml`, e.g.
+备选的 `CredentialsProvider` 也可在 `flink-conf.yaml` 中配置,例如:
{% highlight yaml %}
-# Read Credentials from OSS_ACCESS_KEY_ID and OSS_ACCESS_KEY_SECRET
+# 从 OSS_ACCESS_KEY_ID 和 OSS_ACCESS_KEY_SECRET 读取凭据 (Credentials)
fs.oss.credentials.provider: com.aliyun.oss.common.auth.EnvironmentVariableCredentialsProvider
{% endhighlight %}
-Other credential providers can be found under https://github.com/aliyun/aliyun-oss-java-sdk/tree/master/src/main/java/com/aliyun/oss/common/auth.
+
+其余的凭据提供者(credential providers)可在[这里](https://github.com/aliyun/aliyun-oss-java-sdk/tree/master/src/main/java/com/aliyun/oss/common/auth)中找到。
diff --git a/docs/ops/filesystems/s3.md b/docs/ops/filesystems/s3.md
index 4e371c3d3fd7ea81b8bfba58f8557b2b96d7e20b..7c47f1f67e7318e972fef3780b005a322b1fd79d 100644
--- a/docs/ops/filesystems/s3.md
+++ b/docs/ops/filesystems/s3.md
@@ -111,7 +111,7 @@ s3.endpoint: your-endpoint-hostname
## Configure Path Style Access
-Some of the S3 compliant object stores might not have virtual host style addressing enabled by default. In such cases, you will have to provide the property to enable path style access in in `flink-conf.yaml`.
+Some of the S3 compliant object stores might not have virtual host style addressing enabled by default. In such cases, you will have to provide the property to enable path style access in `flink-conf.yaml`.
{% highlight yaml %}
s3.path.style.access: true
diff --git a/docs/ops/filesystems/s3.zh.md b/docs/ops/filesystems/s3.zh.md
index 4e371c3d3fd7ea81b8bfba58f8557b2b96d7e20b..5bfd39d01df3c67a3c53e28f34c2d746b56e5b64 100644
--- a/docs/ops/filesystems/s3.zh.md
+++ b/docs/ops/filesystems/s3.zh.md
@@ -23,114 +23,104 @@ specific language governing permissions and limitations
under the License.
-->
-[Amazon Simple Storage Service](http://aws.amazon.com/s3/) (Amazon S3) provides cloud object storage for a variety of use cases. You can use S3 with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl}}/ops/state/state_backends.html).
+[Amazon Simple Storage Service](http://aws.amazon.com/s3/) (Amazon S3) 提供用于多种场景的云对象存储。S3 可与 Flink 一起使用以读取、写入数据,并可与 [流的 **State backends**]({{ site.baseurl}}/ops/state/state_backends.html) 相结合使用。
* This will be replaced by the TOC
{:toc}
-You can use S3 objects like regular files by specifying paths in the following format:
+通过以下格式指定路径,S3 对象可类似于普通文件使用:
{% highlight plain %}
s3://